From 8abf6c914966a4006d6b0089c8b3a8f42e608e1c Mon Sep 17 00:00:00 2001 From: Spade A <71589810+SpadeA-Tang@users.noreply.github.com> Date: Thu, 9 Jan 2025 15:24:56 +0800 Subject: [PATCH 01/34] fix: build text index when loading field data (#39070) fix: https://github.com/milvus-io/milvus/issues/39053 may fix https://github.com/milvus-io/milvus/issues/38644 which could be caused by https://github.com/milvus-io/milvus/issues/39053 --------- Signed-off-by: SpadeA-Tang --- internal/core/src/index/TextMatchIndex.cpp | 33 ++++++++++ internal/core/src/index/TextMatchIndex.h | 4 ++ .../core/src/segcore/SegmentGrowingImpl.cpp | 10 +++ internal/core/unittest/test_text_match.cpp | 63 +++++++++++++++++++ 4 files changed, 110 insertions(+) diff --git a/internal/core/src/index/TextMatchIndex.cpp b/internal/core/src/index/TextMatchIndex.cpp index b3d8e3beee729..662295ddb4cd6 100644 --- a/internal/core/src/index/TextMatchIndex.cpp +++ b/internal/core/src/index/TextMatchIndex.cpp @@ -198,6 +198,39 @@ TextMatchIndex::AddTexts(size_t n, } } +// schema_ may not be initialized so we need this `nullable` parameter +void +TextMatchIndex::BuildIndexFromFieldData( + const std::vector& field_datas, bool nullable) { + int64_t offset = 0; + if (nullable) { + int64_t total = 0; + for (const auto& data : field_datas) { + total += data->get_null_count(); + } + null_offset.reserve(total); + for (const auto& data : field_datas) { + auto n = data->get_num_rows(); + for (int i = 0; i < n; i++) { + if (!data->is_valid(i)) { + null_offset.push_back(i); + } + wrapper_->add_data( + static_cast(data->RawValue(i)), + data->is_valid(i) ? 1 : 0, + offset++); + } + } + } else { + for (const auto& data : field_datas) { + auto n = data->get_num_rows(); + wrapper_->add_data( + static_cast(data->Data()), n, offset); + offset += n; + } + } +} + void TextMatchIndex::Finish() { finish(); diff --git a/internal/core/src/index/TextMatchIndex.h b/internal/core/src/index/TextMatchIndex.h index 5549cc1ee2767..17d14340fc70d 100644 --- a/internal/core/src/index/TextMatchIndex.h +++ b/internal/core/src/index/TextMatchIndex.h @@ -58,6 +58,10 @@ class TextMatchIndex : public InvertedIndexTantivy { const bool* valids, int64_t offset_begin); + void + BuildIndexFromFieldData(const std::vector& field_datas, + bool nullable); + void Finish(); diff --git a/internal/core/src/segcore/SegmentGrowingImpl.cpp b/internal/core/src/segcore/SegmentGrowingImpl.cpp index ccc793fbb65b0..4817df64eaa69 100644 --- a/internal/core/src/segcore/SegmentGrowingImpl.cpp +++ b/internal/core/src/segcore/SegmentGrowingImpl.cpp @@ -274,6 +274,16 @@ SegmentGrowingImpl::LoadFieldData(const LoadFieldDataInfo& infos) { storage::GetByteSizeOfFieldDatas(field_data)); } + // build text match index + if (field_meta.enable_match()) { + auto index = GetTextIndex(field_id); + index->BuildIndexFromFieldData(field_data, + field_meta.is_nullable()); + index->Commit(); + // Reload reader so that the index can be read immediately + index->Reload(); + } + // update the mem size stats_.mem_size += storage::GetByteSizeOfFieldDatas(field_data); diff --git a/internal/core/unittest/test_text_match.cpp b/internal/core/unittest/test_text_match.cpp index a7e9260c95001..f7b6366385e1c 100644 --- a/internal/core/unittest/test_text_match.cpp +++ b/internal/core/unittest/test_text_match.cpp @@ -20,6 +20,8 @@ #include "query/PlanProto.h" #include "query/ExecPlanNodeVisitor.h" #include "expr/ITypeExpr.h" +#include "segcore/segment_c.h" +#include "test_utils/storage_test_utils.h" using namespace milvus; using namespace milvus::query; @@ -751,3 +753,64 @@ TEST(TextMatch, SealedJieBaNullable) { ASSERT_FALSE(final[2]); } } + +// Test that growing segment loading flushed binlogs will build text match index. +TEST(TextMatch, GrowingLoadData) { + int64_t N = 7; + auto schema = GenTestSchema({}, true); + schema->AddField(FieldName("RowID"), FieldId(0), DataType::INT64, false); + schema->AddField( + FieldName("Timestamp"), FieldId(1), DataType::INT64, false); + std::vector raw_str = {"football, basketball, pingpang", + "swimming, football", + "golf", + "", + "baseball", + "kungfu, football", + ""}; + auto raw_data = DataGen(schema, N); + auto str_col = raw_data.raw_->mutable_fields_data() + ->at(1) + .mutable_scalars() + ->mutable_string_data() + ->mutable_data(); + for (int64_t i = 0; i < N; i++) { + str_col->at(i) = raw_str[i]; + } + auto str_col_valid = + raw_data.raw_->mutable_fields_data()->at(1).mutable_valid_data(); + for (int64_t i = 0; i < N; i++) { + str_col_valid->at(i) = true; + } + // so we cannot match the second row + str_col_valid->at(1) = false; + + auto storage_config = get_default_local_storage_config(); + auto cm = storage::CreateChunkManager(storage_config); + auto load_info = PrepareInsertBinlog( + 1, + 2, + 3, + storage_config.root_path + "/" + "test_growing_segment_load_data", + raw_data, + cm); + + auto segment = CreateGrowingSegment(schema, empty_index_meta); + auto status = LoadFieldData(segment.get(), &load_info); + ASSERT_EQ(status.error_code, Success); + ASSERT_EQ(segment->get_real_count(), N); + ASSERT_NE(segment->get_field_avg_size(FieldId(101)), 0); + + // Check whether the text index has been built. + auto expr = GetTextMatchExpr(schema, "football"); + BitsetType final; + final = ExecuteQueryExpr(expr, segment.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_TRUE(final[0]); + ASSERT_FALSE(final[1]); + ASSERT_FALSE(final[2]); + ASSERT_FALSE(final[3]); + ASSERT_FALSE(final[4]); + ASSERT_TRUE(final[5]); + ASSERT_FALSE(final[6]); +} \ No newline at end of file From d6206ad2de05b12d99c389a6cbfda54838560e4e Mon Sep 17 00:00:00 2001 From: Cai Yudong Date: Thu, 9 Jan 2025 15:26:56 +0800 Subject: [PATCH 02/34] fix: Remove duplicated Macro definition (#39076) Issue: #39102 Signed-off-by: Cai Yudong --- internal/core/src/common/File.h | 8 ++++---- internal/core/src/mmap/Utils.h | 10 ++-------- 2 files changed, 6 insertions(+), 12 deletions(-) diff --git a/internal/core/src/common/File.h b/internal/core/src/common/File.h index 4015df78e6473..f2e6149027d02 100644 --- a/internal/core/src/common/File.h +++ b/internal/core/src/common/File.h @@ -22,10 +22,10 @@ namespace milvus { -#define THROW_FILE_WRITE_ERROR \ +#define THROW_FILE_WRITE_ERROR(path) \ PanicInfo(ErrorCode::FileWriteFailed, \ fmt::format("write data to file {} failed, error code {}", \ - file_.Path(), \ + path, \ strerror(errno))); class File { @@ -145,7 +145,7 @@ class BufferedWriter { flush(); ssize_t written_data_size = file_.FWrite(data, size); if (written_data_size != size) { - THROW_FILE_WRITE_ERROR + THROW_FILE_WRITE_ERROR(file_.Path()) } return; } @@ -170,7 +170,7 @@ class BufferedWriter { if (buffer_pos_ > 0) { ssize_t written_data_size = file_.FWrite(buffer_, buffer_pos_); if (written_data_size != buffer_pos_) { - THROW_FILE_WRITE_ERROR + THROW_FILE_WRITE_ERROR(file_.Path()) } buffer_pos_ = 0; } diff --git a/internal/core/src/mmap/Utils.h b/internal/core/src/mmap/Utils.h index fc4892dbfa0b4..b11e105352665 100644 --- a/internal/core/src/mmap/Utils.h +++ b/internal/core/src/mmap/Utils.h @@ -33,12 +33,6 @@ namespace milvus { -#define THROW_FILE_WRITE_ERROR \ - PanicInfo(ErrorCode::FileWriteFailed, \ - fmt::format("write data to file {} failed, error code {}", \ - file.Path(), \ - strerror(errno))); - /* * If string field's value all empty, need a string padding to avoid * mmap failing because size_ is zero which causing invalid argument @@ -78,7 +72,7 @@ WriteFieldPadding(File& file, DataType data_type, uint64_t& total_written) { std::vector padding(padding_size, 0); ssize_t written = file.Write(padding.data(), padding_size); if (written < padding_size) { - THROW_FILE_WRITE_ERROR + THROW_FILE_WRITE_ERROR(file.Path()) } total_written += written; } @@ -163,7 +157,7 @@ WriteFieldData(File& file, // write as: data|data|data|data|data|data...... size_t written = file.FWrite(data->Data(), data->DataSize()); if (written < data->DataSize()) { - THROW_FILE_WRITE_ERROR + THROW_FILE_WRITE_ERROR(file.Path()) } for (auto i = 0; i < data->get_num_rows(); i++) { indices.emplace_back(total_written); From 3bcdd92915c0eea63d16c47d30a7bcf5a7469610 Mon Sep 17 00:00:00 2001 From: Zhen Ye Date: Thu, 9 Jan 2025 16:24:55 +0800 Subject: [PATCH 03/34] enhance: add broadcast for streaming service (#39020) issue: #38399 - Add new rpc for transfer broadcast to streaming coord - Add broadcast service at streaming coord to make broadcast message sent automicly Signed-off-by: chyezh --- internal/.mockery.yaml | 12 +- internal/distributed/streaming/append.go | 17 +- internal/distributed/streaming/streaming.go | 5 + .../distributed/streaming/streaming_test.go | 38 ++-- internal/distributed/streaming/wal.go | 26 ++- internal/distributed/streaming/wal_test.go | 54 ++++- internal/metastore/catalog.go | 9 + .../metastore/kv/streamingcoord/constant.go | 5 +- .../metastore/kv/streamingcoord/kv_catalog.go | 47 +++- .../kv/streamingcoord/kv_catalog_test.go | 59 ++++- .../mock_streaming/mock_WALAccesser.go | 59 +++++ .../mock_StreamingCoordCataLog.go | 105 +++++++++ .../mock_client/mock_BroadcastService.go | 98 +++++++++ .../streamingcoord/mock_client/mock_Client.go | 47 ++++ .../mock_broadcaster/mock_AppendOperator.go | 100 +++++++++ .../client/broadcast/broadcast_impl.go | 56 +++++ internal/streamingcoord/client/client.go | 13 ++ internal/streamingcoord/client/client_impl.go | 6 + .../server/broadcaster/append_operator.go | 14 ++ .../server/broadcaster/broadcaster.go | 24 ++ .../server/broadcaster/broadcaster_impl.go | 207 ++++++++++++++++++ .../server/broadcaster/broadcaster_test.go | 142 ++++++++++++ .../streamingcoord/server/broadcaster/task.go | 126 +++++++++++ internal/streamingcoord/server/builder.go | 4 + .../server/resource/resource.go | 9 + internal/streamingcoord/server/server.go | 33 ++- .../server/service/broadcast.go | 44 ++++ .../flusher/flusherimpl/flusher_impl.go | 2 +- .../streamingnode/server/resource/resource.go | 2 +- .../server/resource/test_utility.go | 2 +- .../handler/producer/produce_server.go | 16 +- .../server/wal/adaptor/wal_test.go | 2 +- .../segment/manager/pchannel_manager_test.go | 2 +- .../interceptors/timetick/timetick_message.go | 2 +- .../resource => util}/idalloc/allocator.go | 0 .../idalloc/allocator_test.go | 0 .../idalloc/basic_allocator.go | 0 .../idalloc/basic_allocator_test.go | 0 .../resource => util}/idalloc/mallocator.go | 0 .../idalloc/test_mock_root_coord_client.go | 0 pkg/streaming/proto/messages.proto | 5 + pkg/streaming/proto/streaming.proto | 38 +++- pkg/streaming/util/message/builder.go | 90 +++++++- pkg/streaming/util/message/message.go | 28 ++- pkg/streaming/util/message/message_impl.go | 59 ++++- pkg/streaming/util/message/properties.go | 1 + pkg/streaming/util/types/streaming_node.go | 23 ++ pkg/util/contextutil/context_util.go | 12 + pkg/util/typeutil/backoff_timer.go | 46 ++++ 49 files changed, 1596 insertions(+), 93 deletions(-) create mode 100644 internal/mocks/streamingcoord/mock_client/mock_BroadcastService.go create mode 100644 internal/mocks/streamingcoord/server/mock_broadcaster/mock_AppendOperator.go create mode 100644 internal/streamingcoord/client/broadcast/broadcast_impl.go create mode 100644 internal/streamingcoord/server/broadcaster/append_operator.go create mode 100644 internal/streamingcoord/server/broadcaster/broadcaster.go create mode 100644 internal/streamingcoord/server/broadcaster/broadcaster_impl.go create mode 100644 internal/streamingcoord/server/broadcaster/broadcaster_test.go create mode 100644 internal/streamingcoord/server/broadcaster/task.go create mode 100644 internal/streamingcoord/server/service/broadcast.go rename internal/{streamingnode/server/resource => util}/idalloc/allocator.go (100%) rename internal/{streamingnode/server/resource => util}/idalloc/allocator_test.go (100%) rename internal/{streamingnode/server/resource => util}/idalloc/basic_allocator.go (100%) rename internal/{streamingnode/server/resource => util}/idalloc/basic_allocator_test.go (100%) rename internal/{streamingnode/server/resource => util}/idalloc/mallocator.go (100%) rename internal/{streamingnode/server/resource => util}/idalloc/test_mock_root_coord_client.go (100%) diff --git a/internal/.mockery.yaml b/internal/.mockery.yaml index 2179959e87d3a..8804105c23322 100644 --- a/internal/.mockery.yaml +++ b/internal/.mockery.yaml @@ -12,12 +12,16 @@ packages: github.com/milvus-io/milvus/internal/streamingcoord/server/balancer: interfaces: Balancer: - github.com/milvus-io/milvus/internal/streamingnode/client/manager: + github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster: interfaces: - ManagerClient: + AppendOperator: github.com/milvus-io/milvus/internal/streamingcoord/client: interfaces: Client: + BroadcastService: + github.com/milvus-io/milvus/internal/streamingnode/client/manager: + interfaces: + ManagerClient: github.com/milvus-io/milvus/internal/streamingnode/client/handler: interfaces: HandlerClient: @@ -46,10 +50,10 @@ packages: InterceptorWithReady: InterceptorBuilder: github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/segment/inspector: - interfaces: + interfaces: SealOperator: github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/timetick/inspector: - interfaces: + interfaces: TimeTickSyncOperator: google.golang.org/grpc: interfaces: diff --git a/internal/distributed/streaming/append.go b/internal/distributed/streaming/append.go index 2fd0820e549b2..b4193d8b94446 100644 --- a/internal/distributed/streaming/append.go +++ b/internal/distributed/streaming/append.go @@ -17,6 +17,12 @@ func (w *walAccesserImpl) appendToWAL(ctx context.Context, msg message.MutableMe return p.Produce(ctx, msg) } +func (w *walAccesserImpl) broadcastToWAL(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) { + // The broadcast operation will be sent to the coordinator. + // The coordinator will dispatch the message to all the vchannels with an eventual consistency guarantee. + return w.streamingCoordClient.Broadcast().Broadcast(ctx, msg) +} + // createOrGetProducer creates or get a producer. // vchannel in same pchannel can share the same producer. func (w *walAccesserImpl) getProducer(pchannel string) *producer.ResumableProducer { @@ -40,14 +46,19 @@ func assertValidMessage(msgs ...message.MutableMessage) { if msg.MessageType().IsSystem() { panic("system message is not allowed to append from client") } - } - for _, msg := range msgs { if msg.VChannel() == "" { - panic("vchannel is empty") + panic("we don't support sent all vchannel message at client now") } } } +// assertValidBroadcastMessage asserts the message is not system message. +func assertValidBroadcastMessage(msg message.BroadcastMutableMessage) { + if msg.MessageType().IsSystem() { + panic("system message is not allowed to broadcast append from client") + } +} + // We only support delete and insert message for txn now. func assertIsDmlMessage(msgs ...message.MutableMessage) { for _, msg := range msgs { diff --git a/internal/distributed/streaming/streaming.go b/internal/distributed/streaming/streaming.go index 810b15065d6a3..efd77d5f2ac34 100644 --- a/internal/distributed/streaming/streaming.go +++ b/internal/distributed/streaming/streaming.go @@ -78,6 +78,7 @@ type Scanner interface { // WALAccesser is the interfaces to interact with the milvus write ahead log. type WALAccesser interface { + // WALName returns the name of the wal. WALName() string // Txn returns a transaction for writing records to the log. @@ -87,6 +88,10 @@ type WALAccesser interface { // RawAppend writes a records to the log. RawAppend(ctx context.Context, msgs message.MutableMessage, opts ...AppendOption) (*types.AppendResult, error) + // BroadcastAppend sends a broadcast message to all target vchannels. + // BroadcastAppend guarantees the atomicity written of the messages and eventual consistency. + BroadcastAppend(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) + // Read returns a scanner for reading records from the wal. Read(ctx context.Context, opts ReadOption) Scanner diff --git a/internal/distributed/streaming/streaming_test.go b/internal/distributed/streaming/streaming_test.go index c24f65261636d..e44e18e7c2005 100644 --- a/internal/distributed/streaming/streaming_test.go +++ b/internal/distributed/streaming/streaming_test.go @@ -14,7 +14,10 @@ import ( "github.com/milvus-io/milvus/pkg/util/paramtable" ) -const vChannel = "by-dev-rootcoord-dml_4" +var vChannels = []string{ + "by-dev-rootcoord-dml_4", + "by-dev-rootcoord-dml_5", +} func TestMain(m *testing.M) { paramtable.Init() @@ -33,10 +36,11 @@ func TestStreamingProduce(t *testing.T) { WithBody(&msgpb.CreateCollectionRequest{ CollectionID: 1, }). - WithVChannel(vChannel). - BuildMutable() - resp, err := streaming.WAL().RawAppend(context.Background(), msg) - fmt.Printf("%+v\t%+v\n", resp, err) + WithBroadcast(vChannels). + BuildBroadcast() + + resp, err := streaming.WAL().BroadcastAppend(context.Background(), msg) + t.Logf("CreateCollection: %+v\t%+v\n", resp, err) for i := 0; i < 500; i++ { time.Sleep(time.Millisecond * 1) @@ -47,17 +51,17 @@ func TestStreamingProduce(t *testing.T) { WithBody(&msgpb.InsertRequest{ CollectionID: 1, }). - WithVChannel(vChannel). + WithVChannel(vChannels[0]). BuildMutable() resp, err := streaming.WAL().RawAppend(context.Background(), msg) - fmt.Printf("%+v\t%+v\n", resp, err) + t.Logf("Insert: %+v\t%+v\n", resp, err) } for i := 0; i < 500; i++ { time.Sleep(time.Millisecond * 1) txn, err := streaming.WAL().Txn(context.Background(), streaming.TxnOption{ - VChannel: vChannel, - Keepalive: 100 * time.Millisecond, + VChannel: vChannels[0], + Keepalive: 500 * time.Millisecond, }) if err != nil { t.Errorf("txn failed: %v", err) @@ -71,7 +75,7 @@ func TestStreamingProduce(t *testing.T) { WithBody(&msgpb.InsertRequest{ CollectionID: 1, }). - WithVChannel(vChannel). + WithVChannel(vChannels[0]). BuildMutable() err := txn.Append(context.Background(), msg) fmt.Printf("%+v\n", err) @@ -80,7 +84,7 @@ func TestStreamingProduce(t *testing.T) { if err != nil { t.Errorf("txn failed: %v", err) } - fmt.Printf("%+v\n", result) + t.Logf("txn commit: %+v\n", result) } msg, _ = message.NewDropCollectionMessageBuilderV1(). @@ -90,10 +94,10 @@ func TestStreamingProduce(t *testing.T) { WithBody(&msgpb.DropCollectionRequest{ CollectionID: 1, }). - WithVChannel(vChannel). - BuildMutable() - resp, err = streaming.WAL().RawAppend(context.Background(), msg) - fmt.Printf("%+v\t%+v\n", resp, err) + WithBroadcast(vChannels). + BuildBroadcast() + resp, err = streaming.WAL().BroadcastAppend(context.Background(), msg) + t.Logf("DropCollection: %+v\t%+v\n", resp, err) } func TestStreamingConsume(t *testing.T) { @@ -102,7 +106,7 @@ func TestStreamingConsume(t *testing.T) { defer streaming.Release() ch := make(message.ChanMessageHandler, 10) s := streaming.WAL().Read(context.Background(), streaming.ReadOption{ - VChannel: vChannel, + VChannel: vChannels[0], DeliverPolicy: options.DeliverPolicyAll(), MessageHandler: ch, }) @@ -115,7 +119,7 @@ func TestStreamingConsume(t *testing.T) { time.Sleep(10 * time.Millisecond) select { case msg := <-ch: - fmt.Printf("msgID=%+v, msgType=%+v, tt=%d, lca=%+v, body=%s, idx=%d\n", + t.Logf("msgID=%+v, msgType=%+v, tt=%d, lca=%+v, body=%s, idx=%d\n", msg.MessageID(), msg.MessageType(), msg.TimeTick(), diff --git a/internal/distributed/streaming/wal.go b/internal/distributed/streaming/wal.go index b4d7fb5f90080..f721f2d63bfdf 100644 --- a/internal/distributed/streaming/wal.go +++ b/internal/distributed/streaming/wal.go @@ -29,11 +29,11 @@ func newWALAccesser(c *clientv3.Client) *walAccesserImpl { // Create a new streamingnode handler client. handlerClient := handler.NewHandlerClient(streamingCoordClient.Assignment()) return &walAccesserImpl{ - lifetime: typeutil.NewLifetime(), - streamingCoordAssignmentClient: streamingCoordClient, - handlerClient: handlerClient, - producerMutex: sync.Mutex{}, - producers: make(map[string]*producer.ResumableProducer), + lifetime: typeutil.NewLifetime(), + streamingCoordClient: streamingCoordClient, + handlerClient: handlerClient, + producerMutex: sync.Mutex{}, + producers: make(map[string]*producer.ResumableProducer), // TODO: optimize the pool size, use the streaming api but not goroutines. appendExecutionPool: conc.NewPool[struct{}](10), @@ -46,8 +46,8 @@ type walAccesserImpl struct { lifetime *typeutil.Lifetime // All services - streamingCoordAssignmentClient client.Client - handlerClient handler.HandlerClient + streamingCoordClient client.Client + handlerClient handler.HandlerClient producerMutex sync.Mutex producers map[string]*producer.ResumableProducer @@ -71,6 +71,16 @@ func (w *walAccesserImpl) RawAppend(ctx context.Context, msg message.MutableMess return w.appendToWAL(ctx, msg) } +func (w *walAccesserImpl) BroadcastAppend(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) { + assertValidBroadcastMessage(msg) + if !w.lifetime.Add(typeutil.LifetimeStateWorking) { + return nil, ErrWALAccesserClosed + } + defer w.lifetime.Done() + + return w.broadcastToWAL(ctx, msg) +} + // Read returns a scanner for reading records from the wal. func (w *walAccesserImpl) Read(_ context.Context, opts ReadOption) Scanner { if !w.lifetime.Add(typeutil.LifetimeStateWorking) { @@ -149,7 +159,7 @@ func (w *walAccesserImpl) Close() { w.producerMutex.Unlock() w.handlerClient.Close() - w.streamingCoordAssignmentClient.Close() + w.streamingCoordClient.Close() } // newErrScanner creates a scanner that returns an error. diff --git a/internal/distributed/streaming/wal_test.go b/internal/distributed/streaming/wal_test.go index db527c044eddb..a850b9cce3a07 100644 --- a/internal/distributed/streaming/wal_test.go +++ b/internal/distributed/streaming/wal_test.go @@ -30,19 +30,33 @@ const ( func TestWAL(t *testing.T) { coordClient := mock_client.NewMockClient(t) coordClient.EXPECT().Close().Return() + broadcastServce := mock_client.NewMockBroadcastService(t) + broadcastServce.EXPECT().Broadcast(mock.Anything, mock.Anything).RunAndReturn( + func(ctx context.Context, bmm message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) { + result := make(map[string]*types.AppendResult) + for idx, msg := range bmm.SplitIntoMutableMessage() { + result[msg.VChannel()] = &types.AppendResult{ + MessageID: walimplstest.NewTestMessageID(int64(idx)), + TimeTick: uint64(time.Now().UnixMilli()), + } + } + return &types.BroadcastAppendResult{ + AppendResults: result, + }, nil + }) + coordClient.EXPECT().Broadcast().Return(broadcastServce) handler := mock_handler.NewMockHandlerClient(t) handler.EXPECT().Close().Return() w := &walAccesserImpl{ - lifetime: typeutil.NewLifetime(), - streamingCoordAssignmentClient: coordClient, - handlerClient: handler, - producerMutex: sync.Mutex{}, - producers: make(map[string]*producer.ResumableProducer), - appendExecutionPool: conc.NewPool[struct{}](10), - dispatchExecutionPool: conc.NewPool[struct{}](10), + lifetime: typeutil.NewLifetime(), + streamingCoordClient: coordClient, + handlerClient: handler, + producerMutex: sync.Mutex{}, + producers: make(map[string]*producer.ResumableProducer), + appendExecutionPool: conc.NewPool[struct{}](10), + dispatchExecutionPool: conc.NewPool[struct{}](10), } - defer w.Close() ctx := context.Background() @@ -114,6 +128,18 @@ func TestWAL(t *testing.T) { newInsertMessage(vChannel3), ) assert.NoError(t, resp.UnwrapFirstError()) + + r, err := w.BroadcastAppend(ctx, newBroadcastMessage([]string{vChannel1, vChannel2, vChannel3})) + assert.NoError(t, err) + assert.Len(t, r.AppendResults, 3) + + w.Close() + + resp = w.AppendMessages(ctx, newInsertMessage(vChannel1)) + assert.Error(t, resp.UnwrapFirstError()) + r, err = w.BroadcastAppend(ctx, newBroadcastMessage([]string{vChannel1, vChannel2, vChannel3})) + assert.Error(t, err) + assert.Nil(t, r) } func newInsertMessage(vChannel string) message.MutableMessage { @@ -127,3 +153,15 @@ func newInsertMessage(vChannel string) message.MutableMessage { } return msg } + +func newBroadcastMessage(vchannels []string) message.BroadcastMutableMessage { + msg, err := message.NewDropCollectionMessageBuilderV1(). + WithBroadcast(vchannels). + WithHeader(&message.DropCollectionMessageHeader{}). + WithBody(&msgpb.DropCollectionRequest{}). + BuildBroadcast() + if err != nil { + panic(err) + } + return msg +} diff --git a/internal/metastore/catalog.go b/internal/metastore/catalog.go index 090296d11bf1d..c7a2042dd73f0 100644 --- a/internal/metastore/catalog.go +++ b/internal/metastore/catalog.go @@ -210,6 +210,15 @@ type StreamingCoordCataLog interface { // SavePChannel save a pchannel info to metastore. SavePChannels(ctx context.Context, info []*streamingpb.PChannelMeta) error + + // ListBroadcastTask list all broadcast tasks. + // Used to recovery the broadcast tasks. + ListBroadcastTask(ctx context.Context) ([]*streamingpb.BroadcastTask, error) + + // SaveBroadcastTask save the broadcast task to metastore. + // Make the task recoverable after restart. + // When broadcast task is done, it will be removed from metastore. + SaveBroadcastTask(ctx context.Context, task *streamingpb.BroadcastTask) error } // StreamingNodeCataLog is the interface for streamingnode catalog diff --git a/internal/metastore/kv/streamingcoord/constant.go b/internal/metastore/kv/streamingcoord/constant.go index 5ae1f85b7d6bc..1f92dc9977da5 100644 --- a/internal/metastore/kv/streamingcoord/constant.go +++ b/internal/metastore/kv/streamingcoord/constant.go @@ -1,6 +1,7 @@ package streamingcoord const ( - MetaPrefix = "streamingcoord-meta" - PChannelMeta = MetaPrefix + "/pchannel" + MetaPrefix = "streamingcoord-meta/" + PChannelMetaPrefix = MetaPrefix + "pchannel/" + BroadcastTaskPrefix = MetaPrefix + "broadcast-task/" ) diff --git a/internal/metastore/kv/streamingcoord/kv_catalog.go b/internal/metastore/kv/streamingcoord/kv_catalog.go index d3d804052e026..c0a16a525106e 100644 --- a/internal/metastore/kv/streamingcoord/kv_catalog.go +++ b/internal/metastore/kv/streamingcoord/kv_catalog.go @@ -2,6 +2,7 @@ package streamingcoord import ( "context" + "strconv" "github.com/cockroachdb/errors" "google.golang.org/protobuf/proto" @@ -14,6 +15,14 @@ import ( ) // NewCataLog creates a new catalog instance +// streamingcoord-meta +// ├── broadcast +// │   ├── task-1 +// │   └── task-2 +// └── pchannel +// +// ├── pchannel-1 +// └── pchannel-2 func NewCataLog(metaKV kv.MetaKv) metastore.StreamingCoordCataLog { return &catalog{ metaKV: metaKV, @@ -27,7 +36,7 @@ type catalog struct { // ListPChannels returns all pchannels func (c *catalog) ListPChannel(ctx context.Context) ([]*streamingpb.PChannelMeta, error) { - keys, values, err := c.metaKV.LoadWithPrefix(ctx, PChannelMeta) + keys, values, err := c.metaKV.LoadWithPrefix(ctx, PChannelMetaPrefix) if err != nil { return nil, err } @@ -60,7 +69,41 @@ func (c *catalog) SavePChannels(ctx context.Context, infos []*streamingpb.PChann }) } +func (c *catalog) ListBroadcastTask(ctx context.Context) ([]*streamingpb.BroadcastTask, error) { + keys, values, err := c.metaKV.LoadWithPrefix(ctx, BroadcastTaskPrefix) + if err != nil { + return nil, err + } + infos := make([]*streamingpb.BroadcastTask, 0, len(values)) + for k, value := range values { + info := &streamingpb.BroadcastTask{} + err = proto.Unmarshal([]byte(value), info) + if err != nil { + return nil, errors.Wrapf(err, "unmarshal broadcast task %s failed", keys[k]) + } + infos = append(infos, info) + } + return infos, nil +} + +func (c *catalog) SaveBroadcastTask(ctx context.Context, task *streamingpb.BroadcastTask) error { + key := buildBroadcastTaskPath(task.TaskId) + if task.State == streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_DONE { + return c.metaKV.Remove(ctx, key) + } + v, err := proto.Marshal(task) + if err != nil { + return errors.Wrapf(err, "marshal broadcast task failed") + } + return c.metaKV.Save(ctx, key, string(v)) +} + // buildPChannelInfoPath builds the path for pchannel info. func buildPChannelInfoPath(name string) string { - return PChannelMeta + "/" + name + return PChannelMetaPrefix + name +} + +// buildBroadcastTaskPath builds the path for broadcast task. +func buildBroadcastTaskPath(id int64) string { + return BroadcastTaskPrefix + strconv.FormatInt(id, 10) } diff --git a/internal/metastore/kv/streamingcoord/kv_catalog_test.go b/internal/metastore/kv/streamingcoord/kv_catalog_test.go index 227ad0469bca3..215aee3d15ee3 100644 --- a/internal/metastore/kv/streamingcoord/kv_catalog_test.go +++ b/internal/metastore/kv/streamingcoord/kv_catalog_test.go @@ -2,6 +2,7 @@ package streamingcoord import ( "context" + "strings" "testing" "github.com/cockroachdb/errors" @@ -20,8 +21,10 @@ func TestCatalog(t *testing.T) { keys := make([]string, 0, len(kvStorage)) vals := make([]string, 0, len(kvStorage)) for k, v := range kvStorage { - keys = append(keys, k) - vals = append(vals, v) + if strings.HasPrefix(k, s) { + keys = append(keys, k) + vals = append(vals, v) + } } return keys, vals, nil }) @@ -31,12 +34,21 @@ func TestCatalog(t *testing.T) { } return nil }) + kv.EXPECT().Save(mock.Anything, mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, key, value string) error { + kvStorage[key] = value + return nil + }) + kv.EXPECT().Remove(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, key string) error { + delete(kvStorage, key) + return nil + }) catalog := NewCataLog(kv) metas, err := catalog.ListPChannel(context.Background()) assert.NoError(t, err) assert.Empty(t, metas) + // PChannel test err = catalog.SavePChannels(context.Background(), []*streamingpb.PChannelMeta{ { Channel: &streamingpb.PChannelInfo{Name: "test", Term: 1}, @@ -53,6 +65,37 @@ func TestCatalog(t *testing.T) { assert.NoError(t, err) assert.Len(t, metas, 2) + // BroadcastTask test + err = catalog.SaveBroadcastTask(context.Background(), &streamingpb.BroadcastTask{ + TaskId: 1, + State: streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING, + }) + assert.NoError(t, err) + err = catalog.SaveBroadcastTask(context.Background(), &streamingpb.BroadcastTask{ + TaskId: 2, + State: streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING, + }) + assert.NoError(t, err) + + tasks, err := catalog.ListBroadcastTask(context.Background()) + assert.NoError(t, err) + assert.Len(t, tasks, 2) + for _, task := range tasks { + assert.Equal(t, streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING, task.State) + } + + err = catalog.SaveBroadcastTask(context.Background(), &streamingpb.BroadcastTask{ + TaskId: 1, + State: streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_DONE, + }) + assert.NoError(t, err) + tasks, err = catalog.ListBroadcastTask(context.Background()) + assert.NoError(t, err) + assert.Len(t, tasks, 1) + for _, task := range tasks { + assert.Equal(t, streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING, task.State) + } + // error path. kv.EXPECT().LoadWithPrefix(mock.Anything, mock.Anything).Unset() kv.EXPECT().LoadWithPrefix(mock.Anything, mock.Anything).Return(nil, nil, errors.New("load error")) @@ -60,7 +103,19 @@ func TestCatalog(t *testing.T) { assert.Error(t, err) assert.Nil(t, metas) + tasks, err = catalog.ListBroadcastTask(context.Background()) + assert.Error(t, err) + assert.Nil(t, tasks) + kv.EXPECT().MultiSave(mock.Anything, mock.Anything).Unset() kv.EXPECT().MultiSave(mock.Anything, mock.Anything).Return(errors.New("save error")) + kv.EXPECT().Save(mock.Anything, mock.Anything, mock.Anything).Unset() + kv.EXPECT().Save(mock.Anything, mock.Anything, mock.Anything).Return(errors.New("save error")) + err = catalog.SavePChannels(context.Background(), []*streamingpb.PChannelMeta{{ + Channel: &streamingpb.PChannelInfo{Name: "test", Term: 1}, + Node: &streamingpb.StreamingNodeInfo{ServerId: 1}, + }}) + assert.Error(t, err) + err = catalog.SaveBroadcastTask(context.Background(), &streamingpb.BroadcastTask{}) assert.Error(t, err) } diff --git a/internal/mocks/distributed/mock_streaming/mock_WALAccesser.go b/internal/mocks/distributed/mock_streaming/mock_WALAccesser.go index e077e04030f15..eb9f7ce2d8b0a 100644 --- a/internal/mocks/distributed/mock_streaming/mock_WALAccesser.go +++ b/internal/mocks/distributed/mock_streaming/mock_WALAccesser.go @@ -149,6 +149,65 @@ func (_c *MockWALAccesser_AppendMessagesWithOption_Call) RunAndReturn(run func(c return _c } +// BroadcastAppend provides a mock function with given fields: ctx, msg +func (_m *MockWALAccesser) BroadcastAppend(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) { + ret := _m.Called(ctx, msg) + + if len(ret) == 0 { + panic("no return value specified for BroadcastAppend") + } + + var r0 *types.BroadcastAppendResult + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error)); ok { + return rf(ctx, msg) + } + if rf, ok := ret.Get(0).(func(context.Context, message.BroadcastMutableMessage) *types.BroadcastAppendResult); ok { + r0 = rf(ctx, msg) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*types.BroadcastAppendResult) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, message.BroadcastMutableMessage) error); ok { + r1 = rf(ctx, msg) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockWALAccesser_BroadcastAppend_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'BroadcastAppend' +type MockWALAccesser_BroadcastAppend_Call struct { + *mock.Call +} + +// BroadcastAppend is a helper method to define mock.On call +// - ctx context.Context +// - msg message.BroadcastMutableMessage +func (_e *MockWALAccesser_Expecter) BroadcastAppend(ctx interface{}, msg interface{}) *MockWALAccesser_BroadcastAppend_Call { + return &MockWALAccesser_BroadcastAppend_Call{Call: _e.mock.On("BroadcastAppend", ctx, msg)} +} + +func (_c *MockWALAccesser_BroadcastAppend_Call) Run(run func(ctx context.Context, msg message.BroadcastMutableMessage)) *MockWALAccesser_BroadcastAppend_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(message.BroadcastMutableMessage)) + }) + return _c +} + +func (_c *MockWALAccesser_BroadcastAppend_Call) Return(_a0 *types.BroadcastAppendResult, _a1 error) *MockWALAccesser_BroadcastAppend_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockWALAccesser_BroadcastAppend_Call) RunAndReturn(run func(context.Context, message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error)) *MockWALAccesser_BroadcastAppend_Call { + _c.Call.Return(run) + return _c +} + // RawAppend provides a mock function with given fields: ctx, msgs, opts func (_m *MockWALAccesser) RawAppend(ctx context.Context, msgs message.MutableMessage, opts ...streaming.AppendOption) (*types.AppendResult, error) { _va := make([]interface{}, len(opts)) diff --git a/internal/mocks/mock_metastore/mock_StreamingCoordCataLog.go b/internal/mocks/mock_metastore/mock_StreamingCoordCataLog.go index b0bc3b77756d0..651554d48b3f3 100644 --- a/internal/mocks/mock_metastore/mock_StreamingCoordCataLog.go +++ b/internal/mocks/mock_metastore/mock_StreamingCoordCataLog.go @@ -23,6 +23,64 @@ func (_m *MockStreamingCoordCataLog) EXPECT() *MockStreamingCoordCataLog_Expecte return &MockStreamingCoordCataLog_Expecter{mock: &_m.Mock} } +// ListBroadcastTask provides a mock function with given fields: ctx +func (_m *MockStreamingCoordCataLog) ListBroadcastTask(ctx context.Context) ([]*streamingpb.BroadcastTask, error) { + ret := _m.Called(ctx) + + if len(ret) == 0 { + panic("no return value specified for ListBroadcastTask") + } + + var r0 []*streamingpb.BroadcastTask + var r1 error + if rf, ok := ret.Get(0).(func(context.Context) ([]*streamingpb.BroadcastTask, error)); ok { + return rf(ctx) + } + if rf, ok := ret.Get(0).(func(context.Context) []*streamingpb.BroadcastTask); ok { + r0 = rf(ctx) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]*streamingpb.BroadcastTask) + } + } + + if rf, ok := ret.Get(1).(func(context.Context) error); ok { + r1 = rf(ctx) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockStreamingCoordCataLog_ListBroadcastTask_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ListBroadcastTask' +type MockStreamingCoordCataLog_ListBroadcastTask_Call struct { + *mock.Call +} + +// ListBroadcastTask is a helper method to define mock.On call +// - ctx context.Context +func (_e *MockStreamingCoordCataLog_Expecter) ListBroadcastTask(ctx interface{}) *MockStreamingCoordCataLog_ListBroadcastTask_Call { + return &MockStreamingCoordCataLog_ListBroadcastTask_Call{Call: _e.mock.On("ListBroadcastTask", ctx)} +} + +func (_c *MockStreamingCoordCataLog_ListBroadcastTask_Call) Run(run func(ctx context.Context)) *MockStreamingCoordCataLog_ListBroadcastTask_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context)) + }) + return _c +} + +func (_c *MockStreamingCoordCataLog_ListBroadcastTask_Call) Return(_a0 []*streamingpb.BroadcastTask, _a1 error) *MockStreamingCoordCataLog_ListBroadcastTask_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockStreamingCoordCataLog_ListBroadcastTask_Call) RunAndReturn(run func(context.Context) ([]*streamingpb.BroadcastTask, error)) *MockStreamingCoordCataLog_ListBroadcastTask_Call { + _c.Call.Return(run) + return _c +} + // ListPChannel provides a mock function with given fields: ctx func (_m *MockStreamingCoordCataLog) ListPChannel(ctx context.Context) ([]*streamingpb.PChannelMeta, error) { ret := _m.Called(ctx) @@ -81,6 +139,53 @@ func (_c *MockStreamingCoordCataLog_ListPChannel_Call) RunAndReturn(run func(con return _c } +// SaveBroadcastTask provides a mock function with given fields: ctx, task +func (_m *MockStreamingCoordCataLog) SaveBroadcastTask(ctx context.Context, task *streamingpb.BroadcastTask) error { + ret := _m.Called(ctx, task) + + if len(ret) == 0 { + panic("no return value specified for SaveBroadcastTask") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, *streamingpb.BroadcastTask) error); ok { + r0 = rf(ctx, task) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// MockStreamingCoordCataLog_SaveBroadcastTask_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SaveBroadcastTask' +type MockStreamingCoordCataLog_SaveBroadcastTask_Call struct { + *mock.Call +} + +// SaveBroadcastTask is a helper method to define mock.On call +// - ctx context.Context +// - task *streamingpb.BroadcastTask +func (_e *MockStreamingCoordCataLog_Expecter) SaveBroadcastTask(ctx interface{}, task interface{}) *MockStreamingCoordCataLog_SaveBroadcastTask_Call { + return &MockStreamingCoordCataLog_SaveBroadcastTask_Call{Call: _e.mock.On("SaveBroadcastTask", ctx, task)} +} + +func (_c *MockStreamingCoordCataLog_SaveBroadcastTask_Call) Run(run func(ctx context.Context, task *streamingpb.BroadcastTask)) *MockStreamingCoordCataLog_SaveBroadcastTask_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(*streamingpb.BroadcastTask)) + }) + return _c +} + +func (_c *MockStreamingCoordCataLog_SaveBroadcastTask_Call) Return(_a0 error) *MockStreamingCoordCataLog_SaveBroadcastTask_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockStreamingCoordCataLog_SaveBroadcastTask_Call) RunAndReturn(run func(context.Context, *streamingpb.BroadcastTask) error) *MockStreamingCoordCataLog_SaveBroadcastTask_Call { + _c.Call.Return(run) + return _c +} + // SavePChannels provides a mock function with given fields: ctx, info func (_m *MockStreamingCoordCataLog) SavePChannels(ctx context.Context, info []*streamingpb.PChannelMeta) error { ret := _m.Called(ctx, info) diff --git a/internal/mocks/streamingcoord/mock_client/mock_BroadcastService.go b/internal/mocks/streamingcoord/mock_client/mock_BroadcastService.go new file mode 100644 index 0000000000000..3c84e0cce1f5d --- /dev/null +++ b/internal/mocks/streamingcoord/mock_client/mock_BroadcastService.go @@ -0,0 +1,98 @@ +// Code generated by mockery v2.46.0. DO NOT EDIT. + +package mock_client + +import ( + context "context" + + message "github.com/milvus-io/milvus/pkg/streaming/util/message" + mock "github.com/stretchr/testify/mock" + + types "github.com/milvus-io/milvus/pkg/streaming/util/types" +) + +// MockBroadcastService is an autogenerated mock type for the BroadcastService type +type MockBroadcastService struct { + mock.Mock +} + +type MockBroadcastService_Expecter struct { + mock *mock.Mock +} + +func (_m *MockBroadcastService) EXPECT() *MockBroadcastService_Expecter { + return &MockBroadcastService_Expecter{mock: &_m.Mock} +} + +// Broadcast provides a mock function with given fields: ctx, msg +func (_m *MockBroadcastService) Broadcast(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) { + ret := _m.Called(ctx, msg) + + if len(ret) == 0 { + panic("no return value specified for Broadcast") + } + + var r0 *types.BroadcastAppendResult + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error)); ok { + return rf(ctx, msg) + } + if rf, ok := ret.Get(0).(func(context.Context, message.BroadcastMutableMessage) *types.BroadcastAppendResult); ok { + r0 = rf(ctx, msg) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*types.BroadcastAppendResult) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, message.BroadcastMutableMessage) error); ok { + r1 = rf(ctx, msg) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockBroadcastService_Broadcast_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Broadcast' +type MockBroadcastService_Broadcast_Call struct { + *mock.Call +} + +// Broadcast is a helper method to define mock.On call +// - ctx context.Context +// - msg message.BroadcastMutableMessage +func (_e *MockBroadcastService_Expecter) Broadcast(ctx interface{}, msg interface{}) *MockBroadcastService_Broadcast_Call { + return &MockBroadcastService_Broadcast_Call{Call: _e.mock.On("Broadcast", ctx, msg)} +} + +func (_c *MockBroadcastService_Broadcast_Call) Run(run func(ctx context.Context, msg message.BroadcastMutableMessage)) *MockBroadcastService_Broadcast_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(message.BroadcastMutableMessage)) + }) + return _c +} + +func (_c *MockBroadcastService_Broadcast_Call) Return(_a0 *types.BroadcastAppendResult, _a1 error) *MockBroadcastService_Broadcast_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockBroadcastService_Broadcast_Call) RunAndReturn(run func(context.Context, message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error)) *MockBroadcastService_Broadcast_Call { + _c.Call.Return(run) + return _c +} + +// NewMockBroadcastService creates a new instance of MockBroadcastService. 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 NewMockBroadcastService(t interface { + mock.TestingT + Cleanup(func()) +}) *MockBroadcastService { + mock := &MockBroadcastService{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/internal/mocks/streamingcoord/mock_client/mock_Client.go b/internal/mocks/streamingcoord/mock_client/mock_Client.go index 02923644d235c..574e08d01533e 100644 --- a/internal/mocks/streamingcoord/mock_client/mock_Client.go +++ b/internal/mocks/streamingcoord/mock_client/mock_Client.go @@ -67,6 +67,53 @@ func (_c *MockClient_Assignment_Call) RunAndReturn(run func() client.AssignmentS return _c } +// Broadcast provides a mock function with given fields: +func (_m *MockClient) Broadcast() client.BroadcastService { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for Broadcast") + } + + var r0 client.BroadcastService + if rf, ok := ret.Get(0).(func() client.BroadcastService); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(client.BroadcastService) + } + } + + return r0 +} + +// MockClient_Broadcast_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Broadcast' +type MockClient_Broadcast_Call struct { + *mock.Call +} + +// Broadcast is a helper method to define mock.On call +func (_e *MockClient_Expecter) Broadcast() *MockClient_Broadcast_Call { + return &MockClient_Broadcast_Call{Call: _e.mock.On("Broadcast")} +} + +func (_c *MockClient_Broadcast_Call) Run(run func()) *MockClient_Broadcast_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockClient_Broadcast_Call) Return(_a0 client.BroadcastService) *MockClient_Broadcast_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockClient_Broadcast_Call) RunAndReturn(run func() client.BroadcastService) *MockClient_Broadcast_Call { + _c.Call.Return(run) + return _c +} + // Close provides a mock function with given fields: func (_m *MockClient) Close() { _m.Called() diff --git a/internal/mocks/streamingcoord/server/mock_broadcaster/mock_AppendOperator.go b/internal/mocks/streamingcoord/server/mock_broadcaster/mock_AppendOperator.go new file mode 100644 index 0000000000000..8f049c5616cf6 --- /dev/null +++ b/internal/mocks/streamingcoord/server/mock_broadcaster/mock_AppendOperator.go @@ -0,0 +1,100 @@ +// Code generated by mockery v2.46.0. DO NOT EDIT. + +package mock_broadcaster + +import ( + context "context" + + message "github.com/milvus-io/milvus/pkg/streaming/util/message" + mock "github.com/stretchr/testify/mock" + + streaming "github.com/milvus-io/milvus/internal/distributed/streaming" +) + +// MockAppendOperator is an autogenerated mock type for the AppendOperator type +type MockAppendOperator struct { + mock.Mock +} + +type MockAppendOperator_Expecter struct { + mock *mock.Mock +} + +func (_m *MockAppendOperator) EXPECT() *MockAppendOperator_Expecter { + return &MockAppendOperator_Expecter{mock: &_m.Mock} +} + +// AppendMessages provides a mock function with given fields: ctx, msgs +func (_m *MockAppendOperator) AppendMessages(ctx context.Context, msgs ...message.MutableMessage) streaming.AppendResponses { + _va := make([]interface{}, len(msgs)) + for _i := range msgs { + _va[_i] = msgs[_i] + } + var _ca []interface{} + _ca = append(_ca, ctx) + _ca = append(_ca, _va...) + ret := _m.Called(_ca...) + + if len(ret) == 0 { + panic("no return value specified for AppendMessages") + } + + var r0 streaming.AppendResponses + if rf, ok := ret.Get(0).(func(context.Context, ...message.MutableMessage) streaming.AppendResponses); ok { + r0 = rf(ctx, msgs...) + } else { + r0 = ret.Get(0).(streaming.AppendResponses) + } + + return r0 +} + +// MockAppendOperator_AppendMessages_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'AppendMessages' +type MockAppendOperator_AppendMessages_Call struct { + *mock.Call +} + +// AppendMessages is a helper method to define mock.On call +// - ctx context.Context +// - msgs ...message.MutableMessage +func (_e *MockAppendOperator_Expecter) AppendMessages(ctx interface{}, msgs ...interface{}) *MockAppendOperator_AppendMessages_Call { + return &MockAppendOperator_AppendMessages_Call{Call: _e.mock.On("AppendMessages", + append([]interface{}{ctx}, msgs...)...)} +} + +func (_c *MockAppendOperator_AppendMessages_Call) Run(run func(ctx context.Context, msgs ...message.MutableMessage)) *MockAppendOperator_AppendMessages_Call { + _c.Call.Run(func(args mock.Arguments) { + variadicArgs := make([]message.MutableMessage, len(args)-1) + for i, a := range args[1:] { + if a != nil { + variadicArgs[i] = a.(message.MutableMessage) + } + } + run(args[0].(context.Context), variadicArgs...) + }) + return _c +} + +func (_c *MockAppendOperator_AppendMessages_Call) Return(_a0 streaming.AppendResponses) *MockAppendOperator_AppendMessages_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockAppendOperator_AppendMessages_Call) RunAndReturn(run func(context.Context, ...message.MutableMessage) streaming.AppendResponses) *MockAppendOperator_AppendMessages_Call { + _c.Call.Return(run) + return _c +} + +// NewMockAppendOperator creates a new instance of MockAppendOperator. 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 NewMockAppendOperator(t interface { + mock.TestingT + Cleanup(func()) +}) *MockAppendOperator { + mock := &MockAppendOperator{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/internal/streamingcoord/client/broadcast/broadcast_impl.go b/internal/streamingcoord/client/broadcast/broadcast_impl.go new file mode 100644 index 0000000000000..b6296748d1eba --- /dev/null +++ b/internal/streamingcoord/client/broadcast/broadcast_impl.go @@ -0,0 +1,56 @@ +package broadcast + +import ( + "context" + + "github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc" + "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb" + "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/streaming/util/types" +) + +// NewBroadcastService creates a new broadcast service. +func NewBroadcastService(walName string, service lazygrpc.Service[streamingpb.StreamingCoordBroadcastServiceClient]) *BroadcastServiceImpl { + return &BroadcastServiceImpl{ + walName: walName, + service: service, + } +} + +// BroadcastServiceImpl is the implementation of BroadcastService. +type BroadcastServiceImpl struct { + walName string + service lazygrpc.Service[streamingpb.StreamingCoordBroadcastServiceClient] +} + +// Broadcast sends a broadcast message to the streaming coord to perform a broadcast. +func (c *BroadcastServiceImpl) Broadcast(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) { + client, err := c.service.GetService(ctx) + if err != nil { + return nil, err + } + resp, err := client.Broadcast(ctx, &streamingpb.BroadcastRequest{ + Message: &messagespb.Message{ + Payload: msg.Payload(), + Properties: msg.Properties().ToRawMap(), + }, + }) + if err != nil { + return nil, err + } + results := make(map[string]*types.AppendResult, len(resp.Results)) + for channel, result := range resp.Results { + msgID, err := message.UnmarshalMessageID(c.walName, result.Id.Id) + if err != nil { + return nil, err + } + results[channel] = &types.AppendResult{ + MessageID: msgID, + TimeTick: result.GetTimetick(), + TxnCtx: message.NewTxnContextFromProto(result.GetTxnContext()), + Extra: result.GetExtra(), + } + } + return &types.BroadcastAppendResult{AppendResults: results}, nil +} diff --git a/internal/streamingcoord/client/client.go b/internal/streamingcoord/client/client.go index 83a55fd107159..07f0937360bfd 100644 --- a/internal/streamingcoord/client/client.go +++ b/internal/streamingcoord/client/client.go @@ -11,12 +11,15 @@ import ( "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/streamingcoord/client/assignment" + "github.com/milvus-io/milvus/internal/streamingcoord/client/broadcast" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/streamingutil/service/balancer/picker" streamingserviceinterceptor "github.com/milvus-io/milvus/internal/util/streamingutil/service/interceptor" "github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc" "github.com/milvus-io/milvus/internal/util/streamingutil/service/resolver" + "github.com/milvus-io/milvus/internal/util/streamingutil/util" "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/tracer" "github.com/milvus-io/milvus/pkg/util/interceptor" @@ -32,8 +35,16 @@ type AssignmentService interface { types.AssignmentDiscoverWatcher } +// BroadcastService is the interface of broadcast service. +type BroadcastService interface { + // Broadcast sends a broadcast message to the streaming service. + Broadcast(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) +} + // Client is the interface of log service client. type Client interface { + Broadcast() BroadcastService + // Assignment access assignment service. Assignment() AssignmentService @@ -58,10 +69,12 @@ func NewClient(etcdCli *clientv3.Client) Client { ) }) assignmentService := lazygrpc.WithServiceCreator(conn, streamingpb.NewStreamingCoordAssignmentServiceClient) + broadcastService := lazygrpc.WithServiceCreator(conn, streamingpb.NewStreamingCoordBroadcastServiceClient) return &clientImpl{ conn: conn, rb: rb, assignmentService: assignment.NewAssignmentService(assignmentService), + broadcastService: broadcast.NewBroadcastService(util.MustSelectWALName(), broadcastService), } } diff --git a/internal/streamingcoord/client/client_impl.go b/internal/streamingcoord/client/client_impl.go index ffb0b0355a3a5..88c94794e1c4d 100644 --- a/internal/streamingcoord/client/client_impl.go +++ b/internal/streamingcoord/client/client_impl.go @@ -2,6 +2,7 @@ package client import ( "github.com/milvus-io/milvus/internal/streamingcoord/client/assignment" + "github.com/milvus-io/milvus/internal/streamingcoord/client/broadcast" "github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc" "github.com/milvus-io/milvus/internal/util/streamingutil/service/resolver" ) @@ -11,6 +12,11 @@ type clientImpl struct { conn lazygrpc.Conn rb resolver.Builder assignmentService *assignment.AssignmentServiceImpl + broadcastService *broadcast.BroadcastServiceImpl +} + +func (c *clientImpl) Broadcast() BroadcastService { + return c.broadcastService } // Assignment access assignment service. diff --git a/internal/streamingcoord/server/broadcaster/append_operator.go b/internal/streamingcoord/server/broadcaster/append_operator.go new file mode 100644 index 0000000000000..ec849ea2be917 --- /dev/null +++ b/internal/streamingcoord/server/broadcaster/append_operator.go @@ -0,0 +1,14 @@ +package broadcaster + +import ( + "github.com/milvus-io/milvus/internal/distributed/streaming" + "github.com/milvus-io/milvus/internal/util/streamingutil" +) + +// NewAppendOperator creates an append operator to handle the incoming messages for broadcaster. +func NewAppendOperator() AppendOperator { + if streamingutil.IsStreamingServiceEnabled() { + return streaming.WAL() + } + return nil +} diff --git a/internal/streamingcoord/server/broadcaster/broadcaster.go b/internal/streamingcoord/server/broadcaster/broadcaster.go new file mode 100644 index 0000000000000..79e77bb8829cf --- /dev/null +++ b/internal/streamingcoord/server/broadcaster/broadcaster.go @@ -0,0 +1,24 @@ +package broadcaster + +import ( + "context" + + "github.com/milvus-io/milvus/internal/distributed/streaming" + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/streaming/util/types" +) + +type Broadcaster interface { + // Broadcast broadcasts the message to all channels. + Broadcast(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) + + // Close closes the broadcaster. + Close() +} + +// AppendOperator is used to append messages, there's only two implement of this interface: +// 1. streaming.WAL() +// 2. old msgstream interface +type AppendOperator interface { + AppendMessages(ctx context.Context, msgs ...message.MutableMessage) streaming.AppendResponses +} diff --git a/internal/streamingcoord/server/broadcaster/broadcaster_impl.go b/internal/streamingcoord/server/broadcaster/broadcaster_impl.go new file mode 100644 index 0000000000000..2da0e0679f907 --- /dev/null +++ b/internal/streamingcoord/server/broadcaster/broadcaster_impl.go @@ -0,0 +1,207 @@ +package broadcaster + +import ( + "context" + "sync" + "time" + + "go.uber.org/zap" + + "github.com/milvus-io/milvus/internal/streamingcoord/server/resource" + "github.com/milvus-io/milvus/internal/util/streamingutil/status" + "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb" + "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/streaming/util/types" + "github.com/milvus-io/milvus/pkg/util/contextutil" + "github.com/milvus-io/milvus/pkg/util/syncutil" + "github.com/milvus-io/milvus/pkg/util/typeutil" +) + +func RecoverBroadcaster( + ctx context.Context, + appendOperator AppendOperator, +) (Broadcaster, error) { + logger := resource.Resource().Logger().With(log.FieldComponent("broadcaster")) + tasks, err := resource.Resource().StreamingCatalog().ListBroadcastTask(ctx) + if err != nil { + return nil, err + } + pendings := make([]*broadcastTask, 0, len(tasks)) + for _, task := range tasks { + if task.State == streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING { + // recover pending task + t := newTask(task, logger) + pendings = append(pendings, t) + } + } + b := &broadcasterImpl{ + logger: logger, + lifetime: typeutil.NewLifetime(), + backgroundTaskNotifier: syncutil.NewAsyncTaskNotifier[struct{}](), + pendings: pendings, + backoffs: typeutil.NewHeap[*broadcastTask](&broadcastTaskArray{}), + backoffChan: make(chan *broadcastTask), + pendingChan: make(chan *broadcastTask), + workerChan: make(chan *broadcastTask), + appendOperator: appendOperator, + } + go b.execute() + return b, nil +} + +// broadcasterImpl is the implementation of Broadcaster +type broadcasterImpl struct { + logger *log.MLogger + lifetime *typeutil.Lifetime + backgroundTaskNotifier *syncutil.AsyncTaskNotifier[struct{}] + pendings []*broadcastTask + backoffs typeutil.Heap[*broadcastTask] + pendingChan chan *broadcastTask + backoffChan chan *broadcastTask + workerChan chan *broadcastTask + appendOperator AppendOperator +} + +// Broadcast broadcasts the message to all channels. +func (b *broadcasterImpl) Broadcast(ctx context.Context, msg message.BroadcastMutableMessage) (result *types.BroadcastAppendResult, err error) { + if !b.lifetime.Add(typeutil.LifetimeStateWorking) { + return nil, status.NewOnShutdownError("broadcaster is closing") + } + defer func() { + if err != nil { + b.logger.Warn("broadcast message failed", zap.Error(err)) + return + } + }() + + // Once the task is persisted, it must be successful. + task, err := b.persistBroadcastTask(ctx, msg) + if err != nil { + return nil, err + } + t := newTask(task, b.logger) + select { + case <-b.backgroundTaskNotifier.Context().Done(): + // We can only check the background context but not the request context here. + // Because we want the new incoming task must be delivered to the background task queue + // otherwise the broadcaster is closing + return nil, status.NewOnShutdownError("broadcaster is closing") + case b.pendingChan <- t: + } + + // Wait both request context and the background task context. + ctx, _ = contextutil.MergeContext(ctx, b.backgroundTaskNotifier.Context()) + return t.BlockUntilTaskDone(ctx) +} + +// persistBroadcastTask persists the broadcast task into catalog. +func (b *broadcasterImpl) persistBroadcastTask(ctx context.Context, msg message.BroadcastMutableMessage) (*streamingpb.BroadcastTask, error) { + defer b.lifetime.Done() + + id, err := resource.Resource().IDAllocator().Allocate(ctx) + if err != nil { + return nil, status.NewInner("allocate new id failed, %s", err.Error()) + } + task := &streamingpb.BroadcastTask{ + TaskId: int64(id), + Message: &messagespb.Message{Payload: msg.Payload(), Properties: msg.Properties().ToRawMap()}, + State: streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING, + } + // Save the task into catalog to help recovery. + if err := resource.Resource().StreamingCatalog().SaveBroadcastTask(ctx, task); err != nil { + return nil, status.NewInner("save broadcast task failed, %s", err.Error()) + } + return task, nil +} + +func (b *broadcasterImpl) Close() { + b.lifetime.SetState(typeutil.LifetimeStateStopped) + b.lifetime.Wait() + + b.backgroundTaskNotifier.Cancel() + b.backgroundTaskNotifier.BlockUntilFinish() +} + +// execute the broadcaster +func (b *broadcasterImpl) execute() { + b.logger.Info("broadcaster start to execute") + defer func() { + b.backgroundTaskNotifier.Finish(struct{}{}) + b.logger.Info("broadcaster execute exit") + }() + + // Start n workers to handle the broadcast task. + wg := sync.WaitGroup{} + for i := 0; i < 4; i++ { + i := i + // Start n workers to handle the broadcast task. + wg.Add(1) + go func() { + defer wg.Done() + b.worker(i) + }() + } + defer wg.Wait() + + b.dispatch() +} + +func (b *broadcasterImpl) dispatch() { + for { + var workerChan chan *broadcastTask + var nextTask *broadcastTask + var nextBackOff <-chan time.Time + // Wait for new task. + if len(b.pendings) > 0 { + workerChan = b.workerChan + nextTask = b.pendings[0] + } + if b.backoffs.Len() > 0 { + var nextInterval time.Duration + nextBackOff, nextInterval = b.backoffs.Peek().NextTimer() + b.logger.Info("backoff task", zap.Duration("nextInterval", nextInterval)) + } + + select { + case <-b.backgroundTaskNotifier.Context().Done(): + return + case task := <-b.pendingChan: + b.pendings = append(b.pendings, task) + case task := <-b.backoffChan: + // task is backoff, push it into backoff queue to make a delay retry. + b.backoffs.Push(task) + case <-nextBackOff: + // backoff is done, move all the backoff done task into pending to retry. + for b.backoffs.Len() > 0 && b.backoffs.Peek().NextInterval() < time.Millisecond { + b.pendings = append(b.pendings, b.backoffs.Pop()) + } + case workerChan <- nextTask: + // The task is sent to worker, remove it from pending list. + b.pendings = b.pendings[1:] + } + } +} + +func (b *broadcasterImpl) worker(no int) { + defer func() { + b.logger.Info("broadcaster worker exit", zap.Int("no", no)) + }() + + for { + select { + case <-b.backgroundTaskNotifier.Context().Done(): + return + case task := <-b.workerChan: + if err := task.Poll(b.backgroundTaskNotifier.Context(), b.appendOperator); err != nil { + // If the task is not done, repush it into pendings and retry infinitely. + select { + case <-b.backgroundTaskNotifier.Context().Done(): + return + case b.backoffChan <- task: + } + } + } + } +} diff --git a/internal/streamingcoord/server/broadcaster/broadcaster_test.go b/internal/streamingcoord/server/broadcaster/broadcaster_test.go new file mode 100644 index 0000000000000..624535f1c8755 --- /dev/null +++ b/internal/streamingcoord/server/broadcaster/broadcaster_test.go @@ -0,0 +1,142 @@ +package broadcaster + +import ( + "context" + "math/rand" + "testing" + "time" + + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" + "go.uber.org/atomic" + + "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" + "github.com/milvus-io/milvus/internal/distributed/streaming" + "github.com/milvus-io/milvus/internal/mocks/mock_metastore" + "github.com/milvus-io/milvus/internal/mocks/streamingcoord/server/mock_broadcaster" + "github.com/milvus-io/milvus/internal/streamingcoord/server/resource" + internaltypes "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/internal/util/idalloc" + "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb" + "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/streaming/util/types" + "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/walimplstest" + "github.com/milvus-io/milvus/pkg/util/syncutil" +) + +func TestBroadcaster(t *testing.T) { + meta := mock_metastore.NewMockStreamingCoordCataLog(t) + meta.EXPECT().ListBroadcastTask(mock.Anything). + RunAndReturn(func(ctx context.Context) ([]*streamingpb.BroadcastTask, error) { + return []*streamingpb.BroadcastTask{ + createNewBroadcastTask(1, []string{"v1"}), + createNewBroadcastTask(2, []string{"v1", "v2"}), + createNewBroadcastTask(3, []string{"v1", "v2", "v3"}), + }, nil + }).Times(1) + done := atomic.NewInt64(0) + meta.EXPECT().SaveBroadcastTask(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, bt *streamingpb.BroadcastTask) error { + // may failure + if rand.Int31n(10) < 5 { + return errors.New("save task failed") + } + if bt.State == streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_DONE { + done.Inc() + } + return nil + }) + rc := idalloc.NewMockRootCoordClient(t) + f := syncutil.NewFuture[internaltypes.RootCoordClient]() + f.Set(rc) + resource.InitForTest(resource.OptStreamingCatalog(meta), resource.OptRootCoordClient(f)) + + operator, appended := createOpeartor(t) + bc, err := RecoverBroadcaster(context.Background(), operator) + assert.NoError(t, err) + assert.NotNil(t, bc) + assert.Eventually(t, func() bool { + return appended.Load() == 6 && done.Load() == 3 + }, 10*time.Second, 10*time.Millisecond) + + var result *types.BroadcastAppendResult + for { + var err error + result, err = bc.Broadcast(context.Background(), createNewBroadcastMsg([]string{"v1", "v2", "v3"})) + if err == nil { + break + } + } + assert.Equal(t, int(appended.Load()), 9) + assert.Equal(t, len(result.AppendResults), 3) + + assert.Eventually(t, func() bool { + return done.Load() == 4 + }, 10*time.Second, 10*time.Millisecond) + + // TODO: error path. + bc.Close() + + result, err = bc.Broadcast(context.Background(), createNewBroadcastMsg([]string{"v1", "v2", "v3"})) + assert.Error(t, err) + assert.Nil(t, result) +} + +func createOpeartor(t *testing.T) (AppendOperator, *atomic.Int64) { + id := atomic.NewInt64(1) + appended := atomic.NewInt64(0) + operator := mock_broadcaster.NewMockAppendOperator(t) + f := func(ctx context.Context, msgs ...message.MutableMessage) streaming.AppendResponses { + resps := streaming.AppendResponses{ + Responses: make([]streaming.AppendResponse, len(msgs)), + } + for idx := range msgs { + newID := walimplstest.NewTestMessageID(id.Inc()) + if rand.Int31n(10) < 5 { + resps.Responses[idx] = streaming.AppendResponse{ + Error: errors.New("append failed"), + } + continue + } + resps.Responses[idx] = streaming.AppendResponse{ + AppendResult: &types.AppendResult{ + MessageID: newID, + TimeTick: uint64(time.Now().UnixMilli()), + }, + Error: nil, + } + appended.Inc() + } + return resps + } + operator.EXPECT().AppendMessages(mock.Anything, mock.Anything).RunAndReturn(f) + operator.EXPECT().AppendMessages(mock.Anything, mock.Anything, mock.Anything).RunAndReturn(f) + operator.EXPECT().AppendMessages(mock.Anything, mock.Anything, mock.Anything, mock.Anything).RunAndReturn(f) + operator.EXPECT().AppendMessages(mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).RunAndReturn(f) + return operator, appended +} + +func createNewBroadcastMsg(vchannels []string) message.BroadcastMutableMessage { + msg, err := message.NewDropCollectionMessageBuilderV1(). + WithHeader(&messagespb.DropCollectionMessageHeader{}). + WithBody(&msgpb.DropCollectionRequest{}). + WithBroadcast(vchannels). + BuildBroadcast() + if err != nil { + panic(err) + } + return msg +} + +func createNewBroadcastTask(taskID int64, vchannels []string) *streamingpb.BroadcastTask { + msg := createNewBroadcastMsg(vchannels) + return &streamingpb.BroadcastTask{ + TaskId: taskID, + Message: &messagespb.Message{ + Payload: msg.Payload(), + Properties: msg.Properties().ToRawMap(), + }, + State: streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING, + } +} diff --git a/internal/streamingcoord/server/broadcaster/task.go b/internal/streamingcoord/server/broadcaster/task.go new file mode 100644 index 0000000000000..52a2b0e77d0c6 --- /dev/null +++ b/internal/streamingcoord/server/broadcaster/task.go @@ -0,0 +1,126 @@ +package broadcaster + +import ( + "context" + "time" + + "github.com/cockroachdb/errors" + "go.uber.org/zap" + + "github.com/milvus-io/milvus/internal/streamingcoord/server/resource" + "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/streaming/util/types" + "github.com/milvus-io/milvus/pkg/util/syncutil" + "github.com/milvus-io/milvus/pkg/util/typeutil" +) + +var errBroadcastTaskIsNotDone = errors.New("broadcast task is not done") + +// newTask creates a new task +func newTask(task *streamingpb.BroadcastTask, logger *log.MLogger) *broadcastTask { + bt := message.NewBroadcastMutableMessage(task.Message.Payload, task.Message.Properties) + msgs := bt.SplitIntoMutableMessage() + return &broadcastTask{ + logger: logger.With(zap.Int64("taskID", task.TaskId), zap.Int("broadcastTotal", len(msgs))), + task: task, + pendingMessages: msgs, + appendResult: make(map[string]*types.AppendResult, len(msgs)), + future: syncutil.NewFuture[*types.BroadcastAppendResult](), + BackoffWithInstant: typeutil.NewBackoffWithInstant(typeutil.BackoffTimerConfig{ + Default: 10 * time.Second, + Backoff: typeutil.BackoffConfig{ + InitialInterval: 10 * time.Millisecond, + Multiplier: 2.0, + MaxInterval: 10 * time.Second, + }, + }), + } +} + +// broadcastTask is the task for broadcasting messages. +type broadcastTask struct { + logger *log.MLogger + task *streamingpb.BroadcastTask + pendingMessages []message.MutableMessage + appendResult map[string]*types.AppendResult + future *syncutil.Future[*types.BroadcastAppendResult] + *typeutil.BackoffWithInstant +} + +// Poll polls the task, return nil if the task is done, otherwise not done. +// Poll can be repeated called until the task is done. +func (b *broadcastTask) Poll(ctx context.Context, operator AppendOperator) error { + if len(b.pendingMessages) > 0 { + b.logger.Debug("broadcast task is polling to make sent...", zap.Int("pendingMessages", len(b.pendingMessages))) + resps := operator.AppendMessages(ctx, b.pendingMessages...) + newPendings := make([]message.MutableMessage, 0) + for idx, resp := range resps.Responses { + if resp.Error != nil { + newPendings = append(newPendings, b.pendingMessages[idx]) + continue + } + b.appendResult[b.pendingMessages[idx].VChannel()] = resp.AppendResult + } + b.pendingMessages = newPendings + if len(newPendings) == 0 { + b.future.Set(&types.BroadcastAppendResult{AppendResults: b.appendResult}) + } + b.logger.Info("broadcast task make a new broadcast done", zap.Int("pendingMessages", len(b.pendingMessages))) + } + if len(b.pendingMessages) == 0 { + // There's no more pending message, mark the task as done. + b.task.State = streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_DONE + if err := resource.Resource().StreamingCatalog().SaveBroadcastTask(ctx, b.task); err != nil { + b.logger.Warn("save broadcast task failed", zap.Error(err)) + b.UpdateInstantWithNextBackOff() + return err + } + return nil + } + b.UpdateInstantWithNextBackOff() + return errBroadcastTaskIsNotDone +} + +// BlockUntilTaskDone blocks until the task is done. +func (b *broadcastTask) BlockUntilTaskDone(ctx context.Context) (*types.BroadcastAppendResult, error) { + return b.future.GetWithContext(ctx) +} + +type broadcastTaskArray []*broadcastTask + +// Len returns the length of the heap. +func (h broadcastTaskArray) Len() int { + return len(h) +} + +// Less returns true if the element at index i is less than the element at index j. +func (h broadcastTaskArray) Less(i, j int) bool { + return h[i].NextInstant().Before(h[j].NextInstant()) +} + +// Swap swaps the elements at indexes i and j. +func (h broadcastTaskArray) Swap(i, j int) { h[i], h[j] = h[j], h[i] } + +// Push pushes the last one at len. +func (h *broadcastTaskArray) Push(x interface{}) { + // Push and Pop use pointer receivers because they modify the slice's length, + // not just its contents. + *h = append(*h, x.(*broadcastTask)) +} + +// Pop pop the last one at len. +func (h *broadcastTaskArray) Pop() interface{} { + old := *h + n := len(old) + x := old[n-1] + *h = old[0 : n-1] + return x +} + +// Peek returns the element at the top of the heap. +// Panics if the heap is empty. +func (h *broadcastTaskArray) Peek() interface{} { + return (*h)[0] +} diff --git a/internal/streamingcoord/server/builder.go b/internal/streamingcoord/server/builder.go index 4d2215b6df638..dcbb5eeb4c0c7 100644 --- a/internal/streamingcoord/server/builder.go +++ b/internal/streamingcoord/server/builder.go @@ -5,6 +5,7 @@ import ( "github.com/milvus-io/milvus/internal/metastore/kv/streamingcoord" "github.com/milvus-io/milvus/internal/streamingcoord/server/balancer" + "github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster" "github.com/milvus-io/milvus/internal/streamingcoord/server/resource" "github.com/milvus-io/milvus/internal/streamingcoord/server/service" "github.com/milvus-io/milvus/internal/types" @@ -52,10 +53,13 @@ func (s *ServerBuilder) Build() *Server { resource.OptRootCoordClient(s.rootCoordClient), ) balancer := syncutil.NewFuture[balancer.Balancer]() + broadcaster := syncutil.NewFuture[broadcaster.Broadcaster]() return &Server{ logger: resource.Resource().Logger().With(log.FieldComponent("server")), session: s.session, assignmentService: service.NewAssignmentService(balancer), + broadcastService: service.NewBroadcastService(broadcaster), balancer: balancer, + broadcaster: broadcaster, } } diff --git a/internal/streamingcoord/server/resource/resource.go b/internal/streamingcoord/server/resource/resource.go index 89b8dee5730c1..96a92e3727125 100644 --- a/internal/streamingcoord/server/resource/resource.go +++ b/internal/streamingcoord/server/resource/resource.go @@ -8,6 +8,7 @@ import ( "github.com/milvus-io/milvus/internal/metastore" "github.com/milvus-io/milvus/internal/streamingnode/client/manager" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/internal/util/idalloc" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/syncutil" "github.com/milvus-io/milvus/pkg/util/typeutil" @@ -29,6 +30,7 @@ func OptETCD(etcd *clientv3.Client) optResourceInit { func OptRootCoordClient(rootCoordClient *syncutil.Future[types.RootCoordClient]) optResourceInit { return func(r *resourceImpl) { r.rootCoordClient = rootCoordClient + r.idAllocator = idalloc.NewIDAllocator(r.rootCoordClient) } } @@ -48,6 +50,7 @@ func Init(opts ...optResourceInit) { for _, opt := range opts { opt(newR) } + assertNotNil(newR.IDAllocator()) assertNotNil(newR.RootCoordClient()) assertNotNil(newR.ETCD()) assertNotNil(newR.StreamingCatalog()) @@ -64,6 +67,7 @@ func Resource() *resourceImpl { // resourceImpl is a basic resource dependency for streamingnode server. // All utility on it is concurrent-safe and singleton. type resourceImpl struct { + idAllocator idalloc.Allocator rootCoordClient *syncutil.Future[types.RootCoordClient] etcdClient *clientv3.Client streamingCatalog metastore.StreamingCoordCataLog @@ -76,6 +80,11 @@ func (r *resourceImpl) RootCoordClient() *syncutil.Future[types.RootCoordClient] return r.rootCoordClient } +// IDAllocator returns the IDAllocator client. +func (r *resourceImpl) IDAllocator() idalloc.Allocator { + return r.idAllocator +} + // StreamingCatalog returns the StreamingCatalog client. func (r *resourceImpl) StreamingCatalog() metastore.StreamingCoordCataLog { return r.streamingCatalog diff --git a/internal/streamingcoord/server/server.go b/internal/streamingcoord/server/server.go index 2b9e50f3c2be4..f465d1b4b6cfc 100644 --- a/internal/streamingcoord/server/server.go +++ b/internal/streamingcoord/server/server.go @@ -8,6 +8,7 @@ import ( "github.com/milvus-io/milvus/internal/streamingcoord/server/balancer" _ "github.com/milvus-io/milvus/internal/streamingcoord/server/balancer/policy" // register the balancer policy + "github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster" "github.com/milvus-io/milvus/internal/streamingcoord/server/service" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/streamingutil" @@ -27,9 +28,11 @@ type Server struct { // service level variables. assignmentService service.AssignmentService + broadcastService service.BroadcastService // basic component variables can be used at service level. - balancer *syncutil.Future[balancer.Balancer] + balancer *syncutil.Future[balancer.Balancer] + broadcaster *syncutil.Future[broadcaster.Broadcaster] } // Init initializes the streamingcoord server. @@ -46,8 +49,9 @@ func (s *Server) Start(ctx context.Context) (err error) { // initBasicComponent initialize all underlying dependency for streamingcoord. func (s *Server) initBasicComponent(ctx context.Context) (err error) { + futures := make([]*conc.Future[struct{}], 0) if streamingutil.IsStreamingServiceEnabled() { - fBalancer := conc.Go(func() (struct{}, error) { + futures = append(futures, conc.Go(func() (struct{}, error) { s.logger.Info("start recovery balancer...") // Read new incoming topics from configuration, and register it into balancer. newIncomingTopics := util.GetAllTopicsFromConfiguration() @@ -59,10 +63,22 @@ func (s *Server) initBasicComponent(ctx context.Context) (err error) { s.balancer.Set(balancer) s.logger.Info("recover balancer done") return struct{}{}, nil - }) - return conc.AwaitAll(fBalancer) + })) } - return nil + // The broadcaster of msgstream is implemented on current streamingcoord to reduce the development complexity. + // So we need to recover it. + futures = append(futures, conc.Go(func() (struct{}, error) { + s.logger.Info("start recovery broadcaster...") + broadcaster, err := broadcaster.RecoverBroadcaster(ctx, broadcaster.NewAppendOperator()) + if err != nil { + s.logger.Warn("recover broadcaster failed", zap.Error(err)) + return struct{}{}, err + } + s.broadcaster.Set(broadcaster) + s.logger.Info("recover broadcaster done") + return struct{}{}, nil + })) + return conc.AwaitAll(futures...) } // RegisterGRPCService register all grpc service to grpc server. @@ -70,6 +86,7 @@ func (s *Server) RegisterGRPCService(grpcServer *grpc.Server) { if streamingutil.IsStreamingServiceEnabled() { streamingpb.RegisterStreamingCoordAssignmentServiceServer(grpcServer, s.assignmentService) } + streamingpb.RegisterStreamingCoordBroadcastServiceServer(grpcServer, s.broadcastService) } // Close closes the streamingcoord server. @@ -80,5 +97,11 @@ func (s *Server) Stop() { } else { s.logger.Info("balancer not ready, skip close") } + if s.broadcaster.Ready() { + s.logger.Info("start close broadcaster...") + s.broadcaster.Get().Close() + } else { + s.logger.Info("broadcaster not ready, skip close") + } s.logger.Info("streamingcoord server stopped") } diff --git a/internal/streamingcoord/server/service/broadcast.go b/internal/streamingcoord/server/service/broadcast.go new file mode 100644 index 0000000000000..6d192615e32d4 --- /dev/null +++ b/internal/streamingcoord/server/service/broadcast.go @@ -0,0 +1,44 @@ +package service + +import ( + "context" + + "github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster" + "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/util/syncutil" +) + +// BroadcastService is the interface of the broadcast service. +type BroadcastService interface { + streamingpb.StreamingCoordBroadcastServiceServer +} + +// NewBroadcastService creates a new broadcast service. +func NewBroadcastService(bc *syncutil.Future[broadcaster.Broadcaster]) BroadcastService { + return &broadcastServceImpl{ + broadcaster: bc, + } +} + +// broadcastServiceeeeImpl is the implementation of the broadcast service. +type broadcastServceImpl struct { + broadcaster *syncutil.Future[broadcaster.Broadcaster] +} + +// Broadcast broadcasts the message to all channels. +func (s *broadcastServceImpl) Broadcast(ctx context.Context, req *streamingpb.BroadcastRequest) (*streamingpb.BroadcastResponse, error) { + broadcaster, err := s.broadcaster.GetWithContext(ctx) + if err != nil { + return nil, err + } + results, err := broadcaster.Broadcast(ctx, message.NewBroadcastMutableMessage(req.Message.Payload, req.Message.Properties)) + if err != nil { + return nil, err + } + protoResult := make(map[string]*streamingpb.ProduceMessageResponseResult, len(results.AppendResults)) + for vchannel, result := range results.AppendResults { + protoResult[vchannel] = result.IntoProto() + } + return &streamingpb.BroadcastResponse{Results: protoResult}, nil +} diff --git a/internal/streamingnode/server/flusher/flusherimpl/flusher_impl.go b/internal/streamingnode/server/flusher/flusherimpl/flusher_impl.go index c97c9b491bba4..a5c417b64b212 100644 --- a/internal/streamingnode/server/flusher/flusherimpl/flusher_impl.go +++ b/internal/streamingnode/server/flusher/flusherimpl/flusher_impl.go @@ -33,8 +33,8 @@ import ( "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/streamingnode/server/flusher" "github.com/milvus-io/milvus/internal/streamingnode/server/resource" - "github.com/milvus-io/milvus/internal/streamingnode/server/resource/idalloc" "github.com/milvus-io/milvus/internal/streamingnode/server/wal" + "github.com/milvus-io/milvus/internal/util/idalloc" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/funcutil" diff --git a/internal/streamingnode/server/resource/resource.go b/internal/streamingnode/server/resource/resource.go index 0626d9de28648..cb762dccdb8b3 100644 --- a/internal/streamingnode/server/resource/resource.go +++ b/internal/streamingnode/server/resource/resource.go @@ -8,10 +8,10 @@ import ( "github.com/milvus-io/milvus/internal/metastore" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/streamingnode/server/flusher" - "github.com/milvus-io/milvus/internal/streamingnode/server/resource/idalloc" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/segment/stats" tinspector "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/timetick/inspector" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/internal/util/idalloc" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/syncutil" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/streamingnode/server/resource/test_utility.go b/internal/streamingnode/server/resource/test_utility.go index a287d856933d1..a05ec41c697f0 100644 --- a/internal/streamingnode/server/resource/test_utility.go +++ b/internal/streamingnode/server/resource/test_utility.go @@ -6,10 +6,10 @@ package resource import ( "testing" - "github.com/milvus-io/milvus/internal/streamingnode/server/resource/idalloc" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/segment/stats" tinspector "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/timetick/inspector" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/internal/util/idalloc" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/syncutil" ) diff --git a/internal/streamingnode/server/service/handler/producer/produce_server.go b/internal/streamingnode/server/service/handler/producer/produce_server.go index 0bd9b35721dae..4075efe80882f 100644 --- a/internal/streamingnode/server/service/handler/producer/produce_server.go +++ b/internal/streamingnode/server/service/handler/producer/produce_server.go @@ -13,7 +13,6 @@ import ( "github.com/milvus-io/milvus/internal/util/streamingutil/service/contextutil" "github.com/milvus-io/milvus/internal/util/streamingutil/status" "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb" "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/types" @@ -221,20 +220,9 @@ func (p *ProduceServer) sendProduceResult(reqID int64, appendResult *wal.AppendR } if err != nil { p.logger.Warn("append message to wal failed", zap.Int64("requestID", reqID), zap.Error(err)) - resp.Response = &streamingpb.ProduceMessageResponse_Error{ - Error: status.AsStreamingError(err).AsPBError(), - } + resp.Response = &streamingpb.ProduceMessageResponse_Error{Error: status.AsStreamingError(err).AsPBError()} } else { - resp.Response = &streamingpb.ProduceMessageResponse_Result{ - Result: &streamingpb.ProduceMessageResponseResult{ - Id: &messagespb.MessageID{ - Id: appendResult.MessageID.Marshal(), - }, - Timetick: appendResult.TimeTick, - TxnContext: appendResult.TxnCtx.IntoProto(), - Extra: appendResult.Extra, - }, - } + resp.Response = &streamingpb.ProduceMessageResponse_Result{Result: appendResult.IntoProto()} } // If server context is canceled, it means the stream has been closed. diff --git a/internal/streamingnode/server/wal/adaptor/wal_test.go b/internal/streamingnode/server/wal/adaptor/wal_test.go index b217af0d521e9..8a222b04b35c0 100644 --- a/internal/streamingnode/server/wal/adaptor/wal_test.go +++ b/internal/streamingnode/server/wal/adaptor/wal_test.go @@ -21,10 +21,10 @@ import ( "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/streamingnode/server/resource" - "github.com/milvus-io/milvus/internal/streamingnode/server/resource/idalloc" "github.com/milvus-io/milvus/internal/streamingnode/server/wal" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/registry" internaltypes "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/internal/util/idalloc" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/options" "github.com/milvus-io/milvus/pkg/streaming/util/types" diff --git a/internal/streamingnode/server/wal/interceptors/segment/manager/pchannel_manager_test.go b/internal/streamingnode/server/wal/interceptors/segment/manager/pchannel_manager_test.go index 7093f1139e89a..4497551c2bc58 100644 --- a/internal/streamingnode/server/wal/interceptors/segment/manager/pchannel_manager_test.go +++ b/internal/streamingnode/server/wal/interceptors/segment/manager/pchannel_manager_test.go @@ -15,12 +15,12 @@ import ( "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/streamingnode/server/resource" - "github.com/milvus-io/milvus/internal/streamingnode/server/resource/idalloc" "github.com/milvus-io/milvus/internal/streamingnode/server/wal" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/segment/inspector" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/segment/stats" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/txn" internaltypes "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/internal/util/idalloc" "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/rmq" diff --git a/internal/streamingnode/server/wal/interceptors/timetick/timetick_message.go b/internal/streamingnode/server/wal/interceptors/timetick/timetick_message.go index 2ed586859fcb1..a705663ad8a77 100644 --- a/internal/streamingnode/server/wal/interceptors/timetick/timetick_message.go +++ b/internal/streamingnode/server/wal/interceptors/timetick/timetick_message.go @@ -21,7 +21,7 @@ func NewTimeTickMsg(ts uint64, lastConfirmedMessageID message.MessageID, sourceI commonpbutil.WithSourceID(sourceID), ), }). - WithBroadcast(). + WithAllVChannel(). BuildMutable() if err != nil { return nil, err diff --git a/internal/streamingnode/server/resource/idalloc/allocator.go b/internal/util/idalloc/allocator.go similarity index 100% rename from internal/streamingnode/server/resource/idalloc/allocator.go rename to internal/util/idalloc/allocator.go diff --git a/internal/streamingnode/server/resource/idalloc/allocator_test.go b/internal/util/idalloc/allocator_test.go similarity index 100% rename from internal/streamingnode/server/resource/idalloc/allocator_test.go rename to internal/util/idalloc/allocator_test.go diff --git a/internal/streamingnode/server/resource/idalloc/basic_allocator.go b/internal/util/idalloc/basic_allocator.go similarity index 100% rename from internal/streamingnode/server/resource/idalloc/basic_allocator.go rename to internal/util/idalloc/basic_allocator.go diff --git a/internal/streamingnode/server/resource/idalloc/basic_allocator_test.go b/internal/util/idalloc/basic_allocator_test.go similarity index 100% rename from internal/streamingnode/server/resource/idalloc/basic_allocator_test.go rename to internal/util/idalloc/basic_allocator_test.go diff --git a/internal/streamingnode/server/resource/idalloc/mallocator.go b/internal/util/idalloc/mallocator.go similarity index 100% rename from internal/streamingnode/server/resource/idalloc/mallocator.go rename to internal/util/idalloc/mallocator.go diff --git a/internal/streamingnode/server/resource/idalloc/test_mock_root_coord_client.go b/internal/util/idalloc/test_mock_root_coord_client.go similarity index 100% rename from internal/streamingnode/server/resource/idalloc/test_mock_root_coord_client.go rename to internal/util/idalloc/test_mock_root_coord_client.go diff --git a/pkg/streaming/proto/messages.proto b/pkg/streaming/proto/messages.proto index 091e59042729c..843556ce4541c 100644 --- a/pkg/streaming/proto/messages.proto +++ b/pkg/streaming/proto/messages.proto @@ -248,3 +248,8 @@ message RMQMessageLayout { bytes payload = 1; // message body map properties = 2; // message properties } + +// VChannels is a layout to represent the virtual channels for broadcast. +message VChannels { + repeated string vchannels = 1; +} \ No newline at end of file diff --git a/pkg/streaming/proto/streaming.proto b/pkg/streaming/proto/streaming.proto index e4a6943ae2645..0a7debc9dad5c 100644 --- a/pkg/streaming/proto/streaming.proto +++ b/pkg/streaming/proto/streaming.proto @@ -60,18 +60,48 @@ message VersionPair { int64 local = 2; } +// BroadcastTaskState is the state of the broadcast task. +enum BroadcastTaskState { + BROADCAST_TASK_STATE_UNKNOWN = 0; // should never used. + BROADCAST_TASK_STATE_PENDING = 1; // task is pending. + BROADCAST_TASK_STATE_DONE = 2; // task is done, the message is broadcasted, and the persisted task can be cleared. +} + +// BroadcastTask is the task to broadcast the message. +message BroadcastTask { + int64 task_id = 1; // task id. + messages.Message message = 2; // message to be broadcast. + BroadcastTaskState state = 3; // state of the task. +} + // // Milvus Service // -service StreamingCoordStateService { +service StreamingNodeStateService { rpc GetComponentStates(milvus.GetComponentStatesRequest) returns (milvus.ComponentStates) {} } -service StreamingNodeStateService { - rpc GetComponentStates(milvus.GetComponentStatesRequest) - returns (milvus.ComponentStates) {} +// +// StreamingCoordBroadcastService +// + +// StreamingCoordBroadcastService is the broadcast service for streaming coord. +service StreamingCoordBroadcastService { + // Broadcast receives broadcast messages from other component and make sure that the message is broadcast to all wal. + // It performs an atomic broadcast to all wal, achieve eventual consistency. + rpc Broadcast(BroadcastRequest) returns (BroadcastResponse) {} +} + +// BroadcastRequest is the request of the Broadcast RPC. +message BroadcastRequest { + messages.Message message = 1; // message to be broadcast. +} + +// BroadcastResponse is the response of the Broadcast RPC. +message BroadcastResponse { + map results = 1; } // diff --git a/pkg/streaming/util/message/builder.go b/pkg/streaming/util/message/builder.go index 32bdad9db6482..0f941c6851bd4 100644 --- a/pkg/streaming/util/message/builder.go +++ b/pkg/streaming/util/message/builder.go @@ -7,16 +7,32 @@ import ( "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" + "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb" "github.com/milvus-io/milvus/pkg/util/tsoutil" + "github.com/milvus-io/milvus/pkg/util/typeutil" ) // NewMutableMessage creates a new mutable message. // !!! Only used at server side for streamingnode internal service, don't use it at client side. func NewMutableMessage(payload []byte, properties map[string]string) MutableMessage { - return &messageImpl{ + m := &messageImpl{ payload: payload, properties: properties, } + // make a assertion by vchannel function. + m.assertNotBroadcast() + return m +} + +// NewBroadcastMutableMessage creates a new broadcast mutable message. +// !!! Only used at server side for streamingcoord internal service, don't use it at client side. +func NewBroadcastMutableMessage(payload []byte, properties map[string]string) BroadcastMutableMessage { + m := &messageImpl{ + payload: payload, + properties: properties, + } + m.assertBroadcast() + return m } // NewImmutableMessage creates a new immutable message. @@ -82,10 +98,10 @@ func newMutableMessageBuilder[H proto.Message, B proto.Message](v Version) *muta // mutableMesasgeBuilder is the builder for message. type mutableMesasgeBuilder[H proto.Message, B proto.Message] struct { - header H - body B - properties propertiesImpl - broadcast bool + header H + body B + properties propertiesImpl + allVChannel bool } // WithMessageHeader creates a new builder with determined message type. @@ -102,16 +118,41 @@ func (b *mutableMesasgeBuilder[H, B]) WithBody(body B) *mutableMesasgeBuilder[H, // WithVChannel creates a new builder with virtual channel. func (b *mutableMesasgeBuilder[H, B]) WithVChannel(vchannel string) *mutableMesasgeBuilder[H, B] { - if b.broadcast { - panic("a broadcast message cannot hold vchannel") + if b.allVChannel { + panic("a all vchannel message cannot set up vchannel property") } b.WithProperty(messageVChannel, vchannel) return b } // WithBroadcast creates a new builder with broadcast property. -func (b *mutableMesasgeBuilder[H, B]) WithBroadcast() *mutableMesasgeBuilder[H, B] { - b.broadcast = true +func (b *mutableMesasgeBuilder[H, B]) WithBroadcast(vchannels []string) *mutableMesasgeBuilder[H, B] { + if len(vchannels) < 1 { + panic("broadcast message must have at least one vchannel") + } + if b.allVChannel { + panic("a all vchannel message cannot set up vchannel property") + } + if b.properties.Exist(messageVChannel) { + panic("a broadcast message cannot set up vchannel property") + } + deduplicated := typeutil.NewSet(vchannels...) + vcs, err := EncodeProto(&messagespb.VChannels{ + Vchannels: deduplicated.Collect(), + }) + if err != nil { + panic("failed to encode vchannels") + } + b.properties.Set(messageVChannels, vcs) + return b +} + +// WithAllVChannel creates a new builder with all vchannel property. +func (b *mutableMesasgeBuilder[H, B]) WithAllVChannel() *mutableMesasgeBuilder[H, B] { + if b.properties.Exist(messageVChannel) || b.properties.Exist(messageVChannels) { + panic("a vchannel or broadcast message cannot set up all vchannel property") + } + b.allVChannel = true return b } @@ -135,6 +176,34 @@ func (b *mutableMesasgeBuilder[H, B]) WithProperties(kvs map[string]string) *mut // Panic if not set payload and message type. // should only used at client side. func (b *mutableMesasgeBuilder[H, B]) BuildMutable() (MutableMessage, error) { + if !b.allVChannel && !b.properties.Exist(messageVChannel) { + panic("a non broadcast message builder not ready for vchannel field") + } + + msg, err := b.build() + if err != nil { + return nil, err + } + return msg, nil +} + +// BuildBroadcast builds a broad mutable message. +// Panic if not set payload and message type. +// should only used at client side. +func (b *mutableMesasgeBuilder[H, B]) BuildBroadcast() (BroadcastMutableMessage, error) { + if !b.properties.Exist(messageVChannels) { + panic("a broadcast message builder not ready for vchannel field") + } + + msg, err := b.build() + if err != nil { + return nil, err + } + return msg, nil +} + +// build builds a message. +func (b *mutableMesasgeBuilder[H, B]) build() (*messageImpl, error) { // payload and header must be a pointer if reflect.ValueOf(b.header).IsNil() { panic("message builder not ready for header field") @@ -142,9 +211,6 @@ func (b *mutableMesasgeBuilder[H, B]) BuildMutable() (MutableMessage, error) { if reflect.ValueOf(b.body).IsNil() { panic("message builder not ready for body field") } - if !b.broadcast && !b.properties.Exist(messageVChannel) { - panic("a non broadcast message builder not ready for vchannel field") - } // setup header. sp, err := EncodeProto(b.header) diff --git a/pkg/streaming/util/message/message.go b/pkg/streaming/util/message/message.go index 733ed568d8450..49a7361c82fcd 100644 --- a/pkg/streaming/util/message/message.go +++ b/pkg/streaming/util/message/message.go @@ -29,11 +29,6 @@ type BasicMessage interface { // Should be used with read-only promise. Properties() RProperties - // VChannel returns the virtual channel of current message. - // Available only when the message's version greater than 0. - // Return "" if message is broadcasted. - VChannel() string - // TimeTick returns the time tick of current message. // Available only when the message's version greater than 0. // Otherwise, it will panic. @@ -52,6 +47,11 @@ type BasicMessage interface { type MutableMessage interface { BasicMessage + // VChannel returns the virtual channel of current message. + // Available only when the message's version greater than 0. + // Return "" if message is can be seen by all vchannels on the pchannel. + VChannel() string + // WithBarrierTimeTick sets the barrier time tick of current message. // these time tick is used to promised the message will be sent after that time tick. // and the message which timetick is less than it will never concurrent append with it. @@ -82,6 +82,19 @@ type MutableMessage interface { IntoImmutableMessage(msgID MessageID) ImmutableMessage } +// BroadcastMutableMessage is the broadcast message interface. +// Indicated the message is broadcasted on various vchannels. +type BroadcastMutableMessage interface { + BasicMessage + + // BroadcastVChannels returns the target vchannels of the message broadcast. + // Those vchannels can be on multi pchannels. + BroadcastVChannels() []string + + // SplitIntoMutableMessage splits the broadcast message into multiple mutable messages. + SplitIntoMutableMessage() []MutableMessage +} + // ImmutableMessage is the read-only message interface. // Once a message is persistent by wal or temporary generated by wal, it will be immutable. type ImmutableMessage interface { @@ -90,6 +103,11 @@ type ImmutableMessage interface { // WALName returns the name of message related wal. WALName() string + // VChannel returns the virtual channel of current message. + // Available only when the message's version greater than 0. + // Return "" if message is can be seen by all vchannels on the pchannel. + VChannel() string + // MessageID returns the message id of current message. MessageID() MessageID diff --git a/pkg/streaming/util/message/message_impl.go b/pkg/streaming/util/message/message_impl.go index 41e9ac0379af2..7e4a4c0be2498 100644 --- a/pkg/streaming/util/message/message_impl.go +++ b/pkg/streaming/util/message/message_impl.go @@ -141,8 +141,11 @@ func (m *messageImpl) BarrierTimeTick() uint64 { } // VChannel returns the vchannel of current message. -// If the message is broadcasted, the vchannel will be empty. +// If the message is a all channel message, it will return "". +// If the message is a broadcast message, it will panic. func (m *messageImpl) VChannel() string { + m.assertNotBroadcast() + value, ok := m.properties.Get(messageVChannel) if !ok { return "" @@ -150,6 +153,60 @@ func (m *messageImpl) VChannel() string { return value } +// BroadcastVChannels returns the vchannels of current message that want to broadcast. +// If the message is not a broadcast message, it will panic. +func (m *messageImpl) BroadcastVChannels() []string { + m.assertBroadcast() + + value, _ := m.properties.Get(messageVChannels) + vcs := &messagespb.VChannels{} + if err := DecodeProto(value, vcs); err != nil { + panic("can not decode vchannels") + } + return vcs.Vchannels +} + +// SplitIntoMutableMessage splits the current broadcast message into multiple messages. +func (m *messageImpl) SplitIntoMutableMessage() []MutableMessage { + vchannels := m.BroadcastVChannels() + + vchannelExist := make(map[string]struct{}, len(vchannels)) + msgs := make([]MutableMessage, 0, len(vchannels)) + for _, vchannel := range vchannels { + newPayload := make([]byte, len(m.payload)) + copy(newPayload, m.payload) + + newProperties := make(propertiesImpl, len(m.properties)) + for key, val := range m.properties { + if key != messageVChannels { + newProperties.Set(key, val) + } + } + newProperties.Set(messageVChannel, vchannel) + if _, ok := vchannelExist[vchannel]; ok { + panic("there's a bug in the message codes, duplicate vchannel in broadcast message") + } + msgs = append(msgs, &messageImpl{ + payload: newPayload, + properties: newProperties, + }) + vchannelExist[vchannel] = struct{}{} + } + return msgs +} + +func (m *messageImpl) assertNotBroadcast() { + if m.properties.Exist(messageVChannels) { + panic("current message is a broadcast message") + } +} + +func (m *messageImpl) assertBroadcast() { + if !m.properties.Exist(messageVChannels) { + panic("current message is not a broadcast message") + } +} + type immutableMessageImpl struct { messageImpl id MessageID diff --git a/pkg/streaming/util/message/properties.go b/pkg/streaming/util/message/properties.go index 575c7d2146b80..3f0d120e32fd4 100644 --- a/pkg/streaming/util/message/properties.go +++ b/pkg/streaming/util/message/properties.go @@ -10,6 +10,7 @@ const ( messageLastConfirmed = "_lc" // message last confirmed message id. messageLastConfirmedIDSameWithMessageID = "_lcs" // message last confirmed message id is the same with message id. messageVChannel = "_vc" // message virtual channel. + messageVChannels = "_vcs" // message virtual channels for broadcast message. messageHeader = "_h" // specialized message header. messageTxnContext = "_tx" // transaction context. ) diff --git a/pkg/streaming/util/types/streaming_node.go b/pkg/streaming/util/types/streaming_node.go index 4c6a13e699d17..0cca5798e19d1 100644 --- a/pkg/streaming/util/types/streaming_node.go +++ b/pkg/streaming/util/types/streaming_node.go @@ -7,6 +7,7 @@ import ( "google.golang.org/protobuf/proto" "google.golang.org/protobuf/types/known/anypb" + "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb" "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/util/typeutil" @@ -88,6 +89,16 @@ func (n *StreamingNodeStatus) ErrorOfNode() error { return n.Err } +// BroadcastAppendResult is the result of broadcast append operation. +type BroadcastAppendResult struct { + AppendResults map[string]*AppendResult // make the channel name to the append result. +} + +// GetAppendResult returns the append result of the given channel. +func (r *BroadcastAppendResult) GetAppendResult(channelName string) *AppendResult { + return r.AppendResults[channelName] +} + // AppendResult is the result of append operation. type AppendResult struct { // MessageID is generated by underlying walimpls. @@ -112,3 +123,15 @@ func (r *AppendResult) GetExtra(m proto.Message) error { AllowPartial: true, }) } + +// IntoProto converts the append result to proto. +func (r *AppendResult) IntoProto() *streamingpb.ProduceMessageResponseResult { + return &streamingpb.ProduceMessageResponseResult{ + Id: &messagespb.MessageID{ + Id: r.MessageID.Marshal(), + }, + Timetick: r.TimeTick, + TxnContext: r.TxnCtx.IntoProto(), + Extra: r.Extra, + } +} diff --git a/pkg/util/contextutil/context_util.go b/pkg/util/contextutil/context_util.go index 8cf699b43079b..2bded437d1ec5 100644 --- a/pkg/util/contextutil/context_util.go +++ b/pkg/util/contextutil/context_util.go @@ -121,3 +121,15 @@ func WithDeadlineCause(parent context.Context, deadline time.Time, err error) (c cancel(context.Canceled) } } + +// MergeContext create a cancellation context that cancels when any of the given contexts are canceled. +func MergeContext(ctx1 context.Context, ctx2 context.Context) (context.Context, context.CancelFunc) { + ctx, cancel := context.WithCancelCause(ctx1) + stop := context.AfterFunc(ctx2, func() { + cancel(context.Cause(ctx2)) + }) + return ctx, func() { + stop() + cancel(context.Canceled) + } +} diff --git a/pkg/util/typeutil/backoff_timer.go b/pkg/util/typeutil/backoff_timer.go index dd26b136fee8d..997ccb2839211 100644 --- a/pkg/util/typeutil/backoff_timer.go +++ b/pkg/util/typeutil/backoff_timer.go @@ -94,3 +94,49 @@ func (t *BackoffTimer) NextInterval() time.Duration { } return t.configFetcher.DefaultInterval() } + +// NewBackoffWithInstant creates a new backoff with instant +func NewBackoffWithInstant(fetcher BackoffTimerConfigFetcher) *BackoffWithInstant { + cfg := fetcher.BackoffConfig() + defaultInterval := fetcher.DefaultInterval() + backoff := backoff.NewExponentialBackOff() + backoff.InitialInterval = cfg.InitialInterval + backoff.Multiplier = cfg.Multiplier + backoff.MaxInterval = cfg.MaxInterval + backoff.MaxElapsedTime = defaultInterval + backoff.Stop = defaultInterval + backoff.Reset() + return &BackoffWithInstant{ + backoff: backoff, + nextInstant: time.Now(), + } +} + +// BackoffWithInstant is a backoff with instant. +// A instant can be recorded with `UpdateInstantWithNextBackOff` +// NextInstant can be used to make priority decision. +type BackoffWithInstant struct { + backoff *backoff.ExponentialBackOff + nextInstant time.Time +} + +// NextInstant returns the next instant +func (t *BackoffWithInstant) NextInstant() time.Time { + return t.nextInstant +} + +// NextInterval returns the next interval +func (t *BackoffWithInstant) NextInterval() time.Duration { + return time.Until(t.nextInstant) +} + +// NextTimer returns the next timer and the duration of the timer +func (t *BackoffWithInstant) NextTimer() (<-chan time.Time, time.Duration) { + next := time.Until(t.nextInstant) + return time.After(next), next +} + +// UpdateInstantWithNextBackOff updates the next instant with next backoff +func (t *BackoffWithInstant) UpdateInstantWithNextBackOff() { + t.nextInstant = time.Now().Add(t.backoff.NextBackOff()) +} From 92a2d608acbc6e7c3bd85edb450d0dc5ae8aa64c Mon Sep 17 00:00:00 2001 From: smellthemoon <64083300+smellthemoon@users.noreply.github.com> Date: Thu, 9 Jan 2025 19:27:03 +0800 Subject: [PATCH 04/34] fix: Bulk insert failed when the nullable/default_value field is not exist (#39063) #39036 Signed-off-by: lixinguo Co-authored-by: lixinguo --- internal/util/importutilv2/json/row_parser.go | 12 ++++++++++++ .../util/importutilv2/json/row_parser_test.go | 18 ++++++++++++++++++ 2 files changed, 30 insertions(+) diff --git a/internal/util/importutilv2/json/row_parser.go b/internal/util/importutilv2/json/row_parser.go index 391427efbd0f2..326a5270abb55 100644 --- a/internal/util/importutilv2/json/row_parser.go +++ b/internal/util/importutilv2/json/row_parser.go @@ -126,6 +126,18 @@ func (r *rowParser) Parse(raw any) (Row, error) { } } for fieldName, fieldID := range r.name2FieldID { + if _, ok = row[fieldID]; !ok { + if r.id2Field[fieldID].GetNullable() { + row[fieldID] = nil + } + if r.id2Field[fieldID].GetDefaultValue() != nil { + data, err := nullutil.GetDefaultValue(r.id2Field[fieldID]) + if err != nil { + return nil, err + } + row[fieldID] = data + } + } if _, ok = row[fieldID]; !ok { return nil, merr.WrapErrImportFailed(fmt.Sprintf("value of field '%s' is missed", fieldName)) } diff --git a/internal/util/importutilv2/json/row_parser_test.go b/internal/util/importutilv2/json/row_parser_test.go index 96fa48beccec5..89b934224cdc5 100644 --- a/internal/util/importutilv2/json/row_parser_test.go +++ b/internal/util/importutilv2/json/row_parser_test.go @@ -72,6 +72,23 @@ func TestRowParser_Parse_Valid(t *testing.T) { }, }, }, + { + FieldID: 6, + Name: "null_fid", + DataType: schemapb.DataType_VarChar, + Nullable: true, + DefaultValue: &schemapb.ValueField{ + Data: &schemapb.ValueField_StringData{ + StringData: "a", + }, + }, + TypeParams: []*commonpb.KeyValuePair{ + { + Key: "max_length", + Value: "256", + }, + }, + }, }, } r, err := NewRowParser(schema) @@ -185,6 +202,7 @@ func TestRowParser_Parse_Invalid(t *testing.T) { {name: `{"id": 1, "vector": [], "arrayField": [1, 2, 3, 4], "x": 6, "$meta": [], "name": "test"}`, expectErr: "not a JSON object"}, {name: `{"id": 1, "vector": [], "arrayField": [1, 2, 3, 4], "x": 8, "$meta": "{\"y\": 8}", "name": "testName"}`, expectErr: "value length 8 exceeds max_length 4"}, {name: `{"id": 1, "vector": [], "arrayField": [1, 2, 3, 4, 5], "x": 8, "$meta": "{\"z\": 9}", "name": "test"}`, expectErr: "array capacity 5 exceeds max_capacity 4"}, + {name: `{"id": 1, "vector": [], "x": 8, "$meta": "{\"z\": 9}", "name": "test"}`, expectErr: "value of field 'arrayField' is missed"}, } for _, c := range cases { From f03a85725a37d807069b31ad4d583e3cac8e9ceb Mon Sep 17 00:00:00 2001 From: jaime Date: Thu, 9 Jan 2025 19:40:59 +0800 Subject: [PATCH 05/34] enhance: add db name in replica (#38672) issue: #36621 Signed-off-by: jaime --- .../datacoord/compaction_task_clustering.go | 9 ++++ internal/datacoord/compaction_task_l0.go | 22 +++++----- internal/datacoord/compaction_task_meta.go | 4 +- internal/datacoord/compaction_task_mix.go | 26 ++++++------ internal/datacoord/index_meta.go | 4 +- internal/datacoord/metrics_info.go | 16 +++---- internal/datacoord/server.go | 6 +-- internal/datanode/data_node.go | 6 ++- .../pipeline/flow_graph_manager.go | 15 +++++-- .../pipeline/flow_graph_manager_test.go | 17 ++++++-- .../flushcommon/pipeline/mock_fgmanager.go | 42 ++++++++++--------- internal/metastore/model/segment_index.go | 3 ++ internal/proto/index_coord.proto | 1 + internal/proto/query_coord.proto | 1 + internal/proxy/http_req_impl.go | 10 +++-- internal/proxy/impl.go | 8 ++-- internal/querycoordv2/handlers.go | 10 ++--- internal/querycoordv2/job/job_load.go | 23 ++++++---- .../querycoordv2/meta/channel_dist_manager.go | 21 +++++++--- .../meta/channel_dist_manager_test.go | 2 +- .../querycoordv2/meta/collection_manager.go | 11 ++++- internal/querycoordv2/meta/dist_manager.go | 8 ++-- .../querycoordv2/meta/dist_manager_test.go | 11 ++++- .../querycoordv2/meta/leader_view_manager.go | 17 +++++++- .../meta/leader_view_manager_test.go | 11 +++-- .../querycoordv2/meta/mock_target_manager.go | 21 +++++----- internal/querycoordv2/meta/replica_manager.go | 13 +++++- .../querycoordv2/meta/replica_manager_test.go | 23 +++++++++- .../querycoordv2/meta/segment_dist_manager.go | 5 ++- internal/querycoordv2/meta/target.go | 17 +++++--- internal/querycoordv2/meta/target_manager.go | 6 +-- .../querycoordv2/meta/target_manager_test.go | 10 ++++- internal/querycoordv2/server.go | 9 ++-- internal/querycoordv2/utils/meta.go | 1 - internal/querynodev2/metrics_info.go | 11 +++-- internal/querynodev2/metrics_info_test.go | 7 ++-- internal/querynodev2/pipeline/manager.go | 7 +++- internal/querynodev2/server.go | 6 ++- pkg/util/metricsinfo/metric_request.go | 28 +++++++++---- pkg/util/metricsinfo/metrics_info.go | 10 +++-- 40 files changed, 315 insertions(+), 163 deletions(-) diff --git a/internal/datacoord/compaction_task_clustering.go b/internal/datacoord/compaction_task_clustering.go index 53a5fbd97e3a6..4f939fefbf043 100644 --- a/internal/datacoord/compaction_task_clustering.go +++ b/internal/datacoord/compaction_task_clustering.go @@ -716,6 +716,15 @@ func (t *clusteringCompactionTask) ShadowClone(opts ...compactionTaskOpt) *datap } func (t *clusteringCompactionTask) updateAndSaveTaskMeta(opts ...compactionTaskOpt) error { + // if task state is completed, cleaned, failed, timeout, then do append end time and save + if t.GetTaskProto().State == datapb.CompactionTaskState_completed || + t.GetTaskProto().State == datapb.CompactionTaskState_cleaned || + t.GetTaskProto().State == datapb.CompactionTaskState_failed || + t.GetTaskProto().State == datapb.CompactionTaskState_timeout { + ts := time.Now().Unix() + opts = append(opts, setEndTime(ts)) + } + task := t.ShadowClone(opts...) err := t.saveTaskMeta(task) if err != nil { diff --git a/internal/datacoord/compaction_task_l0.go b/internal/datacoord/compaction_task_l0.go index e7535bca8e4b5..96bdb207059cd 100644 --- a/internal/datacoord/compaction_task_l0.go +++ b/internal/datacoord/compaction_task_l0.go @@ -130,9 +130,6 @@ func (t *l0CompactionTask) processExecuting() bool { log.Warn("l0CompactionTask failed to get compaction result", zap.Error(err)) return false } - - ts := time.Now().Unix() - updateOps := []compactionTaskOpt{setEndTime(ts)} switch result.GetState() { case datapb.CompactionTaskState_completed: t.result = result @@ -141,15 +138,13 @@ func (t *l0CompactionTask) processExecuting() bool { return false } - updateOps = append(updateOps, setState(datapb.CompactionTaskState_meta_saved)) - if err := t.updateAndSaveTaskMeta(updateOps...); err != nil { + if err := t.updateAndSaveTaskMeta(setState(datapb.CompactionTaskState_meta_saved)); err != nil { log.Warn("l0CompactionTask failed to save task meta_saved state", zap.Error(err)) return false } return t.processMetaSaved() case datapb.CompactionTaskState_failed: - updateOps = append(updateOps, setState(datapb.CompactionTaskState_failed)) - if err := t.updateAndSaveTaskMeta(updateOps...); err != nil { + if err := t.updateAndSaveTaskMeta(setState(datapb.CompactionTaskState_failed)); err != nil { log.Warn("l0CompactionTask failed to set task failed state", zap.Error(err)) return false } @@ -159,9 +154,7 @@ func (t *l0CompactionTask) processExecuting() bool { } func (t *l0CompactionTask) processMetaSaved() bool { - ts := time.Now().Unix() - updateOps := []compactionTaskOpt{setEndTime(ts), setState(datapb.CompactionTaskState_completed)} - err := t.updateAndSaveTaskMeta(updateOps...) + err := t.updateAndSaveTaskMeta(setState(datapb.CompactionTaskState_completed)) if err != nil { log.Warn("l0CompactionTask unable to processMetaSaved", zap.Int64("planID", t.GetTaskProto().GetPlanID()), zap.Error(err)) return false @@ -358,6 +351,15 @@ func (t *l0CompactionTask) SaveTaskMeta() error { } func (t *l0CompactionTask) updateAndSaveTaskMeta(opts ...compactionTaskOpt) error { + // if task state is completed, cleaned, failed, timeout, then do append end time and save + if t.GetTaskProto().State == datapb.CompactionTaskState_completed || + t.GetTaskProto().State == datapb.CompactionTaskState_cleaned || + t.GetTaskProto().State == datapb.CompactionTaskState_failed || + t.GetTaskProto().State == datapb.CompactionTaskState_timeout { + ts := time.Now().Unix() + opts = append(opts, setEndTime(ts)) + } + task := t.ShadowClone(opts...) err := t.saveTaskMeta(task) if err != nil { diff --git a/internal/datacoord/compaction_task_meta.go b/internal/datacoord/compaction_task_meta.go index 33a2ad20b59cb..be18fd0884fae 100644 --- a/internal/datacoord/compaction_task_meta.go +++ b/internal/datacoord/compaction_task_meta.go @@ -43,8 +43,8 @@ func newCompactionTaskStats(task *datapb.CompactionTask) *metricsinfo.Compaction Type: task.Type.String(), State: task.State.String(), FailReason: task.FailReason, - StartTime: typeutil.TimestampToString(uint64(task.StartTime)), - EndTime: typeutil.TimestampToString(uint64(task.EndTime)), + StartTime: typeutil.TimestampToString(uint64(task.StartTime) * 1000), + EndTime: typeutil.TimestampToString(uint64(task.EndTime) * 1000), TotalRows: task.TotalRows, InputSegments: lo.Map(task.InputSegments, func(t int64, i int) string { return strconv.FormatInt(t, 10) diff --git a/internal/datacoord/compaction_task_mix.go b/internal/datacoord/compaction_task_mix.go index c61756949574a..90b43432e065d 100644 --- a/internal/datacoord/compaction_task_mix.go +++ b/internal/datacoord/compaction_task_mix.go @@ -97,9 +97,7 @@ func (t *mixCompactionTask) processPipelining() bool { func (t *mixCompactionTask) processMetaSaved() bool { log := log.With(zap.Int64("triggerID", t.GetTaskProto().GetTriggerID()), zap.Int64("PlanID", t.GetTaskProto().GetPlanID()), zap.Int64("collectionID", t.GetTaskProto().GetCollectionID())) - ts := time.Now().Unix() - updateOps := []compactionTaskOpt{setEndTime(ts), setState(datapb.CompactionTaskState_completed)} - if err := t.updateAndSaveTaskMeta(updateOps...); err != nil { + if err := t.updateAndSaveTaskMeta(setState(datapb.CompactionTaskState_completed)); err != nil { log.Warn("mixCompactionTask failed to proccessMetaSaved", zap.Error(err)) return false } @@ -119,15 +117,12 @@ func (t *mixCompactionTask) processExecuting() bool { log.Warn("mixCompactionTask failed to get compaction result", zap.Error(err)) return false } - - ts := time.Now().Unix() - failedUpdateOps := []compactionTaskOpt{setEndTime(ts), setState(datapb.CompactionTaskState_failed)} switch result.GetState() { case datapb.CompactionTaskState_completed: t.result = result if len(result.GetSegments()) == 0 { log.Info("illegal compaction results") - err := t.updateAndSaveTaskMeta(failedUpdateOps...) + err := t.updateAndSaveTaskMeta(setState(datapb.CompactionTaskState_failed)) if err != nil { log.Warn("mixCompactionTask failed to setState failed", zap.Error(err)) return false @@ -137,7 +132,7 @@ func (t *mixCompactionTask) processExecuting() bool { if err := t.saveSegmentMeta(); err != nil { log.Warn("mixCompactionTask failed to save segment meta", zap.Error(err)) if errors.Is(err, merr.ErrIllegalCompactionPlan) { - err := t.updateAndSaveTaskMeta(failedUpdateOps...) + err = t.updateAndSaveTaskMeta(setState(datapb.CompactionTaskState_failed)) if err != nil { log.Warn("mixCompactionTask failed to setState failed", zap.Error(err)) return false @@ -154,7 +149,7 @@ func (t *mixCompactionTask) processExecuting() bool { return t.processMetaSaved() case datapb.CompactionTaskState_failed: log.Info("mixCompactionTask fail in datanode") - err := t.updateAndSaveTaskMeta(failedUpdateOps...) + err := t.updateAndSaveTaskMeta(setState(datapb.CompactionTaskState_failed)) if err != nil { log.Warn("fail to updateAndSaveTaskMeta") } @@ -240,10 +235,8 @@ func (t *mixCompactionTask) processCompleted() bool { t.resetSegmentCompacting() UpdateCompactionSegmentSizeMetrics(t.result.GetSegments()) + log.Info("mixCompactionTask processCompleted done") - task := t.GetTaskProto() - log.Info("mixCompactionTask processCompleted done", - zap.Int64("planID", task.GetPlanID()), zap.Duration("costs", time.Duration(task.GetEndTime()-task.GetStartTime())*time.Second)) return true } @@ -289,6 +282,15 @@ func (t *mixCompactionTask) doClean() error { } func (t *mixCompactionTask) updateAndSaveTaskMeta(opts ...compactionTaskOpt) error { + // if task state is completed, cleaned, failed, timeout, then do append end time and save + if t.GetTaskProto().State == datapb.CompactionTaskState_completed || + t.GetTaskProto().State == datapb.CompactionTaskState_cleaned || + t.GetTaskProto().State == datapb.CompactionTaskState_failed || + t.GetTaskProto().State == datapb.CompactionTaskState_timeout { + ts := time.Now().Unix() + opts = append(opts, setEndTime(ts)) + } + task := t.ShadowClone(opts...) err := t.saveTaskMeta(task) if err != nil { diff --git a/internal/datacoord/index_meta.go b/internal/datacoord/index_meta.go index bc12f4f88da1c..37e744bd8a4d5 100644 --- a/internal/datacoord/index_meta.go +++ b/internal/datacoord/index_meta.go @@ -76,8 +76,8 @@ func newIndexTaskStats(s *model.SegmentIndex) *metricsinfo.IndexTaskStats { FailReason: s.FailReason, IndexSize: s.IndexSize, IndexVersion: s.IndexVersion, - CreatedUTCTime: typeutil.TimestampToString(s.CreatedUTCTime), - FinishedUTCTime: typeutil.TimestampToString(s.FinishedUTCTime), + CreatedUTCTime: typeutil.TimestampToString(s.CreatedUTCTime * 1000), + FinishedUTCTime: typeutil.TimestampToString(s.FinishedUTCTime * 1000), } } diff --git a/internal/datacoord/metrics_info.go b/internal/datacoord/metrics_info.go index 0dd5c8cd720d8..394d50b89d7f2 100644 --- a/internal/datacoord/metrics_info.go +++ b/internal/datacoord/metrics_info.go @@ -137,23 +137,17 @@ func mergeChannels(dnChannels []*metricsinfo.Channel, dcChannels map[int64]map[s func (s *Server) getSegmentsJSON(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) { v := jsonReq.Get(metricsinfo.MetricRequestParamINKey) if !v.Exists() { - // default to get all segments from dataanode + // default to get all segments from datanode return s.getDataNodeSegmentsJSON(ctx, req) } in := v.String() - if in == "dn" { - // TODO: support filter by collection id + if in == metricsinfo.MetricsRequestParamsInDN { return s.getDataNodeSegmentsJSON(ctx, req) } - if in == "dc" { - v = jsonReq.Get(metricsinfo.MetricRequestParamCollectionIDKey) - collectionID := int64(0) - if v.Exists() { - collectionID = v.Int() - } - + if in == metricsinfo.MetricsRequestParamsInDC { + collectionID := metricsinfo.GetCollectionIDFromRequest(jsonReq) segments := s.meta.getSegmentsMetrics(collectionID) for _, seg := range segments { isIndexed, indexedFields := s.meta.indexMeta.GetSegmentIndexedFields(seg.CollectionID, seg.SegmentID) @@ -163,7 +157,7 @@ func (s *Server) getSegmentsJSON(ctx context.Context, req *milvuspb.GetMetricsRe bs, err := json.Marshal(segments) if err != nil { - log.Warn("marshal segment value failed", zap.Int64("collectionID", collectionID), zap.String("err", err.Error())) + log.Ctx(ctx).Warn("marshal segment value failed", zap.Int64("collectionID", collectionID), zap.String("err", err.Error())) return "", nil } return string(bs), nil diff --git a/internal/datacoord/server.go b/internal/datacoord/server.go index 383be97b77c9b..4ba5d75a03fc6 100644 --- a/internal/datacoord/server.go +++ b/internal/datacoord/server.go @@ -1164,11 +1164,7 @@ func (s *Server) registerMetricsRequest() { s.metricsRequest.RegisterMetricsRequest(metricsinfo.IndexKey, func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) { - v := jsonReq.Get(metricsinfo.MetricRequestParamCollectionIDKey) - collectionID := int64(0) - if v.Exists() { - collectionID = v.Int() - } + collectionID := metricsinfo.GetCollectionIDFromRequest(jsonReq) return s.meta.indexMeta.GetIndexJSON(collectionID), nil }) log.Ctx(s.ctx).Info("register metrics actions finished") diff --git a/internal/datanode/data_node.go b/internal/datanode/data_node.go index 7ee1b6a6be46d..bba865fbaf02a 100644 --- a/internal/datanode/data_node.go +++ b/internal/datanode/data_node.go @@ -292,12 +292,14 @@ func (node *DataNode) registerMetricsRequest() { node.metricsRequest.RegisterMetricsRequest(metricsinfo.SegmentKey, func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) { - return node.flowgraphManager.GetSegmentsJSON(), nil + collectionID := metricsinfo.GetCollectionIDFromRequest(jsonReq) + return node.flowgraphManager.GetSegmentsJSON(collectionID), nil }) node.metricsRequest.RegisterMetricsRequest(metricsinfo.ChannelKey, func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) { - return node.flowgraphManager.GetChannelsJSON(), nil + collectionID := metricsinfo.GetCollectionIDFromRequest(jsonReq) + return node.flowgraphManager.GetChannelsJSON(collectionID), nil }) log.Ctx(node.ctx).Info("register metrics actions finished") } diff --git a/internal/flushcommon/pipeline/flow_graph_manager.go b/internal/flushcommon/pipeline/flow_graph_manager.go index 8527e65ee91c6..9a93b0c7715bd 100644 --- a/internal/flushcommon/pipeline/flow_graph_manager.go +++ b/internal/flushcommon/pipeline/flow_graph_manager.go @@ -43,8 +43,8 @@ type FlowgraphManager interface { GetFlowgraphCount() int GetCollectionIDs() []int64 - GetChannelsJSON() string - GetSegmentsJSON() string + GetChannelsJSON(collectionID int64) string + GetSegmentsJSON(collectionID int64) string Close() } @@ -121,9 +121,12 @@ func (fm *fgManagerImpl) GetCollectionIDs() []int64 { } // GetChannelsJSON returns all channels in json format. -func (fm *fgManagerImpl) GetChannelsJSON() string { +func (fm *fgManagerImpl) GetChannelsJSON(collectionID int64) string { var channels []*metricsinfo.Channel fm.flowgraphs.Range(func(ch string, ds *DataSyncService) bool { + if collectionID > 0 && ds.metacache.Collection() != collectionID { + return true + } latestTimeTick := ds.timetickSender.GetLatestTimestamp(ch) channels = append(channels, &metricsinfo.Channel{ Name: ch, @@ -143,9 +146,13 @@ func (fm *fgManagerImpl) GetChannelsJSON() string { return string(ret) } -func (fm *fgManagerImpl) GetSegmentsJSON() string { +func (fm *fgManagerImpl) GetSegmentsJSON(collectionID int64) string { var segments []*metricsinfo.Segment fm.flowgraphs.Range(func(ch string, ds *DataSyncService) bool { + if collectionID > 0 && ds.metacache.Collection() != collectionID { + return true + } + meta := ds.metacache for _, segment := range meta.GetSegmentsBy() { segments = append(segments, &metricsinfo.Segment{ diff --git a/internal/flushcommon/pipeline/flow_graph_manager_test.go b/internal/flushcommon/pipeline/flow_graph_manager_test.go index 6108d9911ae79..0566cc437aa5e 100644 --- a/internal/flushcommon/pipeline/flow_graph_manager_test.go +++ b/internal/flushcommon/pipeline/flow_graph_manager_test.go @@ -193,8 +193,14 @@ func TestGetChannelsJSON(t *testing.T) { assert.NoError(t, err) expectedJSON := string(expectedBytes) - jsonResult := fm.GetChannelsJSON() + jsonResult := fm.GetChannelsJSON(0) assert.JSONEq(t, expectedJSON, jsonResult) + + jsonResult = fm.GetChannelsJSON(10) + var ret []*metricsinfo.Channel + err = json.Unmarshal([]byte(jsonResult), &ret) + assert.NoError(t, err) + assert.Equal(t, 0, len(ret)) } func TestGetSegmentJSON(t *testing.T) { @@ -228,7 +234,12 @@ func TestGetSegmentJSON(t *testing.T) { expectedJSON := string(expectedBytes) ds.metacache.AddSegment(segment, pkStatsFactory, metacache.NoneBm25StatsFactory) - jsonResult := fm.GetSegmentsJSON() - fmt.Println(jsonResult) + jsonResult := fm.GetSegmentsJSON(0) assert.JSONEq(t, expectedJSON, jsonResult) + + jsonResult = fm.GetSegmentsJSON(10) + var ret []*metricsinfo.Segment + err = json.Unmarshal([]byte(jsonResult), &ret) + assert.NoError(t, err) + assert.Equal(t, 0, len(ret)) } diff --git a/internal/flushcommon/pipeline/mock_fgmanager.go b/internal/flushcommon/pipeline/mock_fgmanager.go index cf8cd6b2aa1ca..f63c5590193c9 100644 --- a/internal/flushcommon/pipeline/mock_fgmanager.go +++ b/internal/flushcommon/pipeline/mock_fgmanager.go @@ -114,17 +114,17 @@ func (_c *MockFlowgraphManager_Close_Call) RunAndReturn(run func()) *MockFlowgra return _c } -// GetChannelsJSON provides a mock function with given fields: -func (_m *MockFlowgraphManager) GetChannelsJSON() string { - ret := _m.Called() +// GetChannelsJSON provides a mock function with given fields: collectionID +func (_m *MockFlowgraphManager) GetChannelsJSON(collectionID int64) string { + ret := _m.Called(collectionID) if len(ret) == 0 { panic("no return value specified for GetChannelsJSON") } var r0 string - if rf, ok := ret.Get(0).(func() string); ok { - r0 = rf() + if rf, ok := ret.Get(0).(func(int64) string); ok { + r0 = rf(collectionID) } else { r0 = ret.Get(0).(string) } @@ -138,13 +138,14 @@ type MockFlowgraphManager_GetChannelsJSON_Call struct { } // GetChannelsJSON is a helper method to define mock.On call -func (_e *MockFlowgraphManager_Expecter) GetChannelsJSON() *MockFlowgraphManager_GetChannelsJSON_Call { - return &MockFlowgraphManager_GetChannelsJSON_Call{Call: _e.mock.On("GetChannelsJSON")} +// - collectionID int64 +func (_e *MockFlowgraphManager_Expecter) GetChannelsJSON(collectionID interface{}) *MockFlowgraphManager_GetChannelsJSON_Call { + return &MockFlowgraphManager_GetChannelsJSON_Call{Call: _e.mock.On("GetChannelsJSON", collectionID)} } -func (_c *MockFlowgraphManager_GetChannelsJSON_Call) Run(run func()) *MockFlowgraphManager_GetChannelsJSON_Call { +func (_c *MockFlowgraphManager_GetChannelsJSON_Call) Run(run func(collectionID int64)) *MockFlowgraphManager_GetChannelsJSON_Call { _c.Call.Run(func(args mock.Arguments) { - run() + run(args[0].(int64)) }) return _c } @@ -154,7 +155,7 @@ func (_c *MockFlowgraphManager_GetChannelsJSON_Call) Return(_a0 string) *MockFlo return _c } -func (_c *MockFlowgraphManager_GetChannelsJSON_Call) RunAndReturn(run func() string) *MockFlowgraphManager_GetChannelsJSON_Call { +func (_c *MockFlowgraphManager_GetChannelsJSON_Call) RunAndReturn(run func(int64) string) *MockFlowgraphManager_GetChannelsJSON_Call { _c.Call.Return(run) return _c } @@ -309,17 +310,17 @@ func (_c *MockFlowgraphManager_GetFlowgraphService_Call) RunAndReturn(run func(s return _c } -// GetSegmentsJSON provides a mock function with given fields: -func (_m *MockFlowgraphManager) GetSegmentsJSON() string { - ret := _m.Called() +// GetSegmentsJSON provides a mock function with given fields: collectionID +func (_m *MockFlowgraphManager) GetSegmentsJSON(collectionID int64) string { + ret := _m.Called(collectionID) if len(ret) == 0 { panic("no return value specified for GetSegmentsJSON") } var r0 string - if rf, ok := ret.Get(0).(func() string); ok { - r0 = rf() + if rf, ok := ret.Get(0).(func(int64) string); ok { + r0 = rf(collectionID) } else { r0 = ret.Get(0).(string) } @@ -333,13 +334,14 @@ type MockFlowgraphManager_GetSegmentsJSON_Call struct { } // GetSegmentsJSON is a helper method to define mock.On call -func (_e *MockFlowgraphManager_Expecter) GetSegmentsJSON() *MockFlowgraphManager_GetSegmentsJSON_Call { - return &MockFlowgraphManager_GetSegmentsJSON_Call{Call: _e.mock.On("GetSegmentsJSON")} +// - collectionID int64 +func (_e *MockFlowgraphManager_Expecter) GetSegmentsJSON(collectionID interface{}) *MockFlowgraphManager_GetSegmentsJSON_Call { + return &MockFlowgraphManager_GetSegmentsJSON_Call{Call: _e.mock.On("GetSegmentsJSON", collectionID)} } -func (_c *MockFlowgraphManager_GetSegmentsJSON_Call) Run(run func()) *MockFlowgraphManager_GetSegmentsJSON_Call { +func (_c *MockFlowgraphManager_GetSegmentsJSON_Call) Run(run func(collectionID int64)) *MockFlowgraphManager_GetSegmentsJSON_Call { _c.Call.Run(func(args mock.Arguments) { - run() + run(args[0].(int64)) }) return _c } @@ -349,7 +351,7 @@ func (_c *MockFlowgraphManager_GetSegmentsJSON_Call) Return(_a0 string) *MockFlo return _c } -func (_c *MockFlowgraphManager_GetSegmentsJSON_Call) RunAndReturn(run func() string) *MockFlowgraphManager_GetSegmentsJSON_Call { +func (_c *MockFlowgraphManager_GetSegmentsJSON_Call) RunAndReturn(run func(int64) string) *MockFlowgraphManager_GetSegmentsJSON_Call { _c.Call.Return(run) return _c } diff --git a/internal/metastore/model/segment_index.go b/internal/metastore/model/segment_index.go index 9f6dfd8dce537..8c2e8e3ef3827 100644 --- a/internal/metastore/model/segment_index.go +++ b/internal/metastore/model/segment_index.go @@ -50,6 +50,7 @@ func UnmarshalSegmentIndexModel(segIndex *indexpb.SegmentIndex) *SegmentIndex { IndexSize: segIndex.SerializeSize, WriteHandoff: segIndex.WriteHandoff, CurrentIndexVersion: segIndex.GetCurrentIndexVersion(), + FinishedUTCTime: segIndex.FinishedTime, } } @@ -75,6 +76,7 @@ func MarshalSegmentIndexModel(segIdx *SegmentIndex) *indexpb.SegmentIndex { SerializeSize: segIdx.IndexSize, WriteHandoff: segIdx.WriteHandoff, CurrentIndexVersion: segIdx.CurrentIndexVersion, + FinishedTime: segIdx.FinishedUTCTime, } } @@ -96,5 +98,6 @@ func CloneSegmentIndex(segIndex *SegmentIndex) *SegmentIndex { IndexSize: segIndex.IndexSize, WriteHandoff: segIndex.WriteHandoff, CurrentIndexVersion: segIndex.CurrentIndexVersion, + FinishedUTCTime: segIndex.FinishedUTCTime, } } diff --git a/internal/proto/index_coord.proto b/internal/proto/index_coord.proto index adcd0aed7bd37..b736098d46b82 100644 --- a/internal/proto/index_coord.proto +++ b/internal/proto/index_coord.proto @@ -80,6 +80,7 @@ message SegmentIndex { bool write_handoff = 15; int32 current_index_version = 16; int64 index_store_version = 17; + uint64 finished_time = 18; } message RegisterNodeRequest { diff --git a/internal/proto/query_coord.proto b/internal/proto/query_coord.proto index d469841c525ff..9b44e6f5c9560 100644 --- a/internal/proto/query_coord.proto +++ b/internal/proto/query_coord.proto @@ -668,6 +668,7 @@ message CollectionLoadInfo { LoadType load_type = 6; int32 recover_times = 7; repeated int64 load_fields = 8; + int64 dbID= 9; } message PartitionLoadInfo { diff --git a/internal/proxy/http_req_impl.go b/internal/proxy/http_req_impl.go index 5697e06d47122..0dfcce4d78e93 100644 --- a/internal/proxy/http_req_impl.go +++ b/internal/proxy/http_req_impl.go @@ -148,10 +148,14 @@ func getSlowQuery(node *Proxy) gin.HandlerFunc { // buildReqParams fetch all parameters from query parameter of URL, add them into a map data structure. // put key and value from query parameter into map, concatenate values with separator if values size is greater than 1 -func buildReqParams(c *gin.Context, metricsType string) map[string]interface{} { +func buildReqParams(c *gin.Context, metricsType string, customParams ...*commonpb.KeyValuePair) map[string]interface{} { ret := make(map[string]interface{}) ret[metricsinfo.MetricTypeKey] = metricsType + for _, kv := range customParams { + ret[kv.Key] = kv.Value + } + queryParams := c.Request.URL.Query() for key, values := range queryParams { if len(values) > 1 { @@ -163,7 +167,7 @@ func buildReqParams(c *gin.Context, metricsType string) map[string]interface{} { return ret } -func getQueryComponentMetrics(node *Proxy, metricsType string) gin.HandlerFunc { +func getQueryComponentMetrics(node *Proxy, metricsType string, customParams ...*commonpb.KeyValuePair) gin.HandlerFunc { return func(c *gin.Context) { params := buildReqParams(c, metricsType) req, err := metricsinfo.ConstructGetMetricsRequest(params) @@ -185,7 +189,7 @@ func getQueryComponentMetrics(node *Proxy, metricsType string) gin.HandlerFunc { } } -func getDataComponentMetrics(node *Proxy, metricsType string) gin.HandlerFunc { +func getDataComponentMetrics(node *Proxy, metricsType string, customParams ...*commonpb.KeyValuePair) gin.HandlerFunc { return func(c *gin.Context) { params := buildReqParams(c, metricsType) req, err := metricsinfo.ConstructGetMetricsRequest(params) diff --git a/internal/proxy/impl.go b/internal/proxy/impl.go index c5fc8fc9ca555..edb5eb65b1b0f 100644 --- a/internal/proxy/impl.go +++ b/internal/proxy/impl.go @@ -6753,10 +6753,10 @@ func (node *Proxy) RegisterRestRouter(router gin.IRouter) { router.GET(http.QCReplicaPath, getQueryComponentMetrics(node, metricsinfo.ReplicaKey)) router.GET(http.QCResourceGroupPath, getQueryComponentMetrics(node, metricsinfo.ResourceGroupKey)) router.GET(http.QCAllTasksPath, getQueryComponentMetrics(node, metricsinfo.AllTaskKey)) - router.GET(http.QCSegmentsPath, getQueryComponentMetrics(node, metricsinfo.SegmentKey)) + router.GET(http.QCSegmentsPath, getQueryComponentMetrics(node, metricsinfo.SegmentKey, metricsinfo.RequestParamsInQC)) // QueryNode requests that are forwarded from querycoord - router.GET(http.QNSegmentsPath, getQueryComponentMetrics(node, metricsinfo.SegmentKey)) + router.GET(http.QNSegmentsPath, getQueryComponentMetrics(node, metricsinfo.SegmentKey, metricsinfo.RequestParamsInQN)) router.GET(http.QNChannelsPath, getQueryComponentMetrics(node, metricsinfo.ChannelKey)) // DataCoord requests that are forwarded from proxy @@ -6765,11 +6765,11 @@ func (node *Proxy) RegisterRestRouter(router gin.IRouter) { router.GET(http.DCImportTasksPath, getDataComponentMetrics(node, metricsinfo.ImportTaskKey)) router.GET(http.DCBuildIndexTasksPath, getDataComponentMetrics(node, metricsinfo.BuildIndexTaskKey)) router.GET(http.IndexListPath, getDataComponentMetrics(node, metricsinfo.IndexKey)) - router.GET(http.DCSegmentsPath, getDataComponentMetrics(node, metricsinfo.SegmentKey)) + router.GET(http.DCSegmentsPath, getDataComponentMetrics(node, metricsinfo.SegmentKey, metricsinfo.RequestParamsInDC)) // Datanode requests that are forwarded from datacoord router.GET(http.DNSyncTasksPath, getDataComponentMetrics(node, metricsinfo.SyncTaskKey)) - router.GET(http.DNSegmentsPath, getDataComponentMetrics(node, metricsinfo.SegmentKey)) + router.GET(http.DNSegmentsPath, getDataComponentMetrics(node, metricsinfo.SegmentKey, metricsinfo.RequestParamsInDN)) router.GET(http.DNChannelsPath, getDataComponentMetrics(node, metricsinfo.ChannelKey)) // Database requests diff --git a/internal/querycoordv2/handlers.go b/internal/querycoordv2/handlers.go index bdfeeb027103b..a9c0a51baab00 100644 --- a/internal/querycoordv2/handlers.go +++ b/internal/querycoordv2/handlers.go @@ -301,17 +301,13 @@ func (s *Server) getSegmentsJSON(ctx context.Context, req *milvuspb.GetMetricsRe } in := v.String() - if in == "qn" { + if in == metricsinfo.MetricsRequestParamsInQN { // TODO: support filter by collection id return s.getSegmentsFromQueryNode(ctx, req) } - if in == "qc" { - v = jsonReq.Get(metricsinfo.MetricRequestParamCollectionIDKey) - collectionID := int64(0) - if v.Exists() { - collectionID = v.Int() - } + if in == metricsinfo.MetricsRequestParamsInQC { + collectionID := metricsinfo.GetCollectionIDFromRequest(jsonReq) filteredSegments := s.dist.SegmentDistManager.GetSegmentDist(collectionID) bs, err := json.Marshal(filteredSegments) if err != nil { diff --git a/internal/querycoordv2/job/job_load.go b/internal/querycoordv2/job/job_load.go index 5009be26fb4ad..998335f9e9a41 100644 --- a/internal/querycoordv2/job/job_load.go +++ b/internal/querycoordv2/job/job_load.go @@ -171,14 +171,15 @@ func (job *LoadCollectionJob) Execute() error { } } + collectionInfo, err := job.broker.DescribeCollection(job.ctx, req.GetCollectionID()) + if err != nil { + log.Warn("failed to describe collection from RootCoord", zap.Error(err)) + return err + } + // 2. create replica if not exist replicas := job.meta.ReplicaManager.GetByCollection(job.ctx, req.GetCollectionID()) if len(replicas) == 0 { - collectionInfo, err := job.broker.DescribeCollection(job.ctx, req.GetCollectionID()) - if err != nil { - return err - } - // API of LoadCollection is wired, we should use map[resourceGroupNames]replicaNumber as input, to keep consistency with `TransferReplica` API. // Then we can implement dynamic replica changed in different resource group independently. _, err = utils.SpawnReplicasWithRG(job.ctx, job.meta, req.GetCollectionID(), req.GetResourceGroups(), req.GetReplicaNumber(), collectionInfo.GetVirtualChannelNames()) @@ -213,6 +214,7 @@ func (job *LoadCollectionJob) Execute() error { FieldIndexID: req.GetFieldIndexID(), LoadType: querypb.LoadType_LoadCollection, LoadFields: req.GetLoadFields(), + DbID: collectionInfo.GetDbId(), }, CreatedAt: time.Now(), LoadSpan: sp, @@ -371,13 +373,15 @@ func (job *LoadPartitionJob) Execute() error { } } + collectionInfo, err := job.broker.DescribeCollection(job.ctx, req.GetCollectionID()) + if err != nil { + log.Warn("failed to describe collection from RootCoord", zap.Error(err)) + return err + } + // 2. create replica if not exist replicas := job.meta.ReplicaManager.GetByCollection(context.TODO(), req.GetCollectionID()) if len(replicas) == 0 { - collectionInfo, err := job.broker.DescribeCollection(job.ctx, req.GetCollectionID()) - if err != nil { - return err - } _, err = utils.SpawnReplicasWithRG(job.ctx, job.meta, req.GetCollectionID(), req.GetResourceGroups(), req.GetReplicaNumber(), collectionInfo.GetVirtualChannelNames()) if err != nil { msg := "failed to spawn replica for collection" @@ -412,6 +416,7 @@ func (job *LoadPartitionJob) Execute() error { FieldIndexID: req.GetFieldIndexID(), LoadType: querypb.LoadType_LoadPartition, LoadFields: req.GetLoadFields(), + DbID: collectionInfo.GetDbId(), }, CreatedAt: time.Now(), LoadSpan: sp, diff --git a/internal/querycoordv2/meta/channel_dist_manager.go b/internal/querycoordv2/meta/channel_dist_manager.go index db3dc8720100a..24e06327c3e54 100644 --- a/internal/querycoordv2/meta/channel_dist_manager.go +++ b/internal/querycoordv2/meta/channel_dist_manager.go @@ -300,15 +300,24 @@ func (m *ChannelDistManager) updateCollectionIndex() { } } -func (m *ChannelDistManager) GetChannelDist() []*metricsinfo.DmChannel { +func (m *ChannelDistManager) GetChannelDist(collectionID int64) []*metricsinfo.DmChannel { m.rwmutex.RLock() defer m.rwmutex.RUnlock() - var channels []*metricsinfo.DmChannel - for _, nodeChannels := range m.channels { - for _, channel := range nodeChannels.channels { - channels = append(channels, newDmChannelMetricsFrom(channel)) + var ret []*metricsinfo.DmChannel + if collectionID > 0 { + if channels, ok := m.collectionIndex[collectionID]; ok { + for _, channel := range channels { + ret = append(ret, newDmChannelMetricsFrom(channel)) + } } + return ret } - return channels + + for _, channels := range m.collectionIndex { + for _, channel := range channels { + ret = append(ret, newDmChannelMetricsFrom(channel)) + } + } + return ret } diff --git a/internal/querycoordv2/meta/channel_dist_manager_test.go b/internal/querycoordv2/meta/channel_dist_manager_test.go index aa5db6eb111d5..734d665d6f3df 100644 --- a/internal/querycoordv2/meta/channel_dist_manager_test.go +++ b/internal/querycoordv2/meta/channel_dist_manager_test.go @@ -207,7 +207,7 @@ func TestGetChannelDistJSON(t *testing.T) { manager.Update(1, channel1) manager.Update(2, channel2) - channels := manager.GetChannelDist() + channels := manager.GetChannelDist(0) assert.Equal(t, 2, len(channels)) checkResult := func(channel *metricsinfo.DmChannel) { diff --git a/internal/querycoordv2/meta/collection_manager.go b/internal/querycoordv2/meta/collection_manager.go index f95d18b0658f3..980331ff64c80 100644 --- a/internal/querycoordv2/meta/collection_manager.go +++ b/internal/querycoordv2/meta/collection_manager.go @@ -285,7 +285,13 @@ func (m *CollectionManager) upgradeRecover(ctx context.Context, broker Broker) e // we should save it's CollectionLoadInfo to meta store for _, partition := range m.GetAllPartitions(ctx) { // In old version, collection would NOT be stored if the partition existed. - if _, ok := m.collections[partition.GetCollectionID()]; !ok { + if !m.Exist(ctx, partition.GetCollectionID()) { + collectionInfo, err := broker.DescribeCollection(ctx, partition.GetCollectionID()) + if err != nil { + log.Warn("failed to describe collection from RootCoord", zap.Error(err)) + return err + } + col := &Collection{ CollectionLoadInfo: &querypb.CollectionLoadInfo{ CollectionID: partition.GetCollectionID(), @@ -293,10 +299,11 @@ func (m *CollectionManager) upgradeRecover(ctx context.Context, broker Broker) e Status: partition.GetStatus(), FieldIndexID: partition.GetFieldIndexID(), LoadType: querypb.LoadType_LoadPartition, + DbID: collectionInfo.GetDbId(), }, LoadPercentage: 100, } - err := m.PutCollection(ctx, col) + err = m.PutCollection(ctx, col) if err != nil { return err } diff --git a/internal/querycoordv2/meta/dist_manager.go b/internal/querycoordv2/meta/dist_manager.go index 05275f213587b..83f8f4ae7ce97 100644 --- a/internal/querycoordv2/meta/dist_manager.go +++ b/internal/querycoordv2/meta/dist_manager.go @@ -42,10 +42,10 @@ func NewDistributionManager() *DistributionManager { // It includes segments, DM channels, and leader views. // If there are no segments, channels, or leader views, it returns an empty string. // In case of an error during JSON marshaling, it returns the error. -func (dm *DistributionManager) GetDistributionJSON() string { - segments := dm.GetSegmentDist(0) - channels := dm.GetChannelDist() - leaderView := dm.GetLeaderView() +func (dm *DistributionManager) GetDistributionJSON(collectionID int64) string { + segments := dm.GetSegmentDist(collectionID) + channels := dm.GetChannelDist(collectionID) + leaderView := dm.GetLeaderView(collectionID) dist := &metricsinfo.QueryCoordDist{ Segments: segments, diff --git a/internal/querycoordv2/meta/dist_manager_test.go b/internal/querycoordv2/meta/dist_manager_test.go index 4af530ea6f4de..3e8f33f4eff26 100644 --- a/internal/querycoordv2/meta/dist_manager_test.go +++ b/internal/querycoordv2/meta/dist_manager_test.go @@ -81,7 +81,7 @@ func TestGetDistributionJSON(t *testing.T) { manager.LeaderViewManager.Update(2, leaderView2) // Call GetDistributionJSON - jsonOutput := manager.GetDistributionJSON() + jsonOutput := manager.GetDistributionJSON(0) // Verify JSON output var dist metricsinfo.QueryCoordDist @@ -91,4 +91,13 @@ func TestGetDistributionJSON(t *testing.T) { assert.Len(t, dist.Segments, 2) assert.Len(t, dist.DMChannels, 2) assert.Len(t, dist.LeaderViews, 2) + + jsonOutput = manager.GetDistributionJSON(1000) + var dist2 metricsinfo.QueryCoordDist + err = json.Unmarshal([]byte(jsonOutput), &dist2) + assert.NoError(t, err) + + assert.Len(t, dist2.Segments, 0) + assert.Len(t, dist2.DMChannels, 0) + assert.Len(t, dist2.LeaderViews, 0) } diff --git a/internal/querycoordv2/meta/leader_view_manager.go b/internal/querycoordv2/meta/leader_view_manager.go index c0cfca82b4ac1..c29171e6f90ae 100644 --- a/internal/querycoordv2/meta/leader_view_manager.go +++ b/internal/querycoordv2/meta/leader_view_manager.go @@ -320,13 +320,26 @@ func (mgr *LeaderViewManager) GetLatestShardLeaderByFilter(filters ...LeaderView // GetLeaderView returns a slice of LeaderView objects, each representing the state of a leader node. // It traverses the views map, converts each LeaderView to a metricsinfo.LeaderView, and collects them into a slice. // The method locks the views map for reading to ensure thread safety. -func (mgr *LeaderViewManager) GetLeaderView() []*metricsinfo.LeaderView { +func (mgr *LeaderViewManager) GetLeaderView(collectionID int64) []*metricsinfo.LeaderView { mgr.rwmutex.RLock() defer mgr.rwmutex.RUnlock() var leaderViews []*metricsinfo.LeaderView for _, nodeViews := range mgr.views { - for _, lv := range nodeViews.views { + var filteredViews []*LeaderView + if collectionID > 0 { + if lv, ok := nodeViews.collectionViews[collectionID]; ok { + filteredViews = lv + } else { + // if collectionID is not found, return empty leader views + return leaderViews + } + } else { + // if collectionID is not set, return all leader views + filteredViews = nodeViews.views + } + + for _, lv := range filteredViews { errString := "" if lv.UnServiceableError != nil { errString = lv.UnServiceableError.Error() diff --git a/internal/querycoordv2/meta/leader_view_manager_test.go b/internal/querycoordv2/meta/leader_view_manager_test.go index 61e441498b924..c4cc7cec2ec31 100644 --- a/internal/querycoordv2/meta/leader_view_manager_test.go +++ b/internal/querycoordv2/meta/leader_view_manager_test.go @@ -23,13 +23,11 @@ import ( "github.com/samber/lo" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/suite" - "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/typeutil" ) @@ -359,7 +357,7 @@ func TestGetLeaderView(t *testing.T) { manager.Update(2, leaderView2) // Call GetLeaderView - leaderViews := manager.GetLeaderView() + leaderViews := manager.GetLeaderView(0) jsonOutput, err := json.Marshal(leaderViews) assert.NoError(t, err) @@ -368,7 +366,6 @@ func TestGetLeaderView(t *testing.T) { assert.NoError(t, err) assert.Len(t, result, 2) - log.Info("====", zap.Any("result", result)) checkResult := func(lv *metricsinfo.LeaderView) { if lv.LeaderID == 1 { assert.Equal(t, int64(100), lv.CollectionID) @@ -394,4 +391,10 @@ func TestGetLeaderView(t *testing.T) { for _, lv := range result { checkResult(lv) } + + leaderViews = manager.GetLeaderView(1) + assert.Len(t, leaderViews, 0) + + leaderViews = manager.GetLeaderView(100) + assert.Len(t, leaderViews, 1) } diff --git a/internal/querycoordv2/meta/mock_target_manager.go b/internal/querycoordv2/meta/mock_target_manager.go index 6c69dff9f24fb..560c055bf30b3 100644 --- a/internal/querycoordv2/meta/mock_target_manager.go +++ b/internal/querycoordv2/meta/mock_target_manager.go @@ -638,17 +638,17 @@ func (_c *MockTargetManager_GetSealedSegmentsByPartition_Call) RunAndReturn(run return _c } -// GetTargetJSON provides a mock function with given fields: ctx, scope -func (_m *MockTargetManager) GetTargetJSON(ctx context.Context, scope int32) string { - ret := _m.Called(ctx, scope) +// GetTargetJSON provides a mock function with given fields: ctx, scope, collectionID +func (_m *MockTargetManager) GetTargetJSON(ctx context.Context, scope int32, collectionID int64) string { + ret := _m.Called(ctx, scope, collectionID) if len(ret) == 0 { panic("no return value specified for GetTargetJSON") } var r0 string - if rf, ok := ret.Get(0).(func(context.Context, int32) string); ok { - r0 = rf(ctx, scope) + if rf, ok := ret.Get(0).(func(context.Context, int32, int64) string); ok { + r0 = rf(ctx, scope, collectionID) } else { r0 = ret.Get(0).(string) } @@ -664,13 +664,14 @@ type MockTargetManager_GetTargetJSON_Call struct { // GetTargetJSON is a helper method to define mock.On call // - ctx context.Context // - scope int32 -func (_e *MockTargetManager_Expecter) GetTargetJSON(ctx interface{}, scope interface{}) *MockTargetManager_GetTargetJSON_Call { - return &MockTargetManager_GetTargetJSON_Call{Call: _e.mock.On("GetTargetJSON", ctx, scope)} +// - collectionID int64 +func (_e *MockTargetManager_Expecter) GetTargetJSON(ctx interface{}, scope interface{}, collectionID interface{}) *MockTargetManager_GetTargetJSON_Call { + return &MockTargetManager_GetTargetJSON_Call{Call: _e.mock.On("GetTargetJSON", ctx, scope, collectionID)} } -func (_c *MockTargetManager_GetTargetJSON_Call) Run(run func(ctx context.Context, scope int32)) *MockTargetManager_GetTargetJSON_Call { +func (_c *MockTargetManager_GetTargetJSON_Call) Run(run func(ctx context.Context, scope int32, collectionID int64)) *MockTargetManager_GetTargetJSON_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(context.Context), args[1].(int32)) + run(args[0].(context.Context), args[1].(int32), args[2].(int64)) }) return _c } @@ -680,7 +681,7 @@ func (_c *MockTargetManager_GetTargetJSON_Call) Return(_a0 string) *MockTargetMa return _c } -func (_c *MockTargetManager_GetTargetJSON_Call) RunAndReturn(run func(context.Context, int32) string) *MockTargetManager_GetTargetJSON_Call { +func (_c *MockTargetManager_GetTargetJSON_Call) RunAndReturn(run func(context.Context, int32, int64) string) *MockTargetManager_GetTargetJSON_Call { _c.Call.Return(run) return _c } diff --git a/internal/querycoordv2/meta/replica_manager.go b/internal/querycoordv2/meta/replica_manager.go index 2c04c52c7cddc..9145504996205 100644 --- a/internal/querycoordv2/meta/replica_manager.go +++ b/internal/querycoordv2/meta/replica_manager.go @@ -30,6 +30,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" @@ -506,7 +507,7 @@ func (m *ReplicaManager) GetResourceGroupByCollection(ctx context.Context, colle // It locks the ReplicaManager for reading, converts the replicas to their protobuf representation, // marshals them into a JSON string, and returns the result. // If an error occurs during marshaling, it logs a warning and returns an empty string. -func (m *ReplicaManager) GetReplicasJSON(ctx context.Context) string { +func (m *ReplicaManager) GetReplicasJSON(ctx context.Context, meta *Meta) string { m.rwmutex.RLock() defer m.rwmutex.RUnlock() @@ -515,9 +516,19 @@ func (m *ReplicaManager) GetReplicasJSON(ctx context.Context) string { for k, v := range r.replicaPB.GetChannelNodeInfos() { channelTowRWNodes[k] = v.GetRwNodes() } + + collectionInfo := meta.GetCollection(ctx, r.GetCollectionID()) + dbID := util.InvalidDBID + if collectionInfo == nil { + log.Ctx(ctx).Warn("failed to get collection info", zap.Int64("collectionID", r.GetCollectionID())) + } else { + dbID = collectionInfo.GetDbID() + } + return &metricsinfo.Replica{ ID: r.GetID(), CollectionID: r.GetCollectionID(), + DatabaseID: dbID, RWNodes: r.GetNodes(), ResourceGroup: r.GetResourceGroup(), RONodes: r.GetRONodes(), diff --git a/internal/querycoordv2/meta/replica_manager_test.go b/internal/querycoordv2/meta/replica_manager_test.go index cdfc1cbdfc8b5..7921129590190 100644 --- a/internal/querycoordv2/meta/replica_manager_test.go +++ b/internal/querycoordv2/meta/replica_manager_test.go @@ -545,7 +545,26 @@ func TestGetReplicasJSON(t *testing.T) { err = replicaManager.put(ctx, replica2) assert.NoError(t, err) - jsonOutput := replicaManager.GetReplicasJSON(ctx) + meta := &Meta{ + CollectionManager: NewCollectionManager(catalog), + } + + err = meta.PutCollectionWithoutSave(ctx, &Collection{ + CollectionLoadInfo: &querypb.CollectionLoadInfo{ + CollectionID: 100, + DbID: int64(1), + }, + }) + assert.NoError(t, err) + + err = meta.PutCollectionWithoutSave(ctx, &Collection{ + CollectionLoadInfo: &querypb.CollectionLoadInfo{ + CollectionID: 200, + }, + }) + assert.NoError(t, err) + + jsonOutput := replicaManager.GetReplicasJSON(ctx, meta) var replicas []*metricsinfo.Replica err = json.Unmarshal([]byte(jsonOutput), &replicas) assert.NoError(t, err) @@ -556,10 +575,12 @@ func TestGetReplicasJSON(t *testing.T) { assert.Equal(t, int64(100), replica.CollectionID) assert.Equal(t, "rg1", replica.ResourceGroup) assert.ElementsMatch(t, []int64{1, 2, 3}, replica.RWNodes) + assert.Equal(t, int64(1), replica.DatabaseID) } else if replica.ID == 2 { assert.Equal(t, int64(200), replica.CollectionID) assert.Equal(t, "rg2", replica.ResourceGroup) assert.ElementsMatch(t, []int64{4, 5, 6}, replica.RWNodes) + assert.Equal(t, int64(0), replica.DatabaseID) } else { assert.Failf(t, "unexpected replica id", "unexpected replica id %d", replica.ID) } diff --git a/internal/querycoordv2/meta/segment_dist_manager.go b/internal/querycoordv2/meta/segment_dist_manager.go index 85519b7770360..38cefdcf7ad8f 100644 --- a/internal/querycoordv2/meta/segment_dist_manager.go +++ b/internal/querycoordv2/meta/segment_dist_manager.go @@ -254,9 +254,10 @@ func (m *SegmentDistManager) GetSegmentDist(collectionID int64) []*metricsinfo.S var segments []*metricsinfo.Segment for _, nodeSeg := range m.segments { for _, segment := range nodeSeg.segments { - if collectionID == 0 || segment.GetCollectionID() == collectionID { - segments = append(segments, newSegmentMetricsFrom(segment)) + if collectionID > 0 && segment.GetCollectionID() != collectionID { + continue } + segments = append(segments, newSegmentMetricsFrom(segment)) } } diff --git a/internal/querycoordv2/meta/target.go b/internal/querycoordv2/meta/target.go index 4795eade4cfae..60491a389ae05 100644 --- a/internal/querycoordv2/meta/target.go +++ b/internal/querycoordv2/meta/target.go @@ -207,8 +207,13 @@ func (t *target) getCollectionTarget(collectionID int64) *CollectionTarget { return t.collectionTargetMap[collectionID] } -func (t *target) toQueryCoordCollectionTargets() []*metricsinfo.QueryCoordTarget { - return lo.MapToSlice(t.collectionTargetMap, func(k int64, v *CollectionTarget) *metricsinfo.QueryCoordTarget { +func (t *target) toQueryCoordCollectionTargets(collectionID int64) []*metricsinfo.QueryCoordTarget { + var ret []*metricsinfo.QueryCoordTarget + for k, v := range t.collectionTargetMap { + if collectionID > 0 && collectionID != k { + continue + } + segments := lo.MapToSlice(v.GetAllSegments(), func(k int64, s *datapb.SegmentInfo) *metricsinfo.Segment { return metrics.NewSegmentFrom(s) }) @@ -217,10 +222,12 @@ func (t *target) toQueryCoordCollectionTargets() []*metricsinfo.QueryCoordTarget return metrics.NewDMChannelFrom(ch.VchannelInfo) }) - return &metricsinfo.QueryCoordTarget{ + ret = append(ret, &metricsinfo.QueryCoordTarget{ CollectionID: k, Segments: segments, DMChannels: dmChannels, - } - }) + }) + } + + return ret } diff --git a/internal/querycoordv2/meta/target_manager.go b/internal/querycoordv2/meta/target_manager.go index 10fe0b787b55d..924f8eb6ecc18 100644 --- a/internal/querycoordv2/meta/target_manager.go +++ b/internal/querycoordv2/meta/target_manager.go @@ -70,7 +70,7 @@ type TargetManagerInterface interface { SaveCurrentTarget(ctx context.Context, catalog metastore.QueryCoordCatalog) Recover(ctx context.Context, catalog metastore.QueryCoordCatalog) error CanSegmentBeMoved(ctx context.Context, collectionID, segmentID int64) bool - GetTargetJSON(ctx context.Context, scope TargetScope) string + GetTargetJSON(ctx context.Context, scope TargetScope, collectionID int64) string GetPartitions(ctx context.Context, collectionID int64, scope TargetScope) ([]int64, error) IsCurrentTargetReady(ctx context.Context, collectionID int64) bool } @@ -638,7 +638,7 @@ func (mgr *TargetManager) CanSegmentBeMoved(ctx context.Context, collectionID, s return false } -func (mgr *TargetManager) GetTargetJSON(ctx context.Context, scope TargetScope) string { +func (mgr *TargetManager) GetTargetJSON(ctx context.Context, scope TargetScope, collectionID int64) string { mgr.rwMutex.RLock() defer mgr.rwMutex.RUnlock() @@ -647,7 +647,7 @@ func (mgr *TargetManager) GetTargetJSON(ctx context.Context, scope TargetScope) return "" } - v, err := json.Marshal(ret.toQueryCoordCollectionTargets()) + v, err := json.Marshal(ret.toQueryCoordCollectionTargets(collectionID)) if err != nil { log.Warn("failed to marshal target", zap.Error(err)) return "" diff --git a/internal/querycoordv2/meta/target_manager_test.go b/internal/querycoordv2/meta/target_manager_test.go index 34bf64136a2e2..91958bfcbd60c 100644 --- a/internal/querycoordv2/meta/target_manager_test.go +++ b/internal/querycoordv2/meta/target_manager_test.go @@ -669,7 +669,7 @@ func (suite *TargetManagerSuite) TestGetTargetJSON() { suite.NoError(suite.mgr.UpdateCollectionNextTarget(ctx, collectionID)) suite.True(suite.mgr.UpdateCollectionCurrentTarget(ctx, collectionID)) - jsonStr := suite.mgr.GetTargetJSON(ctx, CurrentTarget) + jsonStr := suite.mgr.GetTargetJSON(ctx, CurrentTarget, 0) assert.NotEmpty(suite.T(), jsonStr) var currentTarget []*metricsinfo.QueryCoordTarget @@ -679,6 +679,14 @@ func (suite *TargetManagerSuite) TestGetTargetJSON() { assert.Equal(suite.T(), collectionID, currentTarget[0].CollectionID) assert.Len(suite.T(), currentTarget[0].DMChannels, 2) assert.Len(suite.T(), currentTarget[0].Segments, 2) + + jsonStr = suite.mgr.GetTargetJSON(ctx, CurrentTarget, 1) + assert.NotEmpty(suite.T(), jsonStr) + + var currentTarget2 []*metricsinfo.QueryCoordTarget + err = json.Unmarshal([]byte(jsonStr), ¤tTarget) + suite.NoError(err) + assert.Len(suite.T(), currentTarget2, 0) } func BenchmarkTargetManager(b *testing.B) { diff --git a/internal/querycoordv2/server.go b/internal/querycoordv2/server.go index 8ae802cf7848f..8464d3055fcff 100644 --- a/internal/querycoordv2/server.go +++ b/internal/querycoordv2/server.go @@ -202,7 +202,8 @@ func (s *Server) registerMetricsRequest() { } QueryDistAction := func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) { - return s.dist.GetDistributionJSON(), nil + collectionID := metricsinfo.GetCollectionIDFromRequest(jsonReq) + return s.dist.GetDistributionJSON(collectionID), nil } QueryTargetAction := func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) { @@ -211,11 +212,13 @@ func (s *Server) registerMetricsRequest() { if v.Exists() { scope = meta.TargetScope(v.Int()) } - return s.targetMgr.GetTargetJSON(ctx, scope), nil + + collectionID := metricsinfo.GetCollectionIDFromRequest(jsonReq) + return s.targetMgr.GetTargetJSON(ctx, scope, collectionID), nil } QueryReplicasAction := func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) { - return s.meta.GetReplicasJSON(ctx), nil + return s.meta.GetReplicasJSON(ctx, s.meta), nil } QueryResourceGroupsAction := func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) { diff --git a/internal/querycoordv2/utils/meta.go b/internal/querycoordv2/utils/meta.go index 1744e2367849f..fe15c685bbf0c 100644 --- a/internal/querycoordv2/utils/meta.go +++ b/internal/querycoordv2/utils/meta.go @@ -153,7 +153,6 @@ func SpawnReplicasWithRG(ctx context.Context, m *meta.Meta, collection int64, re if err != nil { return nil, err } - // Spawn it in replica manager. replicas, err := m.ReplicaManager.Spawn(ctx, collection, replicaNumInRG, channels) if err != nil { diff --git a/internal/querynodev2/metrics_info.go b/internal/querynodev2/metrics_info.go index 3cedfedb1df0b..722b8b844c45d 100644 --- a/internal/querynodev2/metrics_info.go +++ b/internal/querynodev2/metrics_info.go @@ -185,8 +185,8 @@ func getCollectionMetrics(node *QueryNode) (*metricsinfo.QueryNodeCollectionMetr } // getChannelJSON returns the JSON string of channels -func getChannelJSON(node *QueryNode) string { - stats := node.pipelineManager.GetChannelStats() +func getChannelJSON(node *QueryNode, collectionID int64) string { + stats := node.pipelineManager.GetChannelStats(collectionID) ret, err := json.Marshal(stats) if err != nil { log.Warn("failed to marshal channels", zap.Error(err)) @@ -196,10 +196,14 @@ func getChannelJSON(node *QueryNode) string { } // getSegmentJSON returns the JSON string of segments -func getSegmentJSON(node *QueryNode) string { +func getSegmentJSON(node *QueryNode, collectionID int64) string { allSegments := node.manager.Segment.GetBy() var ms []*metricsinfo.Segment for _, s := range allSegments { + if collectionID > 0 && s.Collection() != collectionID { + continue + } + indexes := make([]*metricsinfo.IndexedField, 0, len(s.Indexes())) for _, index := range s.Indexes() { indexes = append(indexes, &metricsinfo.IndexedField{ @@ -208,6 +212,7 @@ func getSegmentJSON(node *QueryNode) string { IndexSize: index.IndexInfo.IndexSize, BuildID: index.IndexInfo.BuildID, IsLoaded: index.IsLoaded, + HasRawData: s.HasRawData(index.IndexInfo.FieldID), }) } diff --git a/internal/querynodev2/metrics_info_test.go b/internal/querynodev2/metrics_info_test.go index 03c99519a0625..6966f2b050473 100644 --- a/internal/querynodev2/metrics_info_test.go +++ b/internal/querynodev2/metrics_info_test.go @@ -59,7 +59,7 @@ func TestGetPipelineJSON(t *testing.T) { assert.NoError(t, err) assert.Equal(t, 1, pipelineManager.Num()) - stats := pipelineManager.GetChannelStats() + stats := pipelineManager.GetChannelStats(0) expectedStats := []*metricsinfo.Channel{ { Name: ch, @@ -71,7 +71,7 @@ func TestGetPipelineJSON(t *testing.T) { } assert.Equal(t, expectedStats, stats) - JSONStr := getChannelJSON(&QueryNode{pipelineManager: pipelineManager}) + JSONStr := getChannelJSON(&QueryNode{pipelineManager: pipelineManager}, 0) assert.NotEmpty(t, JSONStr) var actualStats []*metricsinfo.Channel @@ -86,6 +86,7 @@ func TestGetSegmentJSON(t *testing.T) { segment.EXPECT().Collection().Return(int64(1001)) segment.EXPECT().Partition().Return(int64(2001)) segment.EXPECT().MemSize().Return(int64(1024)) + segment.EXPECT().HasRawData(mock.Anything).Return(true) segment.EXPECT().Indexes().Return([]*segments.IndexedFieldInfo{ { IndexInfo: &querypb.FieldIndexInfo{ @@ -106,7 +107,7 @@ func TestGetSegmentJSON(t *testing.T) { mockedSegmentManager.EXPECT().GetBy().Return([]segments.Segment{segment}) node.manager = &segments.Manager{Segment: mockedSegmentManager} - jsonStr := getSegmentJSON(node) + jsonStr := getSegmentJSON(node, 0) assert.NotEmpty(t, jsonStr) var segments []*metricsinfo.Segment diff --git a/internal/querynodev2/pipeline/manager.go b/internal/querynodev2/pipeline/manager.go index 6c6e26cfc7348..e7ebaddbb1eec 100644 --- a/internal/querynodev2/pipeline/manager.go +++ b/internal/querynodev2/pipeline/manager.go @@ -42,7 +42,7 @@ type Manager interface { Remove(channels ...string) Start(channels ...string) error Close() - GetChannelStats() []*metricsinfo.Channel + GetChannelStats(collectionID int64) []*metricsinfo.Channel } type manager struct { @@ -157,12 +157,15 @@ func (m *manager) Close() { } } -func (m *manager) GetChannelStats() []*metricsinfo.Channel { +func (m *manager) GetChannelStats(collectionID int64) []*metricsinfo.Channel { m.mu.RLock() defer m.mu.RUnlock() ret := make([]*metricsinfo.Channel, 0, len(m.channel2Pipeline)) for ch, p := range m.channel2Pipeline { + if collectionID > 0 && p.GetCollectionID() != collectionID { + continue + } delegator, ok := m.delegators.Get(ch) if ok { tt := delegator.GetTSafe() diff --git a/internal/querynodev2/server.go b/internal/querynodev2/server.go index d9cbae89b13a7..ea5723e9de55b 100644 --- a/internal/querynodev2/server.go +++ b/internal/querynodev2/server.go @@ -284,12 +284,14 @@ func (node *QueryNode) registerMetricsRequest() { node.metricsRequest.RegisterMetricsRequest(metricsinfo.SegmentKey, func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) { - return getSegmentJSON(node), nil + collectionID := metricsinfo.GetCollectionIDFromRequest(jsonReq) + return getSegmentJSON(node, collectionID), nil }) node.metricsRequest.RegisterMetricsRequest(metricsinfo.ChannelKey, func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) { - return getChannelJSON(node), nil + collectionID := metricsinfo.GetCollectionIDFromRequest(jsonReq) + return getChannelJSON(node, collectionID), nil }) log.Ctx(node.ctx).Info("register metrics actions finished") } diff --git a/pkg/util/metricsinfo/metric_request.go b/pkg/util/metricsinfo/metric_request.go index 02c0e5a6fe650..98a63fa9c9413 100644 --- a/pkg/util/metricsinfo/metric_request.go +++ b/pkg/util/metricsinfo/metric_request.go @@ -86,17 +86,21 @@ const ( MetricRequestParamTargetScopeKey = "target_scope" - MetricRequestParamINKey = "in" - MetricRequestParamCollectionIDKey = "collection_id" + + MetricRequestParamINKey = "in" + MetricsRequestParamsInDC = "dc" + MetricsRequestParamsInQC = "qc" + MetricsRequestParamsInDN = "dn" + MetricsRequestParamsInQN = "qn" ) -var MetricRequestParamINValue = map[string]struct{}{ - "dc": {}, - "qc": {}, - "dn": {}, - "qn": {}, -} +var ( + RequestParamsInDC = &commonpb.KeyValuePair{Key: MetricRequestParamINKey, Value: MetricsRequestParamsInDC} + RequestParamsInQC = &commonpb.KeyValuePair{Key: MetricRequestParamINKey, Value: MetricsRequestParamsInQC} + RequestParamsInDN = &commonpb.KeyValuePair{Key: MetricRequestParamINKey, Value: MetricsRequestParamsInDN} + RequestParamsInQN = &commonpb.KeyValuePair{Key: MetricRequestParamINKey, Value: MetricsRequestParamsInQN} +) type MetricsRequestAction func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) @@ -172,6 +176,14 @@ func ParseMetricRequestType(jsonRet gjson.Result) (string, error) { return "", fmt.Errorf("%s or %s not found in request", MetricTypeKey, MetricRequestTypeKey) } +func GetCollectionIDFromRequest(jsonReq gjson.Result) int64 { + v := jsonReq.Get(MetricRequestParamCollectionIDKey) + if !v.Exists() { + return 0 + } + return v.Int() +} + // ConstructRequestByMetricType constructs a request according to the metric type func ConstructRequestByMetricType(metricType string) (*milvuspb.GetMetricsRequest, error) { m := make(map[string]interface{}) diff --git a/pkg/util/metricsinfo/metrics_info.go b/pkg/util/metricsinfo/metrics_info.go index 6a6b5b46f8679..cded42ef32139 100644 --- a/pkg/util/metricsinfo/metrics_info.go +++ b/pkg/util/metricsinfo/metrics_info.go @@ -147,6 +147,7 @@ type IndexedField struct { BuildID int64 `json:"build_id,omitempty,string"` IndexSize int64 `json:"index_size,omitempty,string"` IsLoaded bool `json:"is_loaded,omitempty,string"` + HasRawData bool `json:"has_raw_data,omitempty"` } type QueryCoordTarget struct { @@ -195,6 +196,7 @@ type ResourceGroup struct { type Replica struct { ID int64 `json:"ID,omitempty,string"` CollectionID int64 `json:"collectionID,omitempty,string"` + DatabaseID int64 `json:"database_id,omitempty,string"` RWNodes []int64 `json:"rw_nodes,omitempty"` ResourceGroup string `json:"resource_group,omitempty"` RONodes []int64 `json:"ro_nodes,omitempty"` @@ -382,8 +384,8 @@ type ImportTask struct { } type CompactionTask struct { - PlanID int64 `json:"plan_id,omitempty"` - CollectionID int64 `json:"collection_id,omitempty"` + PlanID int64 `json:"plan_id,omitempty,string"` + CollectionID int64 `json:"collection_id,omitempty,string"` Type string `json:"type,omitempty"` State string `json:"state,omitempty"` FailReason string `json:"fail_reason,omitempty"` @@ -447,7 +449,7 @@ type Collection struct { ConsistencyLevel string `json:"consistency_level,omitempty"` Aliases []string `json:"aliases,omitempty"` Properties map[string]string `json:"properties,omitempty"` - DBName string `json:"db_name,omitempty,string"` + DBName string `json:"db_name,omitempty"` NumPartitions int `json:"num_partitions,omitempty,string"` VirtualChannelNames []string `json:"virtual_channel_names,omitempty"` PhysicalChannelNames []string `json:"physical_channel_names,omitempty"` @@ -458,7 +460,7 @@ type Collection struct { type Database struct { DBName string `json:"db_name,omitempty"` - DBID int64 `json:"dbID,omitempty"` + DBID int64 `json:"dbID,omitempty,string"` CreatedTimestamp string `json:"created_timestamp,omitempty"` Properties map[string]string `json:"properties,omitempty"` } From 22b8f6a58e50beeaff1349a2027e0063e41ec8d6 Mon Sep 17 00:00:00 2001 From: jaime Date: Thu, 9 Jan 2025 19:43:04 +0800 Subject: [PATCH 06/34] enhance: excessive disk usage logs generated if the path does not exist (#38821) issue: #38820 Signed-off-by: jaime --- pkg/util/hardware/hardware_info.go | 6 ++++++ pkg/util/hardware/hardware_info_test.go | 5 +++++ 2 files changed, 11 insertions(+) diff --git a/pkg/util/hardware/hardware_info.go b/pkg/util/hardware/hardware_info.go index 38fb6e220ea7e..877edd505c1e3 100644 --- a/pkg/util/hardware/hardware_info.go +++ b/pkg/util/hardware/hardware_info.go @@ -17,6 +17,8 @@ import ( "runtime" "sync" + "github.com/cockroachdb/errors" + "github.com/cockroachdb/errors/oserror" "github.com/shirou/gopsutil/v3/cpu" "github.com/shirou/gopsutil/v3/disk" "github.com/shirou/gopsutil/v3/mem" @@ -107,6 +109,10 @@ func GetFreeMemoryCount() uint64 { func GetDiskUsage(path string) (float64, float64, error) { diskStats, err := disk.Usage(path) if err != nil { + // If the path does not exist, ignore the error and return 0. + if errors.Is(err, oserror.ErrNotExist) { + return 0, 0, nil + } return 0, 0, err } usedGB := float64(diskStats.Used) / 1e9 diff --git a/pkg/util/hardware/hardware_info_test.go b/pkg/util/hardware/hardware_info_test.go index 037e1c3a58cb5..7bb79de04d935 100644 --- a/pkg/util/hardware/hardware_info_test.go +++ b/pkg/util/hardware/hardware_info_test.go @@ -47,6 +47,11 @@ func TestGetDiskUsage(t *testing.T) { assert.NoError(t, err) assert.GreaterOrEqual(t, used, 0.0) assert.GreaterOrEqual(t, total, 0.0) + + used, total, err = GetDiskUsage("/dir_not_exist") + assert.NoError(t, err) + assert.Equal(t, 0.0, used) + assert.Equal(t, 0.0, total) } func TestGetIOWait(t *testing.T) { From 461c376b4655b9ec07e32ee875ba0b9f968256c8 Mon Sep 17 00:00:00 2001 From: ThreadDao Date: Fri, 10 Jan 2025 09:38:58 +0800 Subject: [PATCH 07/34] test: add test cases for template param (#38867) issue: #33419 - add case for Get - add case for TemplateParam Signed-off-by: ThreadDao --- tests/go_client/base/milvus_client.go | 6 + .../go_client/testcases/helper/data_helper.go | 2 +- tests/go_client/testcases/index_test.go | 6 +- tests/go_client/testcases/query_test.go | 181 ++++++++++++++++++ tests/go_client/testcases/search_test.go | 19 +- 5 files changed, 206 insertions(+), 8 deletions(-) diff --git a/tests/go_client/base/milvus_client.go b/tests/go_client/base/milvus_client.go index 3d4affd71a8f8..8a8f2a1175c5b 100644 --- a/tests/go_client/base/milvus_client.go +++ b/tests/go_client/base/milvus_client.go @@ -262,3 +262,9 @@ func (mc *MilvusClient) Query(ctx context.Context, option client.QueryOption, ca resultSet, err := mc.mClient.Query(ctx, option, callOptions...) return resultSet, err } + +// Get get from collection +func (mc *MilvusClient) Get(ctx context.Context, option client.QueryOption, callOptions ...grpc.CallOption) (client.ResultSet, error) { + resultSet, err := mc.mClient.Get(ctx, option, callOptions...) + return resultSet, err +} diff --git a/tests/go_client/testcases/helper/data_helper.go b/tests/go_client/testcases/helper/data_helper.go index 79ab1aa557c7e..a15d2a144723e 100644 --- a/tests/go_client/testcases/helper/data_helper.go +++ b/tests/go_client/testcases/helper/data_helper.go @@ -290,7 +290,7 @@ func GenColumnData(nb int, fieldType entity.FieldType, option GenDataOption) col case entity.FieldTypeBool: boolValues := make([]bool, 0, nb) for i := start; i < start+nb; i++ { - boolValues = append(boolValues, i/2 == 0) + boolValues = append(boolValues, i%2 == 0) } return column.NewColumnBool(fieldName, boolValues) diff --git a/tests/go_client/testcases/index_test.go b/tests/go_client/testcases/index_test.go index 98748ef5e57ef..277609f9dfc4f 100644 --- a/tests/go_client/testcases/index_test.go +++ b/tests/go_client/testcases/index_test.go @@ -18,7 +18,7 @@ import ( func TestIndexVectorDefault(t *testing.T) { t.Parallel() - ctx := hp.CreateContext(t, time.Second*common.DefaultTimeout) + ctx := hp.CreateContext(t, time.Second*common.DefaultTimeout*2) mc := createDefaultMilvusClient(ctx, t) cp := hp.NewCreateCollectionParams(hp.Int64MultiVec) @@ -51,7 +51,7 @@ func TestIndexVectorDefault(t *testing.T) { func TestIndexVectorIP(t *testing.T) { t.Parallel() - ctx := hp.CreateContext(t, time.Second*common.DefaultTimeout) + ctx := hp.CreateContext(t, time.Second*common.DefaultTimeout*2) mc := createDefaultMilvusClient(ctx, t) cp := hp.NewCreateCollectionParams(hp.Int64MultiVec) @@ -85,7 +85,7 @@ func TestIndexVectorIP(t *testing.T) { func TestIndexVectorCosine(t *testing.T) { t.Parallel() - ctx := hp.CreateContext(t, time.Second*common.DefaultTimeout) + ctx := hp.CreateContext(t, time.Second*common.DefaultTimeout*2) mc := createDefaultMilvusClient(ctx, t) cp := hp.NewCreateCollectionParams(hp.Int64MultiVec) diff --git a/tests/go_client/testcases/query_test.go b/tests/go_client/testcases/query_test.go index d28c5cadc4082..f67f2b89e55fa 100644 --- a/tests/go_client/testcases/query_test.go +++ b/tests/go_client/testcases/query_test.go @@ -35,6 +35,18 @@ func TestQueryDefault(t *testing.T) { queryRes, err := mc.Query(ctx, client.NewQueryOption(schema.CollectionName).WithFilter(expr)) common.CheckErr(t, err, true) common.CheckQueryResult(t, queryRes.Fields, []column.Column{insertRes.IDs.Slice(0, 100)}) + + // query with limit + LimitRes, err := mc.Query(ctx, client.NewQueryOption(schema.CollectionName).WithFilter(expr).WithLimit(10)) + common.CheckErr(t, err, true) + require.Equal(t, 10, LimitRes.ResultCount) + require.Equal(t, 10, LimitRes.GetColumn(common.DefaultInt64FieldName).Len()) + + // get ids -> same result with query + ids := hp.GenColumnData(100, entity.FieldTypeInt64, *hp.TNewDataOption().TWithFieldName(common.DefaultInt64FieldName)) + getRes, errGet := mc.Get(ctx, client.NewQueryOption(schema.CollectionName).WithIDs(ids)) + common.CheckErr(t, errGet, true) + common.CheckQueryResult(t, getRes.Fields, []column.Column{insertRes.IDs.Slice(0, 100)}) } // test query with varchar field filter @@ -55,6 +67,46 @@ func TestQueryVarcharPkDefault(t *testing.T) { queryRes, err := mc.Query(ctx, client.NewQueryOption(schema.CollectionName).WithFilter(expr)) common.CheckErr(t, err, true) common.CheckQueryResult(t, queryRes.Fields, []column.Column{insertRes.IDs.Slice(0, 5)}) + + // get ids -> same result with query + varcharValues := []string{"0", "1", "2", "3", "4"} + ids := column.NewColumnVarChar(common.DefaultVarcharFieldName, varcharValues) + getRes, errGet := mc.Get(ctx, client.NewQueryOption(schema.CollectionName).WithIDs(ids)) + common.CheckErr(t, errGet, true) + common.CheckQueryResult(t, getRes.Fields, []column.Column{insertRes.IDs.Slice(0, 5)}) +} + +// test get with invalid ids +func TestGetInvalid(t *testing.T) { + ctx := hp.CreateContext(t, time.Second*common.DefaultTimeout) + mc := createDefaultMilvusClient(ctx, t) + + // create and insert + prepare, schema := hp.CollPrepare.CreateCollection(ctx, t, mc, hp.NewCreateCollectionParams(hp.Int64Vec), hp.TNewFieldsOption(), hp.TNewSchemaOption()) + prepare.InsertData(ctx, t, mc, hp.NewInsertParams(schema), hp.TNewDataOption()) + + // flush -> index -> load + prepare.FlushData(ctx, t, mc, schema.CollectionName) + prepare.CreateIndex(ctx, t, mc, hp.TNewIndexParams(schema)) + prepare.Load(ctx, t, mc, hp.NewLoadParams(schema.CollectionName)) + + // get ids with varchar ids -> error + varcharValues := []string{"0", "1", "2", "3", "4"} + ids := column.NewColumnVarChar(common.DefaultVarcharFieldName, varcharValues) + _, errGet := mc.Get(ctx, client.NewQueryOption(schema.CollectionName).WithIDs(ids)) + common.CheckErr(t, errGet, false, "field varchar not exist: invalid parameter") + + // get ids with varchar ids -> error + ids = column.NewColumnVarChar(common.DefaultInt64FieldName, varcharValues) + _, errGet = mc.Get(ctx, client.NewQueryOption(schema.CollectionName).WithIDs(ids)) + common.CheckErr(t, errGet, false, "cannot parse expression: int64 in") + + // get ids with non-pk column -> error for empty filter + t.Log("https://github.com/milvus-io/milvus/issues/38859") + values := []float32{0.0, 1.0} + ids2 := column.NewColumnFloat(common.DefaultInt64FieldName, values) + _, errGet = mc.Get(ctx, client.NewQueryOption(schema.CollectionName).WithIDs(ids2)) + common.CheckErr(t, errGet, false, "empty expression should be used with limit") } // query from not existed collection name and partition name @@ -635,3 +687,132 @@ func TestQueryOutputInvalidOutputFieldCount(t *testing.T) { common.CheckErr(t, err, false, invalidCount.errMsg) } } + +func TestQueryWithTemplateParam(t *testing.T) { + ctx := hp.CreateContext(t, time.Second*common.DefaultTimeout) + mc := createDefaultMilvusClient(ctx, t) + + prepare, schema := hp.CollPrepare.CreateCollection(ctx, t, mc, hp.NewCreateCollectionParams(hp.AllFields), + hp.TNewFieldsOption(), hp.TNewSchemaOption().TWithEnableDynamicField(true)) + prepare.InsertData(ctx, t, mc, hp.NewInsertParams(schema), hp.TNewDataOption()) + prepare.CreateIndex(ctx, t, mc, hp.TNewIndexParams(schema)) + prepare.Load(ctx, t, mc, hp.NewLoadParams(schema.CollectionName)) + + // query + int64Values := make([]int64, 0, 1000) + for i := 10; i < 10+1000; i++ { + int64Values = append(int64Values, int64(i)) + } + // default + queryRes, err := mc.Query(ctx, client.NewQueryOption(schema.CollectionName). + WithFilter(fmt.Sprintf("%s in {int64Values}", common.DefaultInt64FieldName)).WithTemplateParam("int64Values", int64Values)) + common.CheckErr(t, err, true) + common.CheckQueryResult(t, queryRes.Fields, []column.Column{column.NewColumnInt64(common.DefaultInt64FieldName, int64Values)}) + + // cover keys + res, err := mc.Query(ctx, client.NewQueryOption(schema.CollectionName).WithFilter("int64 < {k2}").WithTemplateParam("k2", 10).WithTemplateParam("k2", 5)) + common.CheckErr(t, err, true) + require.Equal(t, 5, res.ResultCount) + + // array contains + anyValues := []int64{0.0, 100.0, 10000.0} + countRes, err := mc.Query(ctx, client.NewQueryOption(schema.CollectionName). + WithFilter(fmt.Sprintf("json_contains_any (%s, {any_values})", common.DefaultFloatArrayField)).WithTemplateParam("any_values", anyValues). + WithOutputFields(common.QueryCountFieldName)) + common.CheckErr(t, err, true) + count, _ := countRes.Fields[0].GetAsInt64(0) + require.EqualValues(t, 101, count) + + // dynamic + countRes, err = mc.Query(ctx, client.NewQueryOption(schema.CollectionName). + WithFilter("dynamicNumber % 2 == {v}").WithTemplateParam("v", 0).WithOutputFields(common.QueryCountFieldName)) + common.CheckErr(t, err, true) + count, _ = countRes.Fields[0].GetAsInt64(0) + require.EqualValues(t, 1500, count) + + // json['bool'] + countRes, err = mc.Query(ctx, client.NewQueryOption(schema.CollectionName). + WithFilter(fmt.Sprintf("%s['bool'] == {v}", common.DefaultJSONFieldName)). + WithTemplateParam("v", false). + WithOutputFields(common.QueryCountFieldName)) + common.CheckErr(t, err, true) + count, _ = countRes.Fields[0].GetAsInt64(0) + require.EqualValues(t, 1500/2, count) + + // bool + countRes, err = mc.Query(ctx, client.NewQueryOption(schema.CollectionName). + WithFilter(fmt.Sprintf("%s == {v}", common.DefaultBoolFieldName)). + WithTemplateParam("v", true). + WithOutputFields(common.QueryCountFieldName)) + common.CheckErr(t, err, true) + count, _ = countRes.Fields[0].GetAsInt64(0) + require.EqualValues(t, common.DefaultNb/2, count) + + // and {expr: fmt.Sprintf("%s >= 1000 && %s < 2000", common.DefaultInt64FieldName, common.DefaultInt64FieldName), count: 1000}, + res, err = mc.Query(ctx, client.NewQueryOption(schema.CollectionName). + WithFilter(fmt.Sprintf("%s >= {k1} && %s < {k2}", common.DefaultInt64FieldName, common.DefaultInt64FieldName)). + WithTemplateParam("v", 0).WithTemplateParam("k1", 1000). + WithTemplateParam("k2", 2000)) + common.CheckErr(t, err, true) + require.EqualValues(t, 1000, res.ResultCount) +} + +func TestQueryWithTemplateParamInvalid(t *testing.T) { + ctx := hp.CreateContext(t, time.Second*common.DefaultTimeout) + mc := createDefaultMilvusClient(ctx, t) + + prepare, schema := hp.CollPrepare.CreateCollection(ctx, t, mc, hp.NewCreateCollectionParams(hp.Int64VarcharSparseVec), + hp.TNewFieldsOption(), hp.TNewSchemaOption().TWithEnableDynamicField(true)) + prepare.InsertData(ctx, t, mc, hp.NewInsertParams(schema), hp.TNewDataOption()) + prepare.CreateIndex(ctx, t, mc, hp.TNewIndexParams(schema)) + prepare.Load(ctx, t, mc, hp.NewLoadParams(schema.CollectionName)) + + // query with invalid template + // expr := "varchar like 'a%' " + _, err2 := mc.Query(ctx, client.NewQueryOption(schema.CollectionName).WithFilter("varchar like {key1}").WithTemplateParam("key1", "'a%'")) + common.CheckErr(t, err2, false, "mismatched input '{' expecting StringLiteral") + + // no template param + _, err := mc.Query(ctx, client.NewQueryOption(schema.CollectionName).WithFilter("int64 in {key1}")) + common.CheckErr(t, err, false, "the value of expression template variable name {key1} is not found") + + // template param with empty expr + _, err = mc.Query(ctx, client.NewQueryOption(schema.CollectionName).WithFilter("").WithTemplateParam("a", 12)) + common.CheckErr(t, err, false, "empty expression should be used with limit") + + // *** template param with field name key -> error *** + _, err = mc.Query(ctx, client.NewQueryOption(schema.CollectionName).WithFilter("{field} < 10").WithTemplateParam("field", "int64")) + common.CheckErr(t, err, false, "cannot parse expression") + _, err = mc.Query(ctx, client.NewQueryOption(schema.CollectionName).WithFilter("{field} < {v}").WithTemplateParam("field", "int64").WithTemplateParam("v", 10)) + common.CheckErr(t, err, false, "placeholder was not supported between two constants with operator") + // exists x + _, err = mc.Query(ctx, client.NewQueryOption(schema.CollectionName).WithFilter("exists {x}").WithTemplateParam("x", "json")) + common.CheckErr(t, err, false, "exists operations are only supported on single fields now") + + // compare two fields + _, err = mc.Query(ctx, client.NewQueryOption(schema.CollectionName).WithFilter("{f1} > {f2}").WithTemplateParam("f1", "f1").WithTemplateParam("f2", "f2")) + common.CheckErr(t, err, false, "placeholder was not supported between two constants with operator") + + // expr key != template key + _, err = mc.Query(ctx, client.NewQueryOption(schema.CollectionName).WithFilter("int64 in {key1}").WithTemplateParam("key2", []int64{0, 1, 2})) + common.CheckErr(t, err, false, "the value of expression template variable name {key1} is not found") + + // template missing some keys + _, err = mc.Query(ctx, client.NewQueryOption(schema.CollectionName).WithFilter("{k1} < int64 < {k2}").WithTemplateParam("k1", 10)) + common.CheckErr(t, err, false, "the upper value of expression template variable name {k2} is not found") + + // template value type is valid + _, err = mc.Query(ctx, client.NewQueryOption(schema.CollectionName).WithFilter("int64 < {k1}").WithTemplateParam("k1", []int64{0, 1, 3})) + common.CheckErr(t, err, false, "cannot cast value to Int64") + + _, err = mc.Query(ctx, client.NewQueryOption(schema.CollectionName).WithFilter("int64 < {k1}").WithTemplateParam("k1", "10")) + common.CheckErr(t, err, false, "cannot cast value to Int64") + + // invalid expr + _, err = mc.Query(ctx, client.NewQueryOption(schema.CollectionName).WithFilter("{name} == 'O'Reilly'").WithTemplateParam("name", common.DefaultVarcharFieldName)) + common.CheckErr(t, err, false, "cannot parse expression") + + // invalid expr + _, err = mc.Query(ctx, client.NewQueryOption(schema.CollectionName).WithFilter("{_123} > 10").WithTemplateParam("_123", common.DefaultInt64FieldName)) + common.CheckErr(t, err, false, "cannot parse expression") +} diff --git a/tests/go_client/testcases/search_test.go b/tests/go_client/testcases/search_test.go index cb3e7373fb4ac..d38465f4f4d0e 100644 --- a/tests/go_client/testcases/search_test.go +++ b/tests/go_client/testcases/search_test.go @@ -610,8 +610,9 @@ func TestSearchExpr(t *testing.T) { prepare.Load(ctx, t, mc, hp.NewLoadParams(schema.CollectionName)) type mExprExpected struct { - expr string - ids []int64 + expr string + ids []int64 + value any } vectors := hp.GenSearchVectors(common.DefaultNq, common.DefaultDim, entity.FieldTypeFloatVector) @@ -619,8 +620,18 @@ func TestSearchExpr(t *testing.T) { {expr: fmt.Sprintf("%s < 10", common.DefaultInt64FieldName), ids: []int64{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}}, {expr: fmt.Sprintf("%s in [10, 100]", common.DefaultInt64FieldName), ids: []int64{10, 100}}, } { - resSearch, errSearch := mc.Search(ctx, client.NewSearchOption(schema.CollectionName, common.DefaultLimit, vectors).WithConsistencyLevel(entity.ClStrong). - WithFilter(_mExpr.expr)) + resSearch, errSearch := mc.Search(ctx, client.NewSearchOption(schema.CollectionName, common.DefaultLimit, vectors).WithConsistencyLevel(entity.ClStrong).WithFilter(_mExpr.expr)) + common.CheckErr(t, errSearch, true) + for _, res := range resSearch { + require.ElementsMatch(t, _mExpr.ids, res.IDs.(*column.ColumnInt64).Data()) + } + } + // search with template param + for _, _mExpr := range []mExprExpected{ + {expr: fmt.Sprintf("%s < {v}", common.DefaultInt64FieldName), ids: []int64{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}, value: 10}, + {expr: fmt.Sprintf("%s in {v}", common.DefaultInt64FieldName), ids: []int64{10, 100}, value: []int64{10, 100}}, + } { + resSearch, errSearch := mc.Search(ctx, client.NewSearchOption(schema.CollectionName, common.DefaultLimit, vectors).WithFilter(_mExpr.expr).WithTemplateParam("v", _mExpr.value)) common.CheckErr(t, errSearch, true) for _, res := range resSearch { require.ElementsMatch(t, _mExpr.ids, res.IDs.(*column.ColumnInt64).Data()) From bb8d1ab3bf09875dc5893116109f6cfb90294610 Mon Sep 17 00:00:00 2001 From: Zhen Ye Date: Fri, 10 Jan 2025 10:49:01 +0800 Subject: [PATCH 08/34] enhance: make new go package to manage proto (#39114) issue: #39095 --------- Signed-off-by: chyezh --- .gitignore | 1 - cmd/milvus/mck.go | 6 +- cmd/tools/datameta/main.go | 2 +- cmd/tools/migration/backend/etcd210.go | 4 +- cmd/tools/migration/meta/210_to_220.go | 4 +- cmd/tools/migration/meta/meta210.go | 2 +- cmd/tools/migration/meta/meta220.go | 2 +- internal/allocator/id_allocator.go | 2 +- internal/allocator/id_allocator_test.go | 2 +- internal/allocator/remote_interface.go | 2 +- internal/coordinator/coordclient/registry.go | 6 +- .../coordinator/coordclient/registry_test.go | 6 +- internal/datacoord/allocator/allocator.go | 2 +- .../datacoord/allocator/allocator_test.go | 2 +- internal/datacoord/analyze_meta.go | 4 +- internal/datacoord/analyze_meta_test.go | 4 +- internal/datacoord/channel.go | 2 +- internal/datacoord/channel_manager.go | 2 +- internal/datacoord/channel_manager_test.go | 2 +- internal/datacoord/channel_store.go | 2 +- internal/datacoord/channel_store_test.go | 2 +- internal/datacoord/cluster.go | 2 +- internal/datacoord/cluster_test.go | 2 +- internal/datacoord/compaction.go | 2 +- internal/datacoord/compaction_l0_view_test.go | 2 +- .../datacoord/compaction_policy_clustering.go | 2 +- .../compaction_policy_clustering_test.go | 2 +- internal/datacoord/compaction_policy_l0.go | 2 +- .../datacoord/compaction_policy_l0_test.go | 2 +- .../datacoord/compaction_policy_single.go | 2 +- .../compaction_policy_single_test.go | 2 +- internal/datacoord/compaction_queue.go | 2 +- internal/datacoord/compaction_queue_test.go | 2 +- internal/datacoord/compaction_task.go | 2 +- .../datacoord/compaction_task_clustering.go | 4 +- .../compaction_task_clustering_test.go | 4 +- internal/datacoord/compaction_task_l0.go | 2 +- internal/datacoord/compaction_task_l0_test.go | 2 +- internal/datacoord/compaction_task_meta.go | 2 +- .../datacoord/compaction_task_meta_test.go | 2 +- internal/datacoord/compaction_task_mix.go | 2 +- .../datacoord/compaction_task_mix_test.go | 2 +- internal/datacoord/compaction_test.go | 2 +- internal/datacoord/compaction_trigger.go | 2 +- internal/datacoord/compaction_trigger_test.go | 2 +- internal/datacoord/compaction_trigger_v2.go | 2 +- .../datacoord/compaction_trigger_v2_test.go | 2 +- internal/datacoord/compaction_view.go | 2 +- internal/datacoord/garbage_collector.go | 2 +- internal/datacoord/garbage_collector_test.go | 4 +- internal/datacoord/handler.go | 2 +- internal/datacoord/handler_test.go | 6 +- internal/datacoord/import_checker.go | 6 +- internal/datacoord/import_checker_test.go | 6 +- internal/datacoord/import_job.go | 4 +- internal/datacoord/import_meta_test.go | 4 +- internal/datacoord/import_scheduler.go | 4 +- internal/datacoord/import_scheduler_test.go | 2 +- internal/datacoord/import_task.go | 2 +- internal/datacoord/import_util.go | 6 +- internal/datacoord/import_util_test.go | 6 +- internal/datacoord/index_meta.go | 4 +- internal/datacoord/index_meta_test.go | 4 +- internal/datacoord/index_service.go | 4 +- internal/datacoord/index_service_test.go | 6 +- internal/datacoord/job_manager.go | 4 +- internal/datacoord/job_manager_test.go | 4 +- internal/datacoord/meta.go | 4 +- internal/datacoord/meta_test.go | 2 +- internal/datacoord/meta_util.go | 2 +- internal/datacoord/metrics_info.go | 4 +- internal/datacoord/metrics_info_test.go | 2 +- internal/datacoord/mock_channelmanager.go | 2 +- internal/datacoord/mock_cluster.go | 2 +- internal/datacoord/mock_compaction_meta.go | 2 +- .../datacoord/mock_compaction_plan_context.go | 2 +- internal/datacoord/mock_handler.go | 2 +- internal/datacoord/mock_job_manager.go | 2 +- internal/datacoord/mock_subcluster.go | 2 +- internal/datacoord/mock_test.go | 8 +- internal/datacoord/partition_stats_meta.go | 2 +- .../datacoord/partition_stats_meta_test.go | 2 +- internal/datacoord/policy_test.go | 2 +- .../datacoord/segment_allocation_policy.go | 2 +- .../segment_allocation_policy_test.go | 2 +- internal/datacoord/segment_info.go | 2 +- internal/datacoord/segment_info_test.go | 2 +- internal/datacoord/segment_manager.go | 2 +- internal/datacoord/segment_manager_test.go | 2 +- internal/datacoord/segment_operator.go | 2 +- internal/datacoord/segment_operator_test.go | 2 +- internal/datacoord/server.go | 2 +- internal/datacoord/server_test.go | 6 +- internal/datacoord/services.go | 4 +- internal/datacoord/services_test.go | 6 +- .../datacoord/session/datanode_manager.go | 2 +- .../session/datanode_manager_test.go | 2 +- .../datacoord/session/indexnode_manager.go | 2 +- .../session/indexnode_manager_test.go | 2 +- .../session/mock_datanode_manager.go | 2 +- internal/datacoord/stats_task_meta.go | 4 +- internal/datacoord/stats_task_meta_test.go | 6 +- internal/datacoord/sync_segments_scheduler.go | 2 +- .../datacoord/sync_segments_scheduler_test.go | 2 +- internal/datacoord/task_analyze.go | 4 +- internal/datacoord/task_index.go | 4 +- internal/datacoord/task_scheduler.go | 4 +- internal/datacoord/task_scheduler_test.go | 6 +- internal/datacoord/task_stats.go | 4 +- internal/datacoord/task_stats_test.go | 6 +- internal/datacoord/types.go | 2 +- internal/datacoord/util.go | 4 +- internal/datacoord/util_test.go | 4 +- internal/datanode/allocator/allocator_test.go | 2 +- internal/datanode/channel/channel_manager.go | 2 +- .../datanode/channel/channel_manager_test.go | 2 +- .../datanode/channel/mock_channelmanager.go | 2 +- .../compaction/clustering_compactor.go | 6 +- .../compaction/clustering_compactor_test.go | 2 +- internal/datanode/compaction/compactor.go | 2 +- .../datanode/compaction/compactor_common.go | 2 +- internal/datanode/compaction/executor.go | 2 +- internal/datanode/compaction/executor_test.go | 2 +- internal/datanode/compaction/l0_compactor.go | 2 +- .../datanode/compaction/l0_compactor_test.go | 2 +- internal/datanode/compaction/load_stats.go | 2 +- internal/datanode/compaction/merge_sort.go | 2 +- internal/datanode/compaction/mix_compactor.go | 2 +- .../datanode/compaction/mix_compactor_test.go | 4 +- .../datanode/compaction/mock_compactor.go | 2 +- .../datanode/compaction/segment_writer.go | 4 +- internal/datanode/data_node_test.go | 2 +- internal/datanode/importv2/hash.go | 2 +- internal/datanode/importv2/scheduler.go | 2 +- internal/datanode/importv2/scheduler_test.go | 4 +- internal/datanode/importv2/task.go | 2 +- internal/datanode/importv2/task_import.go | 4 +- internal/datanode/importv2/task_l0_import.go | 2 +- .../datanode/importv2/task_l0_import_test.go | 4 +- .../datanode/importv2/task_l0_preimport.go | 4 +- .../datanode/importv2/task_manager_test.go | 2 +- internal/datanode/importv2/task_preimport.go | 4 +- internal/datanode/importv2/util.go | 2 +- internal/datanode/importv2/util_test.go | 2 +- .../iterators/binlog_iterator_test.go | 2 +- internal/datanode/services.go | 4 +- internal/datanode/services_test.go | 4 +- internal/datanode/util/meta_util.go | 4 +- internal/distributed/connection_manager.go | 8 +- .../distributed/connection_manager_test.go | 8 +- .../distributed/datacoord/client/client.go | 6 +- .../datacoord/client/client_test.go | 6 +- internal/distributed/datacoord/service.go | 6 +- .../distributed/datacoord/service_test.go | 6 +- .../distributed/datanode/client/client.go | 4 +- .../datanode/client/client_test.go | 2 +- internal/distributed/datanode/service.go | 4 +- internal/distributed/datanode/service_test.go | 4 +- .../distributed/indexnode/client/client.go | 4 +- .../indexnode/client/client_test.go | 4 +- internal/distributed/indexnode/service.go | 4 +- .../distributed/indexnode/service_test.go | 4 +- internal/distributed/proxy/client/client.go | 4 +- .../distributed/proxy/client/client_test.go | 4 +- .../proxy/httpserver/handler_v2.go | 2 +- .../proxy/httpserver/handler_v2_test.go | 2 +- internal/distributed/proxy/service.go | 4 +- .../distributed/querycoord/client/client.go | 4 +- .../querycoord/client/client_test.go | 2 +- internal/distributed/querycoord/service.go | 4 +- .../distributed/querycoord/service_test.go | 4 +- .../distributed/querynode/client/client.go | 4 +- .../querynode/client/client_test.go | 2 +- internal/distributed/querynode/service.go | 4 +- .../distributed/querynode/service_test.go | 4 +- .../distributed/rootcoord/client/client.go | 6 +- .../rootcoord/client/client_test.go | 2 +- internal/distributed/rootcoord/service.go | 6 +- .../distributed/rootcoord/service_test.go | 2 +- internal/distributed/streamingnode/service.go | 2 +- internal/flushcommon/broker/broker.go | 2 +- internal/flushcommon/broker/datacoord.go | 2 +- internal/flushcommon/broker/datacoord_test.go | 2 +- internal/flushcommon/broker/mock_broker.go | 2 +- internal/flushcommon/metacache/actions.go | 2 +- internal/flushcommon/metacache/meta_cache.go | 2 +- .../flushcommon/metacache/meta_cache_test.go | 2 +- .../flushcommon/metacache/mock_meta_cache.go | 2 +- internal/flushcommon/metacache/segment.go | 2 +- .../flushcommon/metacache/segment_test.go | 2 +- .../flushcommon/pipeline/data_sync_service.go | 2 +- .../pipeline/data_sync_service_test.go | 2 +- .../pipeline/flow_graph_dd_node.go | 2 +- .../pipeline/flow_graph_dd_node_test.go | 2 +- .../pipeline/flow_graph_manager_test.go | 2 +- .../flushcommon/pipeline/testutils_test.go | 6 +- internal/flushcommon/syncmgr/meta_writer.go | 2 +- .../flushcommon/syncmgr/meta_writer_test.go | 2 +- internal/flushcommon/syncmgr/options.go | 2 +- internal/flushcommon/syncmgr/serializer.go | 2 +- .../flushcommon/syncmgr/storage_serializer.go | 4 +- .../syncmgr/storage_serializer_test.go | 2 +- .../flushcommon/syncmgr/sync_manager_test.go | 2 +- internal/flushcommon/syncmgr/task.go | 2 +- internal/flushcommon/syncmgr/task_test.go | 2 +- .../writebuffer/l0_write_buffer.go | 2 +- .../writebuffer/l0_write_buffer_test.go | 2 +- .../flushcommon/writebuffer/write_buffer.go | 2 +- .../writebuffer/write_buffer_test.go | 2 +- internal/indexnode/chunk_mgr_factory.go | 2 +- internal/indexnode/chunkmgr_mock.go | 4 +- internal/indexnode/index_test.go | 2 +- internal/indexnode/indexnode.go | 2 +- internal/indexnode/indexnode_service.go | 4 +- internal/indexnode/indexnode_service_test.go | 6 +- internal/indexnode/indexnode_test.go | 6 +- internal/indexnode/task.go | 2 +- internal/indexnode/task_analyze.go | 6 +- internal/indexnode/task_index.go | 6 +- internal/indexnode/task_scheduler.go | 2 +- internal/indexnode/task_scheduler_test.go | 2 +- internal/indexnode/task_stats.go | 8 +- internal/indexnode/task_stats_test.go | 2 +- internal/indexnode/task_test.go | 6 +- internal/indexnode/taskinfo_ops.go | 4 +- internal/indexnode/taskinfo_ops_test.go | 4 +- internal/metastore/catalog.go | 8 +- internal/metastore/kv/binlog/binlog.go | 2 +- internal/metastore/kv/binlog/binlog_test.go | 2 +- internal/metastore/kv/datacoord/kv_catalog.go | 4 +- .../metastore/kv/datacoord/kv_catalog_test.go | 4 +- internal/metastore/kv/datacoord/util.go | 2 +- .../metastore/kv/querycoord/kv_catalog.go | 2 +- .../kv/querycoord/kv_catalog_test.go | 2 +- internal/metastore/kv/rootcoord/kv_catalog.go | 4 +- .../metastore/kv/rootcoord/kv_catalog_test.go | 4 +- .../metastore/kv/streamingcoord/kv_catalog.go | 2 +- .../kv/streamingcoord/kv_catalog_test.go | 2 +- .../metastore/kv/streamingnode/kv_catalog.go | 2 +- .../kv/streamingnode/kv_catalog_test.go | 2 +- .../metastore/mocks/mock_datacoord_catalog.go | 4 +- .../mocks/mock_querycoord_catalog.go | 2 +- internal/metastore/model/alias.go | 2 +- internal/metastore/model/alias_test.go | 2 +- internal/metastore/model/collection.go | 2 +- internal/metastore/model/collection_test.go | 2 +- internal/metastore/model/credential.go | 2 +- internal/metastore/model/credential_test.go | 2 +- internal/metastore/model/database.go | 2 +- internal/metastore/model/database_test.go | 2 +- internal/metastore/model/index.go | 2 +- internal/metastore/model/index_test.go | 2 +- internal/metastore/model/load_info.go | 2 +- internal/metastore/model/partition.go | 2 +- internal/metastore/model/segment_index.go | 2 +- .../metastore/model/segment_index_test.go | 2 +- internal/mocks/mock_datacoord.go | 6 +- internal/mocks/mock_datacoord_client.go | 6 +- internal/mocks/mock_datanode.go | 4 +- internal/mocks/mock_datanode_client.go | 4 +- internal/mocks/mock_indexnode.go | 4 +- internal/mocks/mock_indexnode_client.go | 4 +- .../mock_StreamingCoordCataLog.go | 2 +- .../mock_StreamingNodeCataLog.go | 2 +- internal/mocks/mock_proxy.go | 4 +- internal/mocks/mock_proxy_client.go | 4 +- internal/mocks/mock_querycoord.go | 4 +- internal/mocks/mock_querycoord_client.go | 4 +- internal/mocks/mock_querynode.go | 4 +- internal/mocks/mock_querynode_client.go | 4 +- internal/mocks/mock_rootcoord.go | 6 +- internal/mocks/mock_rootcoord_client.go | 6 +- internal/mocks/util/mock_segcore/mock_data.go | 14 +- .../parser/planparserv2/check_identical.go | 2 +- .../planparserv2/check_identical_test.go | 2 +- .../convert_field_data_to_generic_value.go | 2 +- ...onvert_field_data_to_generic_value_test.go | 2 +- .../planparserv2/fill_expression_value.go | 2 +- .../planparserv2/logical_expr_visitor.go | 2 +- internal/parser/planparserv2/node_ret.go | 2 +- internal/parser/planparserv2/operators.go | 2 +- .../parser/planparserv2/parser_visitor.go | 2 +- internal/parser/planparserv2/pattern_match.go | 2 +- .../parser/planparserv2/pattern_match_test.go | 2 +- .../parser/planparserv2/plan_parser_v2.go | 2 +- .../planparserv2/plan_parser_v2_test.go | 2 +- internal/parser/planparserv2/show_visitor.go | 2 +- internal/parser/planparserv2/utils.go | 2 +- internal/parser/planparserv2/utils_test.go | 2 +- internal/proxy/count_reducer.go | 2 +- internal/proxy/count_reducer_test.go | 2 +- internal/proxy/data_coord_mock_test.go | 6 +- internal/proxy/default_limit_reducer.go | 2 +- internal/proxy/http_req_impl.go | 2 +- internal/proxy/http_req_impl_test.go | 2 +- internal/proxy/impl.go | 8 +- internal/proxy/impl_test.go | 10 +- internal/proxy/interface_def.go | 2 +- internal/proxy/lb_balancer.go | 2 +- internal/proxy/lb_policy.go | 2 +- internal/proxy/lb_policy_test.go | 4 +- internal/proxy/look_aside_balancer.go | 2 +- internal/proxy/look_aside_balancer_test.go | 2 +- internal/proxy/management.go | 4 +- internal/proxy/management_test.go | 4 +- internal/proxy/meta_cache.go | 6 +- internal/proxy/meta_cache_test.go | 6 +- internal/proxy/metrics_info.go | 2 +- internal/proxy/mock_cache.go | 2 +- internal/proxy/mock_lb_balancer.go | 2 +- internal/proxy/mock_lb_policy.go | 2 +- internal/proxy/mock_test.go | 2 +- internal/proxy/mock_tso_test.go | 2 +- internal/proxy/msg_pack_test.go | 2 +- internal/proxy/privilege_interceptor_test.go | 2 +- internal/proxy/proxy.go | 2 +- internal/proxy/proxy_rpc_test.go | 4 +- internal/proxy/proxy_test.go | 8 +- internal/proxy/rate_limit_interceptor_test.go | 2 +- internal/proxy/reducer.go | 4 +- internal/proxy/reducer_test.go | 2 +- internal/proxy/rootcoord_mock_test.go | 8 +- internal/proxy/roundrobin_balancer.go | 2 +- internal/proxy/search_util.go | 2 +- internal/proxy/segment.go | 2 +- internal/proxy/segment_test.go | 2 +- internal/proxy/simple_rate_limiter.go | 4 +- internal/proxy/simple_rate_limiter_test.go | 4 +- internal/proxy/task.go | 6 +- internal/proxy/task_database.go | 2 +- internal/proxy/task_database_test.go | 2 +- internal/proxy/task_delete.go | 6 +- internal/proxy/task_delete_test.go | 8 +- internal/proxy/task_flush.go | 2 +- internal/proxy/task_flush_streaming.go | 2 +- internal/proxy/task_index.go | 2 +- internal/proxy/task_index_test.go | 4 +- internal/proxy/task_query.go | 6 +- internal/proxy/task_query_test.go | 4 +- internal/proxy/task_scheduler_test.go | 2 +- internal/proxy/task_search.go | 6 +- internal/proxy/task_search_test.go | 6 +- internal/proxy/task_statistic.go | 6 +- internal/proxy/task_statistic_test.go | 4 +- internal/proxy/task_test.go | 8 +- internal/proxy/timestamp.go | 2 +- internal/proxy/util.go | 6 +- internal/proxy/util_test.go | 6 +- internal/querycoordv2/.mockery.yaml | 2 +- internal/querycoordv2/balance/balance_test.go | 2 +- .../channel_level_score_balancer_test.go | 4 +- .../balance/multi_target_balancer_test.go | 2 +- .../balance/rowcount_based_balancer_test.go | 4 +- .../balance/score_based_balancer_test.go | 4 +- internal/querycoordv2/balance/utils.go | 2 +- .../querycoordv2/checkers/balance_checker.go | 2 +- .../checkers/balance_checker_test.go | 4 +- .../checkers/channel_checker_test.go | 2 +- .../querycoordv2/checkers/controller_test.go | 2 +- .../querycoordv2/checkers/index_checker.go | 6 +- .../checkers/index_checker_test.go | 6 +- .../querycoordv2/checkers/leader_checker.go | 2 +- .../checkers/leader_checker_test.go | 4 +- .../querycoordv2/checkers/segment_checker.go | 4 +- .../checkers/segment_checker_test.go | 2 +- .../querycoordv2/dist/dist_controller_test.go | 2 +- internal/querycoordv2/dist/dist_handler.go | 4 +- .../querycoordv2/dist/dist_handler_test.go | 4 +- internal/querycoordv2/handlers.go | 2 +- internal/querycoordv2/handlers_test.go | 2 +- internal/querycoordv2/job/job_load.go | 2 +- internal/querycoordv2/job/job_release.go | 4 +- internal/querycoordv2/job/job_sync.go | 2 +- internal/querycoordv2/job/job_test.go | 4 +- internal/querycoordv2/job/job_update.go | 2 +- .../querycoordv2/meta/channel_dist_manager.go | 2 +- .../meta/channel_dist_manager_test.go | 4 +- .../querycoordv2/meta/collection_manager.go | 2 +- .../meta/collection_manager_test.go | 2 +- .../querycoordv2/meta/coordinator_broker.go | 8 +- .../meta/coordinator_broker_test.go | 8 +- .../querycoordv2/meta/dist_manager_test.go | 4 +- .../querycoordv2/meta/leader_view_manager.go | 2 +- .../meta/leader_view_manager_test.go | 4 +- internal/querycoordv2/meta/mock_broker.go | 8 +- .../querycoordv2/meta/mock_target_manager.go | 2 +- internal/querycoordv2/meta/replica.go | 2 +- internal/querycoordv2/meta/replica_manager.go | 2 +- .../meta/replica_manager_helper_test.go | 2 +- .../querycoordv2/meta/replica_manager_test.go | 2 +- internal/querycoordv2/meta/replica_test.go | 2 +- internal/querycoordv2/meta/resource_group.go | 2 +- .../querycoordv2/meta/resource_group_test.go | 2 +- .../querycoordv2/meta/resource_manager.go | 2 +- .../querycoordv2/meta/segment_dist_manager.go | 4 +- .../meta/segment_dist_manager_test.go | 4 +- internal/querycoordv2/meta/target.go | 4 +- internal/querycoordv2/meta/target_manager.go | 4 +- .../querycoordv2/meta/target_manager_test.go | 4 +- internal/querycoordv2/mocks/mock_querynode.go | 4 +- internal/querycoordv2/mocks/querynode.go | 2 +- .../observers/collection_observer.go | 4 +- .../observers/collection_observer_test.go | 4 +- .../observers/leader_cache_observer.go | 4 +- .../observers/leader_cache_observer_test.go | 2 +- .../querycoordv2/observers/target_observer.go | 4 +- .../observers/target_observer_test.go | 4 +- internal/querycoordv2/ops_service_test.go | 4 +- internal/querycoordv2/ops_services.go | 2 +- internal/querycoordv2/server.go | 4 +- internal/querycoordv2/server_test.go | 6 +- internal/querycoordv2/services.go | 4 +- internal/querycoordv2/services_test.go | 6 +- internal/querycoordv2/session/cluster.go | 2 +- internal/querycoordv2/session/cluster_test.go | 2 +- internal/querycoordv2/session/mock_cluster.go | 2 +- internal/querycoordv2/task/action.go | 2 +- internal/querycoordv2/task/executor.go | 4 +- internal/querycoordv2/task/scheduler.go | 4 +- internal/querycoordv2/task/task.go | 2 +- internal/querycoordv2/task/task_test.go | 8 +- internal/querycoordv2/task/utils.go | 6 +- internal/querycoordv2/task/utils_test.go | 2 +- internal/querycoordv2/utils/meta_test.go | 2 +- internal/querycoordv2/utils/test.go | 4 +- internal/querycoordv2/utils/types.go | 4 +- internal/querycoordv2/utils/types_test.go | 2 +- internal/querycoordv2/utils/util.go | 4 +- internal/querycoordv2/utils/util_test.go | 4 +- internal/querynodev2/cluster/mock_worker.go | 4 +- internal/querynodev2/cluster/worker.go | 4 +- internal/querynodev2/cluster/worker_test.go | 4 +- internal/querynodev2/delegator/delegator.go | 4 +- .../querynodev2/delegator/delegator_data.go | 8 +- .../delegator/delegator_data_test.go | 10 +- .../querynodev2/delegator/delegator_test.go | 6 +- .../querynodev2/delegator/delta_forward.go | 4 +- .../delegator/delta_forward_test.go | 6 +- .../querynodev2/delegator/distribution.go | 2 +- internal/querynodev2/delegator/idf_oracle.go | 2 +- .../querynodev2/delegator/mock_delegator.go | 4 +- .../querynodev2/delegator/scalar_pruner.go | 2 +- .../querynodev2/delegator/segment_pruner.go | 4 +- .../delegator/segment_pruner_test.go | 2 +- internal/querynodev2/delegator/types.go | 2 +- internal/querynodev2/delegator/util.go | 4 +- internal/querynodev2/handlers.go | 6 +- internal/querynodev2/handlers_test.go | 4 +- internal/querynodev2/local_worker.go | 4 +- internal/querynodev2/local_worker_test.go | 6 +- internal/querynodev2/metrics_info_test.go | 2 +- internal/querynodev2/mock_data.go | 2 +- .../querynodev2/pipeline/filter_node_test.go | 2 +- .../querynodev2/pipeline/insert_node_test.go | 2 +- internal/querynodev2/pipeline/manager_test.go | 2 +- .../querynodev2/pipeline/pipeline_test.go | 2 +- internal/querynodev2/pipeline/type.go | 2 +- internal/querynodev2/segments/collection.go | 4 +- .../querynodev2/segments/count_reducer.go | 4 +- .../segments/count_reducer_test.go | 4 +- .../segments/default_limit_reducer.go | 6 +- .../querynodev2/segments/index_attr_cache.go | 4 +- .../segments/index_attr_cache_test.go | 4 +- .../segments/load_field_data_info.go | 2 +- .../querynodev2/segments/load_index_info.go | 2 +- internal/querynodev2/segments/manager.go | 4 +- internal/querynodev2/segments/manager_test.go | 4 +- .../segments/mock_collection_manager.go | 4 +- internal/querynodev2/segments/mock_loader.go | 4 +- internal/querynodev2/segments/mock_segment.go | 6 +- .../segments/mock_segment_manager.go | 2 +- internal/querynodev2/segments/reducer.go | 6 +- internal/querynodev2/segments/reducer_test.go | 4 +- internal/querynodev2/segments/result.go | 4 +- .../querynodev2/segments/result_sorter.go | 2 +- internal/querynodev2/segments/result_test.go | 4 +- internal/querynodev2/segments/retrieve.go | 6 +- .../querynodev2/segments/retrieve_test.go | 6 +- internal/querynodev2/segments/search_test.go | 4 +- internal/querynodev2/segments/segment.go | 10 +- .../querynodev2/segments/segment_filter.go | 2 +- .../querynodev2/segments/segment_interface.go | 6 +- internal/querynodev2/segments/segment_l0.go | 6 +- .../querynodev2/segments/segment_loader.go | 4 +- .../segments/segment_loader_test.go | 4 +- internal/querynodev2/segments/segment_test.go | 4 +- internal/querynodev2/segments/utils.go | 6 +- internal/querynodev2/segments/utils_test.go | 4 +- internal/querynodev2/server_test.go | 4 +- internal/querynodev2/services.go | 10 +- internal/querynodev2/services_test.go | 8 +- .../querynodev2/tasks/query_stream_task.go | 4 +- internal/querynodev2/tasks/query_task.go | 6 +- internal/querynodev2/tasks/search_task.go | 4 +- internal/rootcoord/alter_collection_task.go | 2 +- internal/rootcoord/alter_database_task.go | 4 +- .../rootcoord/alter_database_task_test.go | 2 +- internal/rootcoord/broker.go | 6 +- internal/rootcoord/broker_test.go | 6 +- internal/rootcoord/constrant_test.go | 2 +- internal/rootcoord/create_collection_task.go | 2 +- .../rootcoord/create_collection_task_test.go | 2 +- internal/rootcoord/create_db_task.go | 2 +- internal/rootcoord/create_partition_task.go | 2 +- .../rootcoord/create_partition_task_test.go | 2 +- internal/rootcoord/describe_db_task.go | 2 +- internal/rootcoord/describe_db_task_test.go | 2 +- internal/rootcoord/drop_collection_task.go | 2 +- internal/rootcoord/drop_partition_task.go | 2 +- internal/rootcoord/expire_cache.go | 2 +- internal/rootcoord/expire_cache_test.go | 2 +- internal/rootcoord/garbage_collector_test.go | 2 +- internal/rootcoord/meta_table.go | 6 +- internal/rootcoord/meta_table_test.go | 4 +- internal/rootcoord/mock_test.go | 12 +- internal/rootcoord/mocks/meta_table.go | 6 +- internal/rootcoord/quota_center.go | 4 +- internal/rootcoord/quota_center_test.go | 2 +- internal/rootcoord/rbac_task.go | 2 +- internal/rootcoord/root_coord.go | 8 +- internal/rootcoord/root_coord_test.go | 8 +- internal/rootcoord/step.go | 2 +- internal/rootcoord/task_test.go | 2 +- internal/rootcoord/timestamp_bench_test.go | 2 +- internal/rootcoord/timeticksync.go | 2 +- internal/rootcoord/timeticksync_test.go | 2 +- internal/storage/binlog_iterator_test.go | 2 +- internal/storage/data_codec.go | 2 +- internal/storage/data_codec_test.go | 2 +- internal/storage/data_sorter_test.go | 2 +- internal/storage/field_value.go | 2 +- internal/storage/print_binlog_test.go | 2 +- internal/storage/utils.go | 2 +- .../client/assignment/assignment_impl.go | 2 +- .../client/assignment/assignment_test.go | 2 +- .../client/assignment/discoverer.go | 2 +- .../client/broadcast/broadcast_impl.go | 4 +- internal/streamingcoord/client/client.go | 2 +- .../server/balancer/balancer_test.go | 2 +- .../server/balancer/channel/manager.go | 2 +- .../server/balancer/channel/manager_test.go | 2 +- .../server/balancer/channel/metrics.go | 2 +- .../server/balancer/channel/pchannel.go | 2 +- .../server/balancer/channel/pchannel_test.go | 2 +- .../server/broadcaster/broadcaster_impl.go | 4 +- .../server/broadcaster/broadcaster_test.go | 4 +- .../streamingcoord/server/broadcaster/task.go | 2 +- internal/streamingcoord/server/server.go | 2 +- .../server/service/assignment.go | 2 +- .../server/service/broadcast.go | 2 +- .../discover/discover_grpc_server_helper.go | 2 +- .../service/discover/discover_server.go | 2 +- .../service/discover/discover_server_test.go | 2 +- .../client/handler/consumer/consumer_impl.go | 2 +- .../client/handler/consumer/consumer_test.go | 4 +- .../client/handler/handler_client.go | 2 +- .../client/handler/handler_client_impl.go | 2 +- .../client/handler/handler_client_test.go | 2 +- .../handler/producer/produce_grpc_client.go | 4 +- .../client/handler/producer/producer_impl.go | 2 +- .../client/handler/producer/producer_test.go | 4 +- .../client/manager/manager_client.go | 2 +- .../client/manager/manager_client_impl.go | 2 +- .../client/manager/manager_test.go | 2 +- .../flusher/flusherimpl/channel_lifetime.go | 2 +- .../flusher/flusherimpl/flusher_impl.go | 2 +- .../flusher/flusherimpl/flusher_impl_test.go | 4 +- internal/streamingnode/server/server.go | 2 +- .../streamingnode/server/service/handler.go | 2 +- .../consumer/consume_grpc_server_helper.go | 2 +- .../handler/consumer/consume_server.go | 4 +- .../handler/consumer/consume_server_test.go | 2 +- .../producer/produce_grpc_server_helper.go | 2 +- .../handler/producer/produce_server.go | 2 +- .../handler/producer/produce_server_test.go | 4 +- .../streamingnode/server/service/manager.go | 2 +- .../server/wal/adaptor/wal_adaptor_test.go | 2 +- .../server/wal/adaptor/wal_test.go | 4 +- .../segment/manager/partition_manager.go | 6 +- .../segment/manager/partition_managers.go | 4 +- .../segment/manager/pchannel_manager.go | 4 +- .../segment/manager/pchannel_manager_test.go | 6 +- .../segment/manager/seal_queue.go | 2 +- .../segment/manager/segment_manager.go | 2 +- .../segment/segment_assign_interceptor.go | 2 +- .../wal/interceptors/segment/stats/stats.go | 2 +- .../wal/interceptors/timetick/ack/ack_test.go | 2 +- .../wal/interceptors/txn/session_test.go | 2 +- .../server/wal/metricsutil/segment.go | 2 +- .../server/walmanager/manager_impl_test.go | 2 +- .../server/walmanager/wal_state_pair_test.go | 2 +- internal/types/types.go | 14 +- internal/util/analyzecgowrapper/analyze.go | 2 +- .../util/componentutil/componentutil_test.go | 2 +- internal/util/exprutil/expr_checker.go | 2 +- internal/util/exprutil/expr_checker_test.go | 2 +- internal/util/funcutil/count_util.go | 4 +- internal/util/funcutil/count_util_test.go | 4 +- internal/util/grpcclient/client_test.go | 2 +- .../util/grpcclient/local_grpc_client_test.go | 2 +- internal/util/idalloc/allocator_test.go | 2 +- internal/util/idalloc/basic_allocator.go | 2 +- internal/util/idalloc/basic_allocator_test.go | 2 +- .../idalloc/test_mock_root_coord_client.go | 2 +- .../util/importutilv2/binlog/l0_reader.go | 2 +- .../importutilv2/binlog/l0_reader_test.go | 2 +- internal/util/importutilv2/reader.go | 2 +- internal/util/importutilv2/util.go | 2 +- .../util/indexcgowrapper/build_index_info.go | 4 +- .../util/indexcgowrapper/codec_index_test.go | 2 +- internal/util/indexcgowrapper/index.go | 2 +- .../indexparamcheck/vector_index_checker.go | 2 +- internal/util/metrics/utils.go | 2 +- internal/util/mock/grpc_datanode_client.go | 4 +- internal/util/mock/grpc_querycoord_client.go | 4 +- internal/util/mock/grpc_querynode_client.go | 4 +- internal/util/mock/grpc_rootcoord_client.go | 8 +- .../proxyutil/mock_proxy_client_manager.go | 2 +- .../util/proxyutil/proxy_client_manager.go | 2 +- .../proxyutil/proxy_client_manager_test.go | 2 +- internal/util/quota/quota_constant.go | 2 +- internal/util/quota/quota_constant_test.go | 2 +- .../util/ratelimitutil/rate_limiter_tree.go | 4 +- .../ratelimitutil/rate_limiter_tree_test.go | 4 +- .../util/searchutil/optimizers/query_hook.go | 4 +- .../searchutil/optimizers/query_hook_test.go | 6 +- .../searchutil/scheduler/mock_task_test.go | 2 +- internal/util/searchutil/scheduler/tasks.go | 2 +- internal/util/segcore/collection.go | 2 +- internal/util/segcore/plan.go | 2 +- internal/util/segcore/plan_test.go | 6 +- internal/util/segcore/reduce_test.go | 6 +- internal/util/segcore/requests.go | 4 +- internal/util/segcore/requests_test.go | 2 +- internal/util/segcore/responses.go | 2 +- internal/util/segcore/segment_test.go | 4 +- internal/util/segmentutil/utils.go | 2 +- .../service/contextutil/create_consumer.go | 2 +- .../contextutil/create_consumer_test.go | 2 +- .../service/contextutil/create_producer.go | 2 +- .../contextutil/create_producer_test.go | 2 +- .../service/interceptor/client.go | 2 +- .../service/interceptor/server.go | 2 +- .../status/client_stream_wrapper_test.go | 2 +- .../util/streamingutil/status/rpc_error.go | 2 +- .../streamingutil/status/rpc_error_test.go | 2 +- .../streamingutil/status/streaming_error.go | 2 +- .../status/streaming_error_test.go | 2 +- internal/util/streamrpc/streamer.go | 2 +- internal/util/streamrpc/streamer_test.go | 2 +- internal/util/typeutil/hash.go | 2 +- internal/util/typeutil/result_helper_test.go | 4 +- internal/util/typeutil/retrieve_result.go | 4 +- internal/util/wrappers/qn_wrapper.go | 4 +- internal/util/wrappers/qn_wrapper_test.go | 4 +- pkg/.mockery_pkg.yaml | 2 +- ...k_StreamingCoordAssignmentServiceClient.go | 2 +- ...ignmentService_AssignmentDiscoverClient.go | 2 +- ...ignmentService_AssignmentDiscoverServer.go | 2 +- .../mock_StreamingNodeHandlerServiceClient.go | 2 +- ...reamingNodeHandlerService_ConsumeClient.go | 2 +- ...reamingNodeHandlerService_ConsumeServer.go | 2 +- ...reamingNodeHandlerService_ProduceClient.go | 2 +- ...reamingNodeHandlerService_ProduceServer.go | 2 +- .../mock_StreamingNodeManagerServiceClient.go | 2 +- pkg/mq/mqimpl/rocksmq/client/streaming.go | 2 +- {internal => pkg}/proto/OWNERS | 2 +- {internal => pkg}/proto/cgo_msg.proto | 2 +- pkg/proto/cgopb/cgo_msg.pb.go | 302 + {internal => pkg}/proto/clustering.proto | 2 +- pkg/proto/clusteringpb/clustering.pb.go | 741 + {internal => pkg}/proto/data_coord.proto | 2 +- pkg/proto/datapb/data_coord.pb.go | 11904 ++++++++++++++++ pkg/proto/datapb/data_coord_grpc.pb.go | 2697 ++++ {internal => pkg}/proto/etcd_meta.proto | 2 +- pkg/proto/etcdpb/etcd_meta.pb.go | 1429 ++ {internal => pkg}/proto/index_cgo_msg.proto | 2 +- {internal => pkg}/proto/index_coord.proto | 2 +- pkg/proto/indexcgopb/index_cgo_msg.pb.go | 1193 ++ pkg/proto/indexpb/index_coord.pb.go | 3882 +++++ pkg/proto/indexpb/index_coord_grpc.pb.go | 597 + {internal => pkg}/proto/internal.proto | 2 +- pkg/proto/internalpb/internal.pb.go | 4767 +++++++ pkg/{streaming => }/proto/messages.proto | 2 +- pkg/proto/messagespb/messages.pb.go | 2399 ++++ {internal => pkg}/proto/plan.proto | 2 +- pkg/proto/planpb/plan.pb.go | 3375 +++++ {internal => pkg}/proto/proxy.proto | 2 +- pkg/proto/proxypb/proxy.pb.go | 1184 ++ pkg/proto/proxypb/proxy_grpc.pb.go | 593 + {internal => pkg}/proto/query_coord.proto | 2 +- pkg/proto/querypb/query_coord.pb.go | 10390 ++++++++++++++ pkg/proto/querypb/query_coord_grpc.pb.go | 2522 ++++ {internal => pkg}/proto/root_coord.proto | 2 +- pkg/proto/rootcoordpb/root_coord.pb.go | 2093 +++ pkg/proto/rootcoordpb/root_coord_grpc.pb.go | 2209 +++ {internal => pkg}/proto/segcore.proto | 2 +- pkg/proto/segcorepb/segcore.pb.go | 646 + pkg/{streaming => }/proto/streaming.proto | 2 +- .../proto/streamingpb/extends.go | 0 pkg/proto/streamingpb/streaming.pb.go | 4932 +++++++ pkg/proto/streamingpb/streaming_grpc.pb.go | 731 + {internal => pkg}/proto/worker.proto | 2 +- pkg/proto/workerpb/worker.pb.go | 2576 ++++ pkg/proto/workerpb/worker_grpc.pb.go | 482 + pkg/streaming/util/message/builder.go | 2 +- pkg/streaming/util/message/message_impl.go | 2 +- pkg/streaming/util/message/message_type.go | 2 +- .../util/message/specialized_message.go | 2 +- pkg/streaming/util/message/txn.go | 2 +- pkg/streaming/util/message/txn_test.go | 2 +- pkg/streaming/util/options/deliver.go | 4 +- pkg/streaming/util/options/deliver_test.go | 2 +- pkg/streaming/util/types/pchannel_info.go | 2 +- .../util/types/pchannel_info_test.go | 2 +- pkg/streaming/util/types/streaming_node.go | 4 +- pkg/streaming/walimpls/impls/kafka/wal.go | 2 +- pkg/streaming/walimpls/impls/pulsar/wal.go | 2 +- pkg/streaming/walimpls/impls/rmq/wal.go | 2 +- .../walimpls/impls/walimplstest/wal.go | 2 +- scripts/generate_proto.sh | 50 +- tests/integration/balance/balance_test.go | 2 +- .../balance/channel_exclusive_balance_test.go | 2 +- .../channel_balance/channel_balance_test.go | 2 +- .../clustering_compaction_null_data_test.go | 2 +- .../compaction/clustering_compaction_test.go | 2 +- .../compaction/l0_compaction_test.go | 2 +- .../compaction/l2_single_compaction_test.go | 2 +- .../compaction/mix_compaction_test.go | 2 +- .../cross_cluster_routing_test.go | 8 +- tests/integration/datanode/compaction_test.go | 2 +- tests/integration/import/binlog_test.go | 4 +- .../integration/import/dynamic_field_test.go | 2 +- tests/integration/import/import_test.go | 2 +- tests/integration/import/multi_vector_test.go | 2 +- .../integration/import/partition_key_test.go | 2 +- tests/integration/import/util_test.go | 4 +- .../levelzero/delete_on_growing_test.go | 2 +- .../levelzero/delete_partition_key_test.go | 2 +- tests/integration/meta_watcher.go | 4 +- tests/integration/ops/suspend_node_test.go | 2 +- .../partialsearch/partial_search_test.go | 2 +- tests/integration/replicas/load/load_test.go | 2 +- .../manual_rolling_upgrade_test.go | 2 +- .../seal_by_total_growing_test.go | 2 +- .../integration/stats_task/stats_task_test.go | 2 +- .../streaming/hello_streaming_test.go | 2 +- tests/integration/target/target_test.go | 2 +- 748 files changed, 62795 insertions(+), 1166 deletions(-) rename {internal => pkg}/proto/OWNERS (93%) rename {internal => pkg}/proto/cgo_msg.proto (88%) create mode 100644 pkg/proto/cgopb/cgo_msg.pb.go rename {internal => pkg}/proto/clustering.proto (94%) create mode 100644 pkg/proto/clusteringpb/clustering.pb.go rename {internal => pkg}/proto/data_coord.proto (99%) create mode 100644 pkg/proto/datapb/data_coord.pb.go create mode 100644 pkg/proto/datapb/data_coord_grpc.pb.go rename {internal => pkg}/proto/etcd_meta.proto (97%) create mode 100644 pkg/proto/etcdpb/etcd_meta.pb.go rename {internal => pkg}/proto/index_cgo_msg.proto (96%) rename {internal => pkg}/proto/index_coord.proto (99%) create mode 100644 pkg/proto/indexcgopb/index_cgo_msg.pb.go create mode 100644 pkg/proto/indexpb/index_coord.pb.go create mode 100644 pkg/proto/indexpb/index_coord_grpc.pb.go rename {internal => pkg}/proto/internal.proto (99%) create mode 100644 pkg/proto/internalpb/internal.pb.go rename pkg/{streaming => }/proto/messages.proto (98%) create mode 100644 pkg/proto/messagespb/messages.pb.go rename {internal => pkg}/proto/plan.proto (98%) create mode 100644 pkg/proto/planpb/plan.pb.go rename {internal => pkg}/proto/proxy.proto (97%) create mode 100644 pkg/proto/proxypb/proxy.pb.go create mode 100644 pkg/proto/proxypb/proxy_grpc.pb.go rename {internal => pkg}/proto/query_coord.proto (99%) create mode 100644 pkg/proto/querypb/query_coord.pb.go create mode 100644 pkg/proto/querypb/query_coord_grpc.pb.go rename {internal => pkg}/proto/root_coord.proto (99%) create mode 100644 pkg/proto/rootcoordpb/root_coord.pb.go create mode 100644 pkg/proto/rootcoordpb/root_coord_grpc.pb.go rename {internal => pkg}/proto/segcore.proto (92%) create mode 100644 pkg/proto/segcorepb/segcore.pb.go rename pkg/{streaming => }/proto/streaming.proto (99%) rename pkg/{streaming => }/proto/streamingpb/extends.go (100%) create mode 100644 pkg/proto/streamingpb/streaming.pb.go create mode 100644 pkg/proto/streamingpb/streaming_grpc.pb.go rename {internal => pkg}/proto/worker.proto (98%) create mode 100644 pkg/proto/workerpb/worker.pb.go create mode 100644 pkg/proto/workerpb/worker_grpc.pb.go diff --git a/.gitignore b/.gitignore index 724c2ce2ac0c7..b6adfcbdb4b4b 100644 --- a/.gitignore +++ b/.gitignore @@ -103,5 +103,4 @@ cwrapper_rocksdb_build/ internal/proto/**/*.pb.go internal/core/src/pb/*.pb.h internal/core/src/pb/*.pb.cc -*.pb.go **/legacypb/*.pb.go diff --git a/cmd/milvus/mck.go b/cmd/milvus/mck.go index 8bf363a2dbede..b200becd1f2e0 100644 --- a/cmd/milvus/mck.go +++ b/cmd/milvus/mck.go @@ -18,12 +18,12 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" - "github.com/milvus-io/milvus/internal/proto/datapb" - pb "github.com/milvus-io/milvus/internal/proto/etcdpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/kv" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + pb "github.com/milvus-io/milvus/pkg/proto/etcdpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/logutil" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/cmd/tools/datameta/main.go b/cmd/tools/datameta/main.go index 1686fcf1f04a2..4b36a6c4c4395 100644 --- a/cmd/tools/datameta/main.go +++ b/cmd/tools/datameta/main.go @@ -11,8 +11,8 @@ import ( "google.golang.org/protobuf/proto" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/tsoutil" ) diff --git a/cmd/tools/migration/backend/etcd210.go b/cmd/tools/migration/backend/etcd210.go index f006789e28737..88fea0ca8ad76 100644 --- a/cmd/tools/migration/backend/etcd210.go +++ b/cmd/tools/migration/backend/etcd210.go @@ -19,9 +19,9 @@ import ( "github.com/milvus-io/milvus/cmd/tools/migration/versions" "github.com/milvus-io/milvus/internal/metastore/kv/rootcoord" "github.com/milvus-io/milvus/internal/metastore/model" - pb "github.com/milvus-io/milvus/internal/proto/etcdpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/storage" + pb "github.com/milvus-io/milvus/pkg/proto/etcdpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/cmd/tools/migration/meta/210_to_220.go b/cmd/tools/migration/meta/210_to_220.go index 5f15cab2210a1..14b59a992a21e 100644 --- a/cmd/tools/migration/meta/210_to_220.go +++ b/cmd/tools/migration/meta/210_to_220.go @@ -13,10 +13,10 @@ import ( "github.com/milvus-io/milvus/cmd/tools/migration/legacy/legacypb" "github.com/milvus-io/milvus/cmd/tools/migration/versions" "github.com/milvus-io/milvus/internal/metastore/model" - pb "github.com/milvus-io/milvus/internal/proto/etcdpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + pb "github.com/milvus-io/milvus/pkg/proto/etcdpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/cmd/tools/migration/meta/meta210.go b/cmd/tools/migration/meta/meta210.go index 06cba643d85f1..2c735898c8a02 100644 --- a/cmd/tools/migration/meta/meta210.go +++ b/cmd/tools/migration/meta/meta210.go @@ -10,7 +10,7 @@ import ( "github.com/milvus-io/milvus/cmd/tools/migration/legacy/legacypb" "github.com/milvus-io/milvus/internal/metastore/kv/rootcoord" "github.com/milvus-io/milvus/internal/metastore/model" - pb "github.com/milvus-io/milvus/internal/proto/etcdpb" + pb "github.com/milvus-io/milvus/pkg/proto/etcdpb" "github.com/milvus-io/milvus/pkg/util" ) diff --git a/cmd/tools/migration/meta/meta220.go b/cmd/tools/migration/meta/meta220.go index 684f773ea2db2..8616dc151c723 100644 --- a/cmd/tools/migration/meta/meta220.go +++ b/cmd/tools/migration/meta/meta220.go @@ -9,7 +9,7 @@ import ( "github.com/milvus-io/milvus/internal/metastore/kv/querycoord" "github.com/milvus-io/milvus/internal/metastore/kv/rootcoord" "github.com/milvus-io/milvus/internal/metastore/model" - "github.com/milvus-io/milvus/internal/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util" ) diff --git a/internal/allocator/id_allocator.go b/internal/allocator/id_allocator.go index 5cc5211b464fa..3ad76b1dc3c36 100644 --- a/internal/allocator/id_allocator.go +++ b/internal/allocator/id_allocator.go @@ -24,7 +24,7 @@ import ( "github.com/cockroachdb/errors" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" ) diff --git a/internal/allocator/id_allocator_test.go b/internal/allocator/id_allocator_test.go index 3d83f42944bc3..67c0af28e4ea0 100644 --- a/internal/allocator/id_allocator_test.go +++ b/internal/allocator/id_allocator_test.go @@ -24,7 +24,7 @@ import ( "github.com/stretchr/testify/require" "google.golang.org/grpc" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/allocator/remote_interface.go b/internal/allocator/remote_interface.go index 2f70ef55cd9ab..c5ab4fb0ef1e2 100644 --- a/internal/allocator/remote_interface.go +++ b/internal/allocator/remote_interface.go @@ -21,7 +21,7 @@ import ( "google.golang.org/grpc" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" ) type remoteInterface interface { diff --git a/internal/coordinator/coordclient/registry.go b/internal/coordinator/coordclient/registry.go index 12c231ae9cda3..efb0dbed2122e 100644 --- a/internal/coordinator/coordclient/registry.go +++ b/internal/coordinator/coordclient/registry.go @@ -9,12 +9,12 @@ import ( dcc "github.com/milvus-io/milvus/internal/distributed/datacoord/client" qcc "github.com/milvus-io/milvus/internal/distributed/querycoord/client" rcc "github.com/milvus-io/milvus/internal/distributed/rootcoord/client" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/grpcclient" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/syncutil" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/coordinator/coordclient/registry_test.go b/internal/coordinator/coordclient/registry_test.go index 6752b637cc4f4..df36ba8c25ae2 100644 --- a/internal/coordinator/coordclient/registry_test.go +++ b/internal/coordinator/coordclient/registry_test.go @@ -6,9 +6,9 @@ import ( "github.com/stretchr/testify/assert" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/datacoord/allocator/allocator.go b/internal/datacoord/allocator/allocator.go index d2c8cbe4bf76f..e816a3f295a0b 100644 --- a/internal/datacoord/allocator/allocator.go +++ b/internal/datacoord/allocator/allocator.go @@ -21,8 +21,8 @@ import ( "time" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/datacoord/allocator/allocator_test.go b/internal/datacoord/allocator/allocator_test.go index 2de4e742f40c0..6f2d6cd432a22 100644 --- a/internal/datacoord/allocator/allocator_test.go +++ b/internal/datacoord/allocator/allocator_test.go @@ -27,7 +27,7 @@ import ( "google.golang.org/grpc" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/datacoord/analyze_meta.go b/internal/datacoord/analyze_meta.go index 14b97535d0c74..bca8b6e4b443e 100644 --- a/internal/datacoord/analyze_meta.go +++ b/internal/datacoord/analyze_meta.go @@ -25,9 +25,9 @@ import ( "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus/internal/metastore" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/timerecord" ) diff --git a/internal/datacoord/analyze_meta_test.go b/internal/datacoord/analyze_meta_test.go index 9edd917bc2437..6244e6f3b08cb 100644 --- a/internal/datacoord/analyze_meta_test.go +++ b/internal/datacoord/analyze_meta_test.go @@ -25,8 +25,8 @@ import ( "github.com/stretchr/testify/suite" "github.com/milvus-io/milvus/internal/metastore/mocks" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" ) type AnalyzeMetaSuite struct { diff --git a/internal/datacoord/channel.go b/internal/datacoord/channel.go index 4de0ee900245a..91833fa0a73c8 100644 --- a/internal/datacoord/channel.go +++ b/internal/datacoord/channel.go @@ -24,8 +24,8 @@ 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/proto/datapb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" ) type ROChannel interface { diff --git a/internal/datacoord/channel_manager.go b/internal/datacoord/channel_manager.go index 43528122aad7d..b75134489d3a2 100644 --- a/internal/datacoord/channel_manager.go +++ b/internal/datacoord/channel_manager.go @@ -28,10 +28,10 @@ import ( "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus/internal/allocator" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/util/streamingutil" "github.com/milvus-io/milvus/pkg/kv" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/lock" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/datacoord/channel_manager_test.go b/internal/datacoord/channel_manager_test.go index 8cf1f539dc758..4cf58131ca4bb 100644 --- a/internal/datacoord/channel_manager_test.go +++ b/internal/datacoord/channel_manager_test.go @@ -31,9 +31,9 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" globalIDAllocator "github.com/milvus-io/milvus/internal/allocator" kvmock "github.com/milvus-io/milvus/internal/kv/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/kv/predicates" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/datacoord/channel_store.go b/internal/datacoord/channel_store.go index adede46db0a47..d7bf8e1929dfa 100644 --- a/internal/datacoord/channel_store.go +++ b/internal/datacoord/channel_store.go @@ -29,10 +29,10 @@ import ( "go.uber.org/zap/zapcore" "google.golang.org/protobuf/proto" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/kv" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/timerecord" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/datacoord/channel_store_test.go b/internal/datacoord/channel_store_test.go index 02be0e9f9b71b..b5409a04b2562 100644 --- a/internal/datacoord/channel_store_test.go +++ b/internal/datacoord/channel_store_test.go @@ -14,10 +14,10 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/kv/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/kv/predicates" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/testutils" ) diff --git a/internal/datacoord/cluster.go b/internal/datacoord/cluster.go index 6927648ed0593..8b42e4c158099 100644 --- a/internal/datacoord/cluster.go +++ b/internal/datacoord/cluster.go @@ -26,8 +26,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus/internal/datacoord/session" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/datacoord/cluster_test.go b/internal/datacoord/cluster_test.go index 0e3c95aa7fea0..2e4bbd66e32ae 100644 --- a/internal/datacoord/cluster_test.go +++ b/internal/datacoord/cluster_test.go @@ -29,8 +29,8 @@ import ( "github.com/milvus-io/milvus/internal/datacoord/session" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/kv/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/kv" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/testutils" ) diff --git a/internal/datacoord/compaction.go b/internal/datacoord/compaction.go index 0fe51c6496efe..bea8a6a29d224 100644 --- a/internal/datacoord/compaction.go +++ b/internal/datacoord/compaction.go @@ -31,9 +31,9 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus/internal/datacoord/allocator" "github.com/milvus-io/milvus/internal/datacoord/session" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/lock" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/datacoord/compaction_l0_view_test.go b/internal/datacoord/compaction_l0_view_test.go index 5fa941397b483..78d21e5e38f6f 100644 --- a/internal/datacoord/compaction_l0_view_test.go +++ b/internal/datacoord/compaction_l0_view_test.go @@ -9,8 +9,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" ) func TestLevelZeroSegmentsViewSuite(t *testing.T) { diff --git a/internal/datacoord/compaction_policy_clustering.go b/internal/datacoord/compaction_policy_clustering.go index 1e7d652c9b791..9e206f4c76b41 100644 --- a/internal/datacoord/compaction_policy_clustering.go +++ b/internal/datacoord/compaction_policy_clustering.go @@ -27,9 +27,9 @@ 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/datacoord/allocator" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/util/clustering" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/datacoord/compaction_policy_clustering_test.go b/internal/datacoord/compaction_policy_clustering_test.go index 126f5e32240d5..c43c69afb990a 100644 --- a/internal/datacoord/compaction_policy_clustering_test.go +++ b/internal/datacoord/compaction_policy_clustering_test.go @@ -29,8 +29,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/datacoord/allocator" "github.com/milvus-io/milvus/internal/metastore/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/datacoord/compaction_policy_l0.go b/internal/datacoord/compaction_policy_l0.go index 59a8acadb1de9..27df4ca29e1cb 100644 --- a/internal/datacoord/compaction_policy_l0.go +++ b/internal/datacoord/compaction_policy_l0.go @@ -5,8 +5,8 @@ import ( "go.uber.org/atomic" "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" ) type l0CompactionPolicy struct { diff --git a/internal/datacoord/compaction_policy_l0_test.go b/internal/datacoord/compaction_policy_l0_test.go index 3760442bd12fc..760ee5771ad1d 100644 --- a/internal/datacoord/compaction_policy_l0_test.go +++ b/internal/datacoord/compaction_policy_l0_test.go @@ -24,8 +24,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/allocator" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" ) func TestL0CompactionPolicySuite(t *testing.T) { diff --git a/internal/datacoord/compaction_policy_single.go b/internal/datacoord/compaction_policy_single.go index 6fabd2f610520..76ed80b2ea3a4 100644 --- a/internal/datacoord/compaction_policy_single.go +++ b/internal/datacoord/compaction_policy_single.go @@ -25,8 +25,8 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus/internal/datacoord/allocator" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" ) // singleCompactionPolicy is to compact one segment with too many delta logs diff --git a/internal/datacoord/compaction_policy_single_test.go b/internal/datacoord/compaction_policy_single_test.go index bc471167cc18f..66dc4f3bab8fa 100644 --- a/internal/datacoord/compaction_policy_single_test.go +++ b/internal/datacoord/compaction_policy_single_test.go @@ -25,7 +25,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus/internal/datacoord/allocator" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/datacoord/compaction_queue.go b/internal/datacoord/compaction_queue.go index 7685bdb2fe45c..e3b06790a1697 100644 --- a/internal/datacoord/compaction_queue.go +++ b/internal/datacoord/compaction_queue.go @@ -22,7 +22,7 @@ import ( "github.com/cockroachdb/errors" "github.com/samber/lo" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/lock" ) diff --git a/internal/datacoord/compaction_queue_test.go b/internal/datacoord/compaction_queue_test.go index 21bba5e8143c2..8dfb50b2a8dad 100644 --- a/internal/datacoord/compaction_queue_test.go +++ b/internal/datacoord/compaction_queue_test.go @@ -22,7 +22,7 @@ import ( "github.com/stretchr/testify/assert" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" ) func TestCompactionQueue(t *testing.T) { diff --git a/internal/datacoord/compaction_task.go b/internal/datacoord/compaction_task.go index 4f0bd375a180b..cbfbed104a2a5 100644 --- a/internal/datacoord/compaction_task.go +++ b/internal/datacoord/compaction_task.go @@ -19,7 +19,7 @@ package datacoord import ( "go.opentelemetry.io/otel/trace" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" ) type CompactionTask interface { diff --git a/internal/datacoord/compaction_task_clustering.go b/internal/datacoord/compaction_task_clustering.go index 4f939fefbf043..f7ed881d57324 100644 --- a/internal/datacoord/compaction_task_clustering.go +++ b/internal/datacoord/compaction_task_clustering.go @@ -33,12 +33,12 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus/internal/datacoord/allocator" "github.com/milvus-io/milvus/internal/datacoord/session" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metautil" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/datacoord/compaction_task_clustering_test.go b/internal/datacoord/compaction_task_clustering_test.go index 4fdc8cb83e09a..c6f29c1c3794d 100644 --- a/internal/datacoord/compaction_task_clustering_test.go +++ b/internal/datacoord/compaction_task_clustering_test.go @@ -34,10 +34,10 @@ import ( "github.com/milvus-io/milvus/internal/datacoord/session" "github.com/milvus-io/milvus/internal/metastore/kv/datacoord" "github.com/milvus-io/milvus/internal/metastore/model" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metautil" ) diff --git a/internal/datacoord/compaction_task_l0.go b/internal/datacoord/compaction_task_l0.go index 96bdb207059cd..3b34bf87499e1 100644 --- a/internal/datacoord/compaction_task_l0.go +++ b/internal/datacoord/compaction_task_l0.go @@ -31,9 +31,9 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus/internal/datacoord/allocator" "github.com/milvus-io/milvus/internal/datacoord/session" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/datacoord/compaction_task_l0_test.go b/internal/datacoord/compaction_task_l0_test.go index f6eb442c3edaf..54d0edc7de1a9 100644 --- a/internal/datacoord/compaction_task_l0_test.go +++ b/internal/datacoord/compaction_task_l0_test.go @@ -29,7 +29,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus/internal/datacoord/allocator" "github.com/milvus-io/milvus/internal/datacoord/session" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/datacoord/compaction_task_meta.go b/internal/datacoord/compaction_task_meta.go index be18fd0884fae..b49f74a2db4ec 100644 --- a/internal/datacoord/compaction_task_meta.go +++ b/internal/datacoord/compaction_task_meta.go @@ -29,8 +29,8 @@ import ( "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/metastore" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/timerecord" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/datacoord/compaction_task_meta_test.go b/internal/datacoord/compaction_task_meta_test.go index 22aca345e1794..d25b71b76d4a0 100644 --- a/internal/datacoord/compaction_task_meta_test.go +++ b/internal/datacoord/compaction_task_meta_test.go @@ -26,7 +26,7 @@ import ( "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/metastore/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/metricsinfo" ) diff --git a/internal/datacoord/compaction_task_mix.go b/internal/datacoord/compaction_task_mix.go index 90b43432e065d..cfcca52754d75 100644 --- a/internal/datacoord/compaction_task_mix.go +++ b/internal/datacoord/compaction_task_mix.go @@ -14,8 +14,8 @@ import ( "github.com/milvus-io/milvus/internal/datacoord/allocator" "github.com/milvus-io/milvus/internal/datacoord/session" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/datacoord/compaction_task_mix_test.go b/internal/datacoord/compaction_task_mix_test.go index b8fb918e3c66f..7143dab41c9f5 100644 --- a/internal/datacoord/compaction_task_mix_test.go +++ b/internal/datacoord/compaction_task_mix_test.go @@ -11,7 +11,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus/internal/datacoord/allocator" "github.com/milvus-io/milvus/internal/datacoord/session" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/datacoord/compaction_test.go b/internal/datacoord/compaction_test.go index d5053eb7f7327..31800c712162a 100644 --- a/internal/datacoord/compaction_test.go +++ b/internal/datacoord/compaction_test.go @@ -31,7 +31,7 @@ import ( "github.com/milvus-io/milvus/internal/datacoord/session" "github.com/milvus-io/milvus/internal/metastore/kv/binlog" "github.com/milvus-io/milvus/internal/metastore/kv/datacoord" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/metautil" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/datacoord/compaction_trigger.go b/internal/datacoord/compaction_trigger.go index df86450d31d55..c49ce0f9a7c48 100644 --- a/internal/datacoord/compaction_trigger.go +++ b/internal/datacoord/compaction_trigger.go @@ -29,8 +29,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/datacoord/allocator" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/lifetime" "github.com/milvus-io/milvus/pkg/util/lock" "github.com/milvus-io/milvus/pkg/util/logutil" diff --git a/internal/datacoord/compaction_trigger_test.go b/internal/datacoord/compaction_trigger_test.go index b1b912be218ef..53ecf91b5a5fc 100644 --- a/internal/datacoord/compaction_trigger_test.go +++ b/internal/datacoord/compaction_trigger_test.go @@ -36,9 +36,9 @@ import ( "github.com/milvus-io/milvus/internal/datacoord/allocator" "github.com/milvus-io/milvus/internal/metastore/mocks" "github.com/milvus-io/milvus/internal/metastore/model" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/lifetime" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/tsoutil" diff --git a/internal/datacoord/compaction_trigger_v2.go b/internal/datacoord/compaction_trigger_v2.go index e7ee53f172087..bea3b137e2a15 100644 --- a/internal/datacoord/compaction_trigger_v2.go +++ b/internal/datacoord/compaction_trigger_v2.go @@ -25,8 +25,8 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus/internal/datacoord/allocator" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/lock" "github.com/milvus-io/milvus/pkg/util/logutil" ) diff --git a/internal/datacoord/compaction_trigger_v2_test.go b/internal/datacoord/compaction_trigger_v2_test.go index 854d560703e66..8a7d99a424144 100644 --- a/internal/datacoord/compaction_trigger_v2_test.go +++ b/internal/datacoord/compaction_trigger_v2_test.go @@ -14,9 +14,9 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/datacoord/allocator" "github.com/milvus-io/milvus/internal/metastore/model" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/datacoord/compaction_view.go b/internal/datacoord/compaction_view.go index e05eef1f09626..667f474bf38bc 100644 --- a/internal/datacoord/compaction_view.go +++ b/internal/datacoord/compaction_view.go @@ -23,7 +23,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" ) type CompactionView interface { diff --git a/internal/datacoord/garbage_collector.go b/internal/datacoord/garbage_collector.go index f24a67cbbcc4b..f99ba6096c449 100644 --- a/internal/datacoord/garbage_collector.go +++ b/internal/datacoord/garbage_collector.go @@ -32,11 +32,11 @@ import ( "github.com/milvus-io/milvus/internal/datacoord/broker" "github.com/milvus-io/milvus/internal/metastore/kv/binlog" "github.com/milvus-io/milvus/internal/metastore/model" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/datacoord/garbage_collector_test.go b/internal/datacoord/garbage_collector_test.go index c2527c0e3b87f..5ac9f666c7c05 100644 --- a/internal/datacoord/garbage_collector_test.go +++ b/internal/datacoord/garbage_collector_test.go @@ -46,10 +46,10 @@ import ( catalogmocks "github.com/milvus-io/milvus/internal/metastore/mocks" "github.com/milvus-io/milvus/internal/metastore/model" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/lock" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/datacoord/handler.go b/internal/datacoord/handler.go index a16824790627d..322da641f4fdc 100644 --- a/internal/datacoord/handler.go +++ b/internal/datacoord/handler.go @@ -25,9 +25,9 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/retry" "github.com/milvus-io/milvus/pkg/util/tsoutil" diff --git a/internal/datacoord/handler_test.go b/internal/datacoord/handler_test.go index 9774ddf951252..81643636435f0 100644 --- a/internal/datacoord/handler_test.go +++ b/internal/datacoord/handler_test.go @@ -14,10 +14,10 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/metastore/model" mocks2 "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/tsoutil" diff --git a/internal/datacoord/import_checker.go b/internal/datacoord/import_checker.go index d14ea3f451196..d43a3804e4b1c 100644 --- a/internal/datacoord/import_checker.go +++ b/internal/datacoord/import_checker.go @@ -27,12 +27,12 @@ import ( "github.com/milvus-io/milvus/internal/datacoord/allocator" "github.com/milvus-io/milvus/internal/datacoord/broker" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/util/importutilv2" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/tsoutil" ) diff --git a/internal/datacoord/import_checker_test.go b/internal/datacoord/import_checker_test.go index 3feea4e13fedf..06d54eb579e7a 100644 --- a/internal/datacoord/import_checker_test.go +++ b/internal/datacoord/import_checker_test.go @@ -31,9 +31,9 @@ import ( "github.com/milvus-io/milvus/internal/datacoord/allocator" broker2 "github.com/milvus-io/milvus/internal/datacoord/broker" "github.com/milvus-io/milvus/internal/metastore/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/timerecord" "github.com/milvus-io/milvus/pkg/util/tsoutil" ) diff --git a/internal/datacoord/import_job.go b/internal/datacoord/import_job.go index 08f5503d6875e..e9826b6f0d34a 100644 --- a/internal/datacoord/import_job.go +++ b/internal/datacoord/import_job.go @@ -24,9 +24,9 @@ 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/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/timerecord" "github.com/milvus-io/milvus/pkg/util/tsoutil" ) diff --git a/internal/datacoord/import_meta_test.go b/internal/datacoord/import_meta_test.go index a12697aa29114..8f59624e7ea59 100644 --- a/internal/datacoord/import_meta_test.go +++ b/internal/datacoord/import_meta_test.go @@ -28,8 +28,8 @@ import ( "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/metastore/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/metricsinfo" ) diff --git a/internal/datacoord/import_scheduler.go b/internal/datacoord/import_scheduler.go index e5b7585e2fb82..24a0cdf7e01ae 100644 --- a/internal/datacoord/import_scheduler.go +++ b/internal/datacoord/import_scheduler.go @@ -30,10 +30,10 @@ import ( "github.com/milvus-io/milvus/internal/datacoord/allocator" "github.com/milvus-io/milvus/internal/datacoord/session" "github.com/milvus-io/milvus/internal/metastore/kv/binlog" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/lock" ) diff --git a/internal/datacoord/import_scheduler_test.go b/internal/datacoord/import_scheduler_test.go index d6f6ad3c4debc..e52be9e63c295 100644 --- a/internal/datacoord/import_scheduler_test.go +++ b/internal/datacoord/import_scheduler_test.go @@ -30,7 +30,7 @@ import ( "github.com/milvus-io/milvus/internal/datacoord/allocator" "github.com/milvus-io/milvus/internal/datacoord/session" "github.com/milvus-io/milvus/internal/metastore/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/timerecord" ) diff --git a/internal/datacoord/import_task.go b/internal/datacoord/import_task.go index 92021e233091f..e4a1a0b67cd11 100644 --- a/internal/datacoord/import_task.go +++ b/internal/datacoord/import_task.go @@ -20,7 +20,7 @@ import ( "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus/internal/json" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/datacoord/import_util.go b/internal/datacoord/import_util.go index 27aca3085f3e2..25ee27865452e 100644 --- a/internal/datacoord/import_util.go +++ b/internal/datacoord/import_util.go @@ -31,12 +31,12 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/datacoord/allocator" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/importutilv2" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/timerecord" diff --git a/internal/datacoord/import_util_test.go b/internal/datacoord/import_util_test.go index ddfc21e19ef1e..1d9729e0aa89f 100644 --- a/internal/datacoord/import_util_test.go +++ b/internal/datacoord/import_util_test.go @@ -35,11 +35,11 @@ import ( "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/metastore/mocks" mocks2 "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/importutilv2" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/datacoord/index_meta.go b/internal/datacoord/index_meta.go index 37e744bd8a4d5..f8bc0e0d70bc7 100644 --- a/internal/datacoord/index_meta.go +++ b/internal/datacoord/index_meta.go @@ -35,13 +35,13 @@ import ( "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/metastore" "github.com/milvus-io/milvus/internal/metastore/model" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/internal/util/indexparamcheck" "github.com/milvus-io/milvus/internal/util/vecindexmgr" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/indexparams" "github.com/milvus-io/milvus/pkg/util/metricsinfo" diff --git a/internal/datacoord/index_meta_test.go b/internal/datacoord/index_meta_test.go index 8e4ddfccacf76..fd3f76db286eb 100644 --- a/internal/datacoord/index_meta_test.go +++ b/internal/datacoord/index_meta_test.go @@ -34,9 +34,9 @@ import ( "github.com/milvus-io/milvus/internal/metastore/kv/datacoord" catalogmocks "github.com/milvus-io/milvus/internal/metastore/mocks" "github.com/milvus-io/milvus/internal/metastore/model" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/metricsinfo" ) diff --git a/internal/datacoord/index_service.go b/internal/datacoord/index_service.go index 62a51200c812e..37432322bda59 100644 --- a/internal/datacoord/index_service.go +++ b/internal/datacoord/index_service.go @@ -26,12 +26,12 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus/internal/metastore/model" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/util/indexparamcheck" "github.com/milvus-io/milvus/internal/util/vecindexmgr" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metautil" diff --git a/internal/datacoord/index_service_test.go b/internal/datacoord/index_service_test.go index 4923f576da968..e120483065ab9 100644 --- a/internal/datacoord/index_service_test.go +++ b/internal/datacoord/index_service_test.go @@ -38,13 +38,13 @@ import ( catalogmocks "github.com/milvus-io/milvus/internal/metastore/mocks" "github.com/milvus-io/milvus/internal/metastore/model" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/indexparamcheck" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/datacoord/job_manager.go b/internal/datacoord/job_manager.go index 9db974a9e1105..96c910c49b9b5 100644 --- a/internal/datacoord/job_manager.go +++ b/internal/datacoord/job_manager.go @@ -9,9 +9,9 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus/internal/datacoord/allocator" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/datacoord/job_manager_test.go b/internal/datacoord/job_manager_test.go index a0d95e4cd5b3d..0a94a76e05080 100644 --- a/internal/datacoord/job_manager_test.go +++ b/internal/datacoord/job_manager_test.go @@ -14,8 +14,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/datacoord/allocator" "github.com/milvus-io/milvus/internal/metastore/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" ) type jobManagerSuite struct { diff --git a/internal/datacoord/meta.go b/internal/datacoord/meta.go index aab1f9a41e78e..c897b0c8b8a70 100644 --- a/internal/datacoord/meta.go +++ b/internal/datacoord/meta.go @@ -36,13 +36,13 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/datacoord/broker" "github.com/milvus-io/milvus/internal/metastore" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/segmentutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/lock" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/datacoord/meta_test.go b/internal/datacoord/meta_test.go index 8872e2ff839cc..41f137d8142a4 100644 --- a/internal/datacoord/meta_test.go +++ b/internal/datacoord/meta_test.go @@ -39,10 +39,10 @@ import ( mocks2 "github.com/milvus-io/milvus/internal/metastore/mocks" "github.com/milvus-io/milvus/internal/metastore/model" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/kv" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" diff --git a/internal/datacoord/meta_util.go b/internal/datacoord/meta_util.go index 85a342cfe2491..036d9856086a5 100644 --- a/internal/datacoord/meta_util.go +++ b/internal/datacoord/meta_util.go @@ -16,7 +16,7 @@ package datacoord -import "github.com/milvus-io/milvus/internal/proto/datapb" +import "github.com/milvus-io/milvus/pkg/proto/datapb" // reviseVChannelInfo will revise the datapb.VchannelInfo for upgrade compatibility from 2.0.2 func reviseVChannelInfo(vChannel *datapb.VchannelInfo) { diff --git a/internal/datacoord/metrics_info.go b/internal/datacoord/metrics_info.go index 394d50b89d7f2..209f101d57740 100644 --- a/internal/datacoord/metrics_info.go +++ b/internal/datacoord/metrics_info.go @@ -30,11 +30,11 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus/internal/datacoord/session" "github.com/milvus-io/milvus/internal/json" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/metrics" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" "github.com/milvus-io/milvus/pkg/util/hardware" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" diff --git a/internal/datacoord/metrics_info_test.go b/internal/datacoord/metrics_info_test.go index c5d7c5250ca60..2438df1cd240d 100644 --- a/internal/datacoord/metrics_info_test.go +++ b/internal/datacoord/metrics_info_test.go @@ -31,8 +31,8 @@ import ( "github.com/milvus-io/milvus/internal/datacoord/session" "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/metastore/model" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/datacoord/mock_channelmanager.go b/internal/datacoord/mock_channelmanager.go index fccecee419de2..bde3063473df4 100644 --- a/internal/datacoord/mock_channelmanager.go +++ b/internal/datacoord/mock_channelmanager.go @@ -5,7 +5,7 @@ package datacoord import ( context "context" - datapb "github.com/milvus-io/milvus/internal/proto/datapb" + datapb "github.com/milvus-io/milvus/pkg/proto/datapb" mock "github.com/stretchr/testify/mock" ) diff --git a/internal/datacoord/mock_cluster.go b/internal/datacoord/mock_cluster.go index 62c60249c5aaf..19f67df9f9b56 100644 --- a/internal/datacoord/mock_cluster.go +++ b/internal/datacoord/mock_cluster.go @@ -5,7 +5,7 @@ package datacoord import ( context "context" - datapb "github.com/milvus-io/milvus/internal/proto/datapb" + datapb "github.com/milvus-io/milvus/pkg/proto/datapb" mock "github.com/stretchr/testify/mock" session "github.com/milvus-io/milvus/internal/datacoord/session" diff --git a/internal/datacoord/mock_compaction_meta.go b/internal/datacoord/mock_compaction_meta.go index 9ca9de7fab1e7..a809b5022b863 100644 --- a/internal/datacoord/mock_compaction_meta.go +++ b/internal/datacoord/mock_compaction_meta.go @@ -5,7 +5,7 @@ package datacoord import ( context "context" - datapb "github.com/milvus-io/milvus/internal/proto/datapb" + datapb "github.com/milvus-io/milvus/pkg/proto/datapb" mock "github.com/stretchr/testify/mock" ) diff --git a/internal/datacoord/mock_compaction_plan_context.go b/internal/datacoord/mock_compaction_plan_context.go index 94856fc34f3bd..fc6b8ce246411 100644 --- a/internal/datacoord/mock_compaction_plan_context.go +++ b/internal/datacoord/mock_compaction_plan_context.go @@ -5,7 +5,7 @@ package datacoord import ( context "context" - datapb "github.com/milvus-io/milvus/internal/proto/datapb" + datapb "github.com/milvus-io/milvus/pkg/proto/datapb" mock "github.com/stretchr/testify/mock" ) diff --git a/internal/datacoord/mock_handler.go b/internal/datacoord/mock_handler.go index 7c3ec969e8a48..264448b8f4cee 100644 --- a/internal/datacoord/mock_handler.go +++ b/internal/datacoord/mock_handler.go @@ -5,7 +5,7 @@ package datacoord import ( context "context" - datapb "github.com/milvus-io/milvus/internal/proto/datapb" + datapb "github.com/milvus-io/milvus/pkg/proto/datapb" mock "github.com/stretchr/testify/mock" ) diff --git a/internal/datacoord/mock_job_manager.go b/internal/datacoord/mock_job_manager.go index 1e0eb821c770a..be0eac953b921 100644 --- a/internal/datacoord/mock_job_manager.go +++ b/internal/datacoord/mock_job_manager.go @@ -3,7 +3,7 @@ package datacoord import ( - indexpb "github.com/milvus-io/milvus/internal/proto/indexpb" + indexpb "github.com/milvus-io/milvus/pkg/proto/indexpb" mock "github.com/stretchr/testify/mock" ) diff --git a/internal/datacoord/mock_subcluster.go b/internal/datacoord/mock_subcluster.go index 378d8ec6e821f..36b891bbe910f 100644 --- a/internal/datacoord/mock_subcluster.go +++ b/internal/datacoord/mock_subcluster.go @@ -5,7 +5,7 @@ package datacoord import ( context "context" - datapb "github.com/milvus-io/milvus/internal/proto/datapb" + datapb "github.com/milvus-io/milvus/pkg/proto/datapb" mock "github.com/stretchr/testify/mock" ) diff --git a/internal/datacoord/mock_test.go b/internal/datacoord/mock_test.go index cf246ec278192..9630730b88808 100644 --- a/internal/datacoord/mock_test.go +++ b/internal/datacoord/mock_test.go @@ -32,12 +32,12 @@ import ( "github.com/milvus-io/milvus/internal/datacoord/allocator" memkv "github.com/milvus-io/milvus/internal/kv/mem" "github.com/milvus-io/milvus/internal/metastore/kv/datacoord" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/proxypb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/kv" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/proxypb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/tsoutil" diff --git a/internal/datacoord/partition_stats_meta.go b/internal/datacoord/partition_stats_meta.go index 97db8227ecd07..fde6618ade015 100644 --- a/internal/datacoord/partition_stats_meta.go +++ b/internal/datacoord/partition_stats_meta.go @@ -8,8 +8,8 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus/internal/metastore" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/timerecord" ) diff --git a/internal/datacoord/partition_stats_meta_test.go b/internal/datacoord/partition_stats_meta_test.go index 1b27a7bde770f..f0ca84b3ba48a 100644 --- a/internal/datacoord/partition_stats_meta_test.go +++ b/internal/datacoord/partition_stats_meta_test.go @@ -24,7 +24,7 @@ import ( "github.com/stretchr/testify/suite" "github.com/milvus-io/milvus/internal/metastore/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" ) type PartitionStatsMetaSuite struct { diff --git a/internal/datacoord/policy_test.go b/internal/datacoord/policy_test.go index 422e12ccfa527..d3ea88c1bd256 100644 --- a/internal/datacoord/policy_test.go +++ b/internal/datacoord/policy_test.go @@ -25,8 +25,8 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" ) func TestPolicySuite(t *testing.T) { diff --git a/internal/datacoord/segment_allocation_policy.go b/internal/datacoord/segment_allocation_policy.go index 58b1ff5c40ff1..57a11de95607d 100644 --- a/internal/datacoord/segment_allocation_policy.go +++ b/internal/datacoord/segment_allocation_policy.go @@ -27,7 +27,7 @@ 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/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/tsoutil" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/datacoord/segment_allocation_policy_test.go b/internal/datacoord/segment_allocation_policy_test.go index fa803b227ec2c..0e8cef9b1b0c1 100644 --- a/internal/datacoord/segment_allocation_policy_test.go +++ b/internal/datacoord/segment_allocation_policy_test.go @@ -25,8 +25,8 @@ 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/proto/datapb" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/tsoutil" ) diff --git a/internal/datacoord/segment_info.go b/internal/datacoord/segment_info.go index 6d74d1c39cd70..b4cc6e0a7144e 100644 --- a/internal/datacoord/segment_info.go +++ b/internal/datacoord/segment_info.go @@ -26,8 +26,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/datacoord/segment_info_test.go b/internal/datacoord/segment_info_test.go index 51f8b19356008..293a571b5350a 100644 --- a/internal/datacoord/segment_info_test.go +++ b/internal/datacoord/segment_info_test.go @@ -6,7 +6,7 @@ import ( "github.com/samber/lo" "github.com/stretchr/testify/assert" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" ) func TestCompactionTo(t *testing.T) { diff --git a/internal/datacoord/segment_manager.go b/internal/datacoord/segment_manager.go index f4170c6a5215a..f66f5a0b0de5f 100644 --- a/internal/datacoord/segment_manager.go +++ b/internal/datacoord/segment_manager.go @@ -29,8 +29,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus/internal/datacoord/allocator" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/lock" "github.com/milvus-io/milvus/pkg/util/retry" "github.com/milvus-io/milvus/pkg/util/tsoutil" diff --git a/internal/datacoord/segment_manager_test.go b/internal/datacoord/segment_manager_test.go index 3958484969d01..fdeefcc52a429 100644 --- a/internal/datacoord/segment_manager_test.go +++ b/internal/datacoord/segment_manager_test.go @@ -33,7 +33,7 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" mockkv "github.com/milvus-io/milvus/internal/kv/mocks" "github.com/milvus-io/milvus/internal/metastore/kv/datacoord" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/lock" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/datacoord/segment_operator.go b/internal/datacoord/segment_operator.go index 91b4da67ba8ce..a3beba986848e 100644 --- a/internal/datacoord/segment_operator.go +++ b/internal/datacoord/segment_operator.go @@ -16,7 +16,7 @@ package datacoord -import "github.com/milvus-io/milvus/internal/proto/datapb" +import "github.com/milvus-io/milvus/pkg/proto/datapb" // SegmentOperator is function type to update segment info. type SegmentOperator func(segment *SegmentInfo) bool diff --git a/internal/datacoord/segment_operator_test.go b/internal/datacoord/segment_operator_test.go index 7b837f45a2b11..3d7bf4f05558c 100644 --- a/internal/datacoord/segment_operator_test.go +++ b/internal/datacoord/segment_operator_test.go @@ -21,7 +21,7 @@ import ( "github.com/stretchr/testify/suite" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" ) type TestSegmentOperatorSuite struct { diff --git a/internal/datacoord/server.go b/internal/datacoord/server.go index 4ba5d75a03fc6..d2cd6f9180a3c 100644 --- a/internal/datacoord/server.go +++ b/internal/datacoord/server.go @@ -46,7 +46,6 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/kv/tikv" "github.com/milvus-io/milvus/internal/metastore/kv/datacoord" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/dependency" @@ -55,6 +54,7 @@ import ( "github.com/milvus-io/milvus/pkg/kv" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/expr" "github.com/milvus-io/milvus/pkg/util/logutil" diff --git a/internal/datacoord/server_test.go b/internal/datacoord/server_test.go index 83df41f7bb36f..ece0e6539750b 100644 --- a/internal/datacoord/server_test.go +++ b/internal/datacoord/server_test.go @@ -49,14 +49,14 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/metastore/model" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" diff --git a/internal/datacoord/services.go b/internal/datacoord/services.go index 0c3b776ef5094..86c58419734b8 100644 --- a/internal/datacoord/services.go +++ b/internal/datacoord/services.go @@ -32,8 +32,6 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/metastore/kv/binlog" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/componentutil" "github.com/milvus-io/milvus/internal/util/importutilv2" @@ -42,6 +40,8 @@ import ( "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" diff --git a/internal/datacoord/services_test.go b/internal/datacoord/services_test.go index 669d433819f90..3203aa00699d5 100644 --- a/internal/datacoord/services_test.go +++ b/internal/datacoord/services_test.go @@ -25,12 +25,12 @@ import ( "github.com/milvus-io/milvus/internal/metastore/mocks" "github.com/milvus-io/milvus/internal/metastore/model" mocks2 "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metautil" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/datacoord/session/datanode_manager.go b/internal/datacoord/session/datanode_manager.go index b4edb64543259..7dff99a594401 100644 --- a/internal/datacoord/session/datanode_manager.go +++ b/internal/datacoord/session/datanode_manager.go @@ -29,10 +29,10 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" grpcdatanodeclient "github.com/milvus-io/milvus/internal/distributed/datanode/client" "github.com/milvus-io/milvus/internal/metastore/kv/binlog" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/lock" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/datacoord/session/datanode_manager_test.go b/internal/datacoord/session/datanode_manager_test.go index 9586a2cd83b8b..bc2170567df05 100644 --- a/internal/datacoord/session/datanode_manager_test.go +++ b/internal/datacoord/session/datanode_manager_test.go @@ -25,9 +25,9 @@ import ( "github.com/stretchr/testify/suite" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/testutils" diff --git a/internal/datacoord/session/indexnode_manager.go b/internal/datacoord/session/indexnode_manager.go index 5a77f611698d8..65f42828a470f 100644 --- a/internal/datacoord/session/indexnode_manager.go +++ b/internal/datacoord/session/indexnode_manager.go @@ -25,10 +25,10 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" indexnodeclient "github.com/milvus-io/milvus/internal/distributed/indexnode/client" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/lock" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/datacoord/session/indexnode_manager_test.go b/internal/datacoord/session/indexnode_manager_test.go index 25be2669a539f..909a470b5e654 100644 --- a/internal/datacoord/session/indexnode_manager_test.go +++ b/internal/datacoord/session/indexnode_manager_test.go @@ -25,8 +25,8 @@ import ( "github.com/stretchr/testify/mock" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/lock" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/datacoord/session/mock_datanode_manager.go b/internal/datacoord/session/mock_datanode_manager.go index 9bd42f2847052..56ff76589f24a 100644 --- a/internal/datacoord/session/mock_datanode_manager.go +++ b/internal/datacoord/session/mock_datanode_manager.go @@ -5,7 +5,7 @@ package session import ( context "context" - datapb "github.com/milvus-io/milvus/internal/proto/datapb" + datapb "github.com/milvus-io/milvus/pkg/proto/datapb" mock "github.com/stretchr/testify/mock" typeutil "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/datacoord/stats_task_meta.go b/internal/datacoord/stats_task_meta.go index 4734ee45c76b9..0eff0e5e6e005 100644 --- a/internal/datacoord/stats_task_meta.go +++ b/internal/datacoord/stats_task_meta.go @@ -26,10 +26,10 @@ import ( "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus/internal/metastore" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/timerecord" ) diff --git a/internal/datacoord/stats_task_meta_test.go b/internal/datacoord/stats_task_meta_test.go index e4ba92067df67..01f79774186f4 100644 --- a/internal/datacoord/stats_task_meta_test.go +++ b/internal/datacoord/stats_task_meta_test.go @@ -25,9 +25,9 @@ import ( "github.com/stretchr/testify/suite" "github.com/milvus-io/milvus/internal/metastore/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" ) type statsTaskMetaSuite struct { diff --git a/internal/datacoord/sync_segments_scheduler.go b/internal/datacoord/sync_segments_scheduler.go index 7f672e4f1bad0..712b8a334593b 100644 --- a/internal/datacoord/sync_segments_scheduler.go +++ b/internal/datacoord/sync_segments_scheduler.go @@ -25,8 +25,8 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus/internal/datacoord/session" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/logutil" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/datacoord/sync_segments_scheduler_test.go b/internal/datacoord/sync_segments_scheduler_test.go index d9a0043bd2522..9786c9c30c542 100644 --- a/internal/datacoord/sync_segments_scheduler_test.go +++ b/internal/datacoord/sync_segments_scheduler_test.go @@ -28,7 +28,7 @@ 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/datacoord/session" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/lock" ) diff --git a/internal/datacoord/task_analyze.go b/internal/datacoord/task_analyze.go index 9c955df15af88..97082bcaa80dd 100644 --- a/internal/datacoord/task_analyze.go +++ b/internal/datacoord/task_analyze.go @@ -27,11 +27,11 @@ import ( "golang.org/x/exp/slices" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/datacoord/task_index.go b/internal/datacoord/task_index.go index d082d88eac75a..f11356364c419 100644 --- a/internal/datacoord/task_index.go +++ b/internal/datacoord/task_index.go @@ -25,13 +25,13 @@ 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/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/vecindexmgr" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/indexparams" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/datacoord/task_scheduler.go b/internal/datacoord/task_scheduler.go index b306f77461045..e264b80342b8a 100644 --- a/internal/datacoord/task_scheduler.go +++ b/internal/datacoord/task_scheduler.go @@ -27,11 +27,11 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus/internal/datacoord/allocator" "github.com/milvus-io/milvus/internal/datacoord/session" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/lock" ) diff --git a/internal/datacoord/task_scheduler_test.go b/internal/datacoord/task_scheduler_test.go index ddd12afcf5a94..143a932caf343 100644 --- a/internal/datacoord/task_scheduler_test.go +++ b/internal/datacoord/task_scheduler_test.go @@ -35,10 +35,10 @@ import ( catalogmocks "github.com/milvus-io/milvus/internal/metastore/mocks" "github.com/milvus-io/milvus/internal/metastore/model" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/datacoord/task_stats.go b/internal/datacoord/task_stats.go index ca061b1e17da2..8633d214a78f5 100644 --- a/internal/datacoord/task_stats.go +++ b/internal/datacoord/task_stats.go @@ -23,10 +23,10 @@ import ( "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/tsoutil" ) diff --git a/internal/datacoord/task_stats_test.go b/internal/datacoord/task_stats_test.go index 5b0a2a2e1e024..93ba3b1d72244 100644 --- a/internal/datacoord/task_stats_test.go +++ b/internal/datacoord/task_stats_test.go @@ -31,10 +31,10 @@ import ( "github.com/milvus-io/milvus/internal/datacoord/allocator" catalogmocks "github.com/milvus-io/milvus/internal/metastore/mocks" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" ) type statsTaskSuite struct { diff --git a/internal/datacoord/types.go b/internal/datacoord/types.go index db1256015adf1..89215eb3cfcf9 100644 --- a/internal/datacoord/types.go +++ b/internal/datacoord/types.go @@ -20,8 +20,8 @@ import ( "context" "time" - "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/pkg/proto/indexpb" ) type Task interface { diff --git a/internal/datacoord/util.go b/internal/datacoord/util.go index 19b4fa2f68949..d7c504c99c31d 100644 --- a/internal/datacoord/util.go +++ b/internal/datacoord/util.go @@ -28,12 +28,12 @@ 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/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/util/vecindexmgr" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/datacoord/util_test.go b/internal/datacoord/util_test.go index 52a0ccf04e4e6..e48db0f513e2b 100644 --- a/internal/datacoord/util_test.go +++ b/internal/datacoord/util_test.go @@ -25,9 +25,9 @@ import ( "github.com/stretchr/testify/suite" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/tsoutil" ) diff --git a/internal/datanode/allocator/allocator_test.go b/internal/datanode/allocator/allocator_test.go index 63d48690c962a..87844f4a2da77 100644 --- a/internal/datanode/allocator/allocator_test.go +++ b/internal/datanode/allocator/allocator_test.go @@ -24,8 +24,8 @@ import ( "github.com/stretchr/testify/require" "google.golang.org/grpc" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/datanode/channel/channel_manager.go b/internal/datanode/channel/channel_manager.go index d43de2ce7cabd..01f2e5f34739f 100644 --- a/internal/datanode/channel/channel_manager.go +++ b/internal/datanode/channel/channel_manager.go @@ -26,8 +26,8 @@ import ( "github.com/milvus-io/milvus/internal/flushcommon/pipeline" "github.com/milvus-io/milvus/internal/flushcommon/util" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/lifetime" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/datanode/channel/channel_manager_test.go b/internal/datanode/channel/channel_manager_test.go index e4384bf271301..3b2c938b4bb11 100644 --- a/internal/datanode/channel/channel_manager_test.go +++ b/internal/datanode/channel/channel_manager_test.go @@ -34,13 +34,13 @@ import ( "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" util2 "github.com/milvus-io/milvus/internal/flushcommon/util" "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/mq/msgdispatcher" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/datanode/channel/mock_channelmanager.go b/internal/datanode/channel/mock_channelmanager.go index f94f890280b0c..307934a0fc0ac 100644 --- a/internal/datanode/channel/mock_channelmanager.go +++ b/internal/datanode/channel/mock_channelmanager.go @@ -3,7 +3,7 @@ package channel import ( - datapb "github.com/milvus-io/milvus/internal/proto/datapb" + datapb "github.com/milvus-io/milvus/pkg/proto/datapb" mock "github.com/stretchr/testify/mock" ) diff --git a/internal/datanode/compaction/clustering_compactor.go b/internal/datanode/compaction/clustering_compactor.go index e7a618191bc06..94a6004028232 100644 --- a/internal/datanode/compaction/clustering_compactor.go +++ b/internal/datanode/compaction/clustering_compactor.go @@ -41,13 +41,13 @@ import ( "github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/flushcommon/io" "github.com/milvus-io/milvus/internal/metastore/kv/binlog" - "github.com/milvus-io/milvus/internal/proto/clusteringpb" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/etcdpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/clusteringpb" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/etcdpb" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/hardware" diff --git a/internal/datanode/compaction/clustering_compactor_test.go b/internal/datanode/compaction/clustering_compactor_test.go index c1b6a7937b729..1833f878711b7 100644 --- a/internal/datanode/compaction/clustering_compactor_test.go +++ b/internal/datanode/compaction/clustering_compactor_test.go @@ -33,9 +33,9 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/datanode/allocator" "github.com/milvus-io/milvus/internal/flushcommon/io" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/tsoutil" diff --git a/internal/datanode/compaction/compactor.go b/internal/datanode/compaction/compactor.go index 6d929bd30af9a..b04a2afdec4a5 100644 --- a/internal/datanode/compaction/compactor.go +++ b/internal/datanode/compaction/compactor.go @@ -17,7 +17,7 @@ package compaction import ( - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/datanode/compaction/compactor_common.go b/internal/datanode/compaction/compactor_common.go index 2ff9c15b4edfc..0e3499589163a 100644 --- a/internal/datanode/compaction/compactor_common.go +++ b/internal/datanode/compaction/compactor_common.go @@ -28,9 +28,9 @@ import ( "github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/flushcommon/io" "github.com/milvus-io/milvus/internal/metastore/kv/binlog" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/tsoutil" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/datanode/compaction/executor.go b/internal/datanode/compaction/executor.go index 3caa27ef1d561..231ad09101637 100644 --- a/internal/datanode/compaction/executor.go +++ b/internal/datanode/compaction/executor.go @@ -24,8 +24,8 @@ import ( "go.uber.org/zap" "golang.org/x/sync/semaphore" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/datanode/compaction/executor_test.go b/internal/datanode/compaction/executor_test.go index dc491d3afd448..e8f9f32c50a9f 100644 --- a/internal/datanode/compaction/executor_test.go +++ b/internal/datanode/compaction/executor_test.go @@ -24,7 +24,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/datanode/compaction/l0_compactor.go b/internal/datanode/compaction/l0_compactor.go index 6cae672726259..3afc6d963e4b2 100644 --- a/internal/datanode/compaction/l0_compactor.go +++ b/internal/datanode/compaction/l0_compactor.go @@ -32,10 +32,10 @@ import ( "github.com/milvus-io/milvus/internal/flushcommon/io" "github.com/milvus-io/milvus/internal/flushcommon/metacache/pkoracle" "github.com/milvus-io/milvus/internal/metastore/kv/binlog" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/hardware" diff --git a/internal/datanode/compaction/l0_compactor_test.go b/internal/datanode/compaction/l0_compactor_test.go index 98e50b278c4ab..168f5ee089f06 100644 --- a/internal/datanode/compaction/l0_compactor_test.go +++ b/internal/datanode/compaction/l0_compactor_test.go @@ -31,10 +31,10 @@ import ( "github.com/milvus-io/milvus/internal/flushcommon/io" "github.com/milvus-io/milvus/internal/flushcommon/metacache/pkoracle" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/timerecord" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/datanode/compaction/load_stats.go b/internal/datanode/compaction/load_stats.go index f96d2861dda4e..3a58ad79087c3 100644 --- a/internal/datanode/compaction/load_stats.go +++ b/internal/datanode/compaction/load_stats.go @@ -25,9 +25,9 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/datanode/compaction/merge_sort.go b/internal/datanode/compaction/merge_sort.go index c484081d8b7c9..0758d660ec527 100644 --- a/internal/datanode/compaction/merge_sort.go +++ b/internal/datanode/compaction/merge_sort.go @@ -14,10 +14,10 @@ import ( "github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/flushcommon/io" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/timerecord" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/datanode/compaction/mix_compactor.go b/internal/datanode/compaction/mix_compactor.go index f771c40a2b874..1c51db02c2539 100644 --- a/internal/datanode/compaction/mix_compactor.go +++ b/internal/datanode/compaction/mix_compactor.go @@ -32,11 +32,11 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/flushcommon/io" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/timerecord" diff --git a/internal/datanode/compaction/mix_compactor_test.go b/internal/datanode/compaction/mix_compactor_test.go index e94b1629523d9..7c958684121c2 100644 --- a/internal/datanode/compaction/mix_compactor_test.go +++ b/internal/datanode/compaction/mix_compactor_test.go @@ -34,10 +34,10 @@ import ( "github.com/milvus-io/milvus/internal/flushcommon/io" "github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/flushcommon/metacache/pkoracle" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/etcdpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/etcdpb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/tsoutil" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/datanode/compaction/mock_compactor.go b/internal/datanode/compaction/mock_compactor.go index 073a25dac8e25..2ebb0e8e8798b 100644 --- a/internal/datanode/compaction/mock_compactor.go +++ b/internal/datanode/compaction/mock_compactor.go @@ -3,7 +3,7 @@ package compaction import ( - datapb "github.com/milvus-io/milvus/internal/proto/datapb" + datapb "github.com/milvus-io/milvus/pkg/proto/datapb" mock "github.com/stretchr/testify/mock" ) diff --git a/internal/datanode/compaction/segment_writer.go b/internal/datanode/compaction/segment_writer.go index 4cbaf9743f6e7..4a71c94d4ad48 100644 --- a/internal/datanode/compaction/segment_writer.go +++ b/internal/datanode/compaction/segment_writer.go @@ -30,11 +30,11 @@ import ( "github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/flushcommon/io" "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/etcdpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/etcdpb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/datanode/data_node_test.go b/internal/datanode/data_node_test.go index 9994d1e2e92e0..dbe9a702730e9 100644 --- a/internal/datanode/data_node_test.go +++ b/internal/datanode/data_node_test.go @@ -36,13 +36,13 @@ import ( util2 "github.com/milvus-io/milvus/internal/flushcommon/util" "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/mq/msgdispatcher" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/datanode/importv2/hash.go b/internal/datanode/importv2/hash.go index 8ab0834b7bcc9..a4c37f739ec18 100644 --- a/internal/datanode/importv2/hash.go +++ b/internal/datanode/importv2/hash.go @@ -20,8 +20,8 @@ import ( "github.com/samber/lo" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/datanode/importv2/scheduler.go b/internal/datanode/importv2/scheduler.go index e94b2671c4667..1b4a88512aecf 100644 --- a/internal/datanode/importv2/scheduler.go +++ b/internal/datanode/importv2/scheduler.go @@ -24,8 +24,8 @@ import ( "github.com/samber/lo" "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/datanode/importv2/scheduler_test.go b/internal/datanode/importv2/scheduler_test.go index 7752c382187d1..a20c17d077205 100644 --- a/internal/datanode/importv2/scheduler_test.go +++ b/internal/datanode/importv2/scheduler_test.go @@ -34,12 +34,12 @@ import ( "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/importutilv2" "github.com/milvus-io/milvus/internal/util/testutil" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/datanode/importv2/task.go b/internal/datanode/importv2/task.go index 0d7c46e6cc474..df85937c7988c 100644 --- a/internal/datanode/importv2/task.go +++ b/internal/datanode/importv2/task.go @@ -21,7 +21,7 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/conc" ) diff --git a/internal/datanode/importv2/task_import.go b/internal/datanode/importv2/task_import.go index cdd71e53f4f4f..c89c2a13cb270 100644 --- a/internal/datanode/importv2/task_import.go +++ b/internal/datanode/importv2/task_import.go @@ -30,11 +30,11 @@ import ( "github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/importutilv2" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/datanode/importv2/task_l0_import.go b/internal/datanode/importv2/task_l0_import.go index 5be5452a398c1..b9727d5d46da0 100644 --- a/internal/datanode/importv2/task_l0_import.go +++ b/internal/datanode/importv2/task_l0_import.go @@ -31,10 +31,10 @@ import ( "github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/importutilv2/binlog" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/datanode/importv2/task_l0_import_test.go b/internal/datanode/importv2/task_l0_import_test.go index 9c2996eeb1d99..a5793cfedfde5 100644 --- a/internal/datanode/importv2/task_l0_import_test.go +++ b/internal/datanode/importv2/task_l0_import_test.go @@ -29,11 +29,11 @@ import ( "github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/importutilv2" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/datanode/importv2/task_l0_preimport.go b/internal/datanode/importv2/task_l0_preimport.go index 93777c8682e6f..6a7e52ff8e76c 100644 --- a/internal/datanode/importv2/task_l0_preimport.go +++ b/internal/datanode/importv2/task_l0_preimport.go @@ -27,11 +27,11 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/importutilv2/binlog" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/datanode/importv2/task_manager_test.go b/internal/datanode/importv2/task_manager_test.go index 516966289eb66..4c7faa3aa0b40 100644 --- a/internal/datanode/importv2/task_manager_test.go +++ b/internal/datanode/importv2/task_manager_test.go @@ -22,7 +22,7 @@ import ( "github.com/stretchr/testify/assert" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" ) func TestImportManager(t *testing.T) { diff --git a/internal/datanode/importv2/task_preimport.go b/internal/datanode/importv2/task_preimport.go index c507f69ff076f..c36e2093cb887 100644 --- a/internal/datanode/importv2/task_preimport.go +++ b/internal/datanode/importv2/task_preimport.go @@ -28,11 +28,11 @@ 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/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/importutilv2" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/datanode/importv2/util.go b/internal/datanode/importv2/util.go index eb6c592f85b12..488f8d768ae13 100644 --- a/internal/datanode/importv2/util.go +++ b/internal/datanode/importv2/util.go @@ -32,12 +32,12 @@ import ( "github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/flushcommon/metacache/pkoracle" "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/function" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/datanode/importv2/util_test.go b/internal/datanode/importv2/util_test.go index a5b76396e1544..9adcd21929e18 100644 --- a/internal/datanode/importv2/util_test.go +++ b/internal/datanode/importv2/util_test.go @@ -24,9 +24,9 @@ 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/allocator" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/util/testutil" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" ) func Test_AppendSystemFieldsData(t *testing.T) { diff --git a/internal/datanode/iterators/binlog_iterator_test.go b/internal/datanode/iterators/binlog_iterator_test.go index d62fd9cd597dc..da71278972d65 100644 --- a/internal/datanode/iterators/binlog_iterator_test.go +++ b/internal/datanode/iterators/binlog_iterator_test.go @@ -7,9 +7,9 @@ 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/proto/etcdpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/etcdpb" ) func TestInsertBinlogIteratorSuite(t *testing.T) { diff --git a/internal/datanode/services.go b/internal/datanode/services.go index 825f63a14c1ad..db781c2e18ba5 100644 --- a/internal/datanode/services.go +++ b/internal/datanode/services.go @@ -33,13 +33,13 @@ import ( "github.com/milvus-io/milvus/internal/flushcommon/io" "github.com/milvus-io/milvus/internal/flushcommon/metacache/pkoracle" "github.com/milvus-io/milvus/internal/metastore/kv/binlog" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/importutilv2" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/tracer" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/datanode/services_test.go b/internal/datanode/services_test.go index 7fa6cf92f7e12..290efa9f45696 100644 --- a/internal/datanode/services_test.go +++ b/internal/datanode/services_test.go @@ -39,12 +39,12 @@ import ( "github.com/milvus-io/milvus/internal/flushcommon/metacache/pkoracle" "github.com/milvus-io/milvus/internal/flushcommon/pipeline" "github.com/milvus-io/milvus/internal/flushcommon/util" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" diff --git a/internal/datanode/util/meta_util.go b/internal/datanode/util/meta_util.go index 0dbf6a0bb05b5..4dc158d13387b 100644 --- a/internal/datanode/util/meta_util.go +++ b/internal/datanode/util/meta_util.go @@ -18,8 +18,8 @@ package util import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/etcdpb" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/etcdpb" ) // ReviseVChannelInfo will revise the datapb.VchannelInfo for upgrade compatibility from 2.0.2 diff --git a/internal/distributed/connection_manager.go b/internal/distributed/connection_manager.go index 25b5b00d6901c..262eba88fb7a5 100644 --- a/internal/distributed/connection_manager.go +++ b/internal/distributed/connection_manager.go @@ -32,12 +32,12 @@ import ( "google.golang.org/grpc/codes" "google.golang.org/grpc/credentials/insecure" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/tracer" "github.com/milvus-io/milvus/pkg/util/retry" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/distributed/connection_manager_test.go b/internal/distributed/connection_manager_test.go index 68569588c264f..daebf0e23154a 100644 --- a/internal/distributed/connection_manager_test.go +++ b/internal/distributed/connection_manager_test.go @@ -31,12 +31,12 @@ import ( "go.uber.org/zap" "google.golang.org/grpc" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/distributed/datacoord/client/client.go b/internal/distributed/datacoord/client/client.go index c5cb6edc5b029..674d75fc91bc3 100644 --- a/internal/distributed/datacoord/client/client.go +++ b/internal/distributed/datacoord/client/client.go @@ -27,13 +27,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/internal/distributed/utils" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/grpcclient" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/distributed/datacoord/client/client_test.go b/internal/distributed/datacoord/client/client_test.go index c46dab7235126..84c46ca1a43fd 100644 --- a/internal/distributed/datacoord/client/client_test.go +++ b/internal/distributed/datacoord/client/client_test.go @@ -32,10 +32,10 @@ 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/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/distributed/datacoord/service.go b/internal/distributed/datacoord/service.go index eee57efe9423c..947d0f258730d 100644 --- a/internal/distributed/datacoord/service.go +++ b/internal/distributed/datacoord/service.go @@ -35,13 +35,13 @@ import ( "github.com/milvus-io/milvus/internal/coordinator/coordclient" "github.com/milvus-io/milvus/internal/datacoord" "github.com/milvus-io/milvus/internal/distributed/utils" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/dependency" _ "github.com/milvus-io/milvus/internal/util/grpcclient" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/tracer" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/etcd" diff --git a/internal/distributed/datacoord/service_test.go b/internal/distributed/datacoord/service_test.go index fcad74a636b0f..004115a63e932 100644 --- a/internal/distributed/datacoord/service_test.go +++ b/internal/distributed/datacoord/service_test.go @@ -28,9 +28,9 @@ 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/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/tikv" diff --git a/internal/distributed/datanode/client/client.go b/internal/distributed/datanode/client/client.go index aeacb69e7c715..05e09fd5263b5 100644 --- a/internal/distributed/datanode/client/client.go +++ b/internal/distributed/datanode/client/client.go @@ -26,12 +26,12 @@ 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/internal/distributed/utils" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/grpcclient" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/distributed/datanode/client/client_test.go b/internal/distributed/datanode/client/client_test.go index 03e4b64e74e62..412a030cb70aa 100644 --- a/internal/distributed/datanode/client/client_test.go +++ b/internal/distributed/datanode/client/client_test.go @@ -24,8 +24,8 @@ import ( "github.com/stretchr/testify/assert" "google.golang.org/grpc" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/util/mock" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/distributed/datanode/service.go b/internal/distributed/datanode/service.go index f26fb1d63ca47..bd77a592c822b 100644 --- a/internal/distributed/datanode/service.go +++ b/internal/distributed/datanode/service.go @@ -35,13 +35,13 @@ import ( dcc "github.com/milvus-io/milvus/internal/distributed/datacoord/client" rcc "github.com/milvus-io/milvus/internal/distributed/rootcoord/client" "github.com/milvus-io/milvus/internal/distributed/utils" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/componentutil" "github.com/milvus-io/milvus/internal/util/dependency" _ "github.com/milvus-io/milvus/internal/util/grpcclient" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/tracer" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/funcutil" diff --git a/internal/distributed/datanode/service_test.go b/internal/distributed/datanode/service_test.go index 8e0bff6acfcca..4203cce694eb9 100644 --- a/internal/distributed/datanode/service_test.go +++ b/internal/distributed/datanode/service_test.go @@ -28,9 +28,9 @@ 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/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/distributed/indexnode/client/client.go b/internal/distributed/indexnode/client/client.go index 5ae3bd2c19f6c..22ed7b10d4275 100644 --- a/internal/distributed/indexnode/client/client.go +++ b/internal/distributed/indexnode/client/client.go @@ -26,12 +26,12 @@ 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/internal/distributed/utils" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/grpcclient" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/distributed/indexnode/client/client_test.go b/internal/distributed/indexnode/client/client_test.go index afa41c152e832..6747ea55ce5f3 100644 --- a/internal/distributed/indexnode/client/client_test.go +++ b/internal/distributed/indexnode/client/client_test.go @@ -29,9 +29,9 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/distributed/indexnode/service.go b/internal/distributed/indexnode/service.go index 83c67615950a9..0b6c8900b1da0 100644 --- a/internal/distributed/indexnode/service.go +++ b/internal/distributed/indexnode/service.go @@ -33,12 +33,12 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus/internal/distributed/utils" "github.com/milvus-io/milvus/internal/indexnode" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/dependency" _ "github.com/milvus-io/milvus/internal/util/grpcclient" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/tracer" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/funcutil" diff --git a/internal/distributed/indexnode/service_test.go b/internal/distributed/indexnode/service_test.go index 9caea2c2663a8..20a48f7d0ea30 100644 --- a/internal/distributed/indexnode/service_test.go +++ b/internal/distributed/indexnode/service_test.go @@ -26,9 +26,9 @@ 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/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/internal/util/dependency" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/distributed/proxy/client/client.go b/internal/distributed/proxy/client/client.go index c493869aa3984..d8f8bcb6fde73 100644 --- a/internal/distributed/proxy/client/client.go +++ b/internal/distributed/proxy/client/client.go @@ -26,12 +26,12 @@ 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/internal/distributed/utils" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/proxypb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/grpcclient" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/proxypb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/distributed/proxy/client/client_test.go b/internal/distributed/proxy/client/client_test.go index e43b02869cbf2..1716665c2efee 100644 --- a/internal/distributed/proxy/client/client_test.go +++ b/internal/distributed/proxy/client/client_test.go @@ -26,8 +26,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/proxypb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/proxypb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/distributed/proxy/httpserver/handler_v2.go b/internal/distributed/proxy/httpserver/handler_v2.go index 2fc9c9dc60a4f..3d7c2daaca177 100644 --- a/internal/distributed/proxy/httpserver/handler_v2.go +++ b/internal/distributed/proxy/httpserver/handler_v2.go @@ -41,13 +41,13 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/json" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proxy" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/hookutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/crypto" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/distributed/proxy/httpserver/handler_v2_test.go b/internal/distributed/proxy/httpserver/handler_v2_test.go index b8f146ed1be32..44eeb6cb5640b 100644 --- a/internal/distributed/proxy/httpserver/handler_v2_test.go +++ b/internal/distributed/proxy/httpserver/handler_v2_test.go @@ -35,9 +35,9 @@ import ( mhttp "github.com/milvus-io/milvus/internal/http" "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proxy" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/distributed/proxy/service.go b/internal/distributed/proxy/service.go index 928fc4b7de538..1fb95a7b15780 100644 --- a/internal/distributed/proxy/service.go +++ b/internal/distributed/proxy/service.go @@ -53,8 +53,6 @@ import ( rcc "github.com/milvus-io/milvus/internal/distributed/rootcoord/client" "github.com/milvus-io/milvus/internal/distributed/utils" mhttp "github.com/milvus-io/milvus/internal/http" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/proxypb" "github.com/milvus-io/milvus/internal/proxy" "github.com/milvus-io/milvus/internal/proxy/accesslog" "github.com/milvus-io/milvus/internal/proxy/connection" @@ -66,6 +64,8 @@ import ( "github.com/milvus-io/milvus/internal/util/hookutil" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/proxypb" "github.com/milvus-io/milvus/pkg/tracer" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/etcd" diff --git a/internal/distributed/querycoord/client/client.go b/internal/distributed/querycoord/client/client.go index 9034d1dfc9bd5..1ff8198f90322 100644 --- a/internal/distributed/querycoord/client/client.go +++ b/internal/distributed/querycoord/client/client.go @@ -26,12 +26,12 @@ 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/internal/distributed/utils" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/grpcclient" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/distributed/querycoord/client/client_test.go b/internal/distributed/querycoord/client/client_test.go index 0b14ed48b2fa0..ab48fefdfc86b 100644 --- a/internal/distributed/querycoord/client/client_test.go +++ b/internal/distributed/querycoord/client/client_test.go @@ -29,9 +29,9 @@ import ( "go.uber.org/zap" "google.golang.org/grpc" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/util/mock" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/distributed/querycoord/service.go b/internal/distributed/querycoord/service.go index ffb6662e950e2..2287596f15866 100644 --- a/internal/distributed/querycoord/service.go +++ b/internal/distributed/querycoord/service.go @@ -33,13 +33,13 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus/internal/coordinator/coordclient" "github.com/milvus-io/milvus/internal/distributed/utils" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" qc "github.com/milvus-io/milvus/internal/querycoordv2" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/dependency" _ "github.com/milvus-io/milvus/internal/util/grpcclient" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/tracer" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/etcd" diff --git a/internal/distributed/querycoord/service_test.go b/internal/distributed/querycoord/service_test.go index 79301bdd563fb..1119b038ed3a7 100644 --- a/internal/distributed/querycoord/service_test.go +++ b/internal/distributed/querycoord/service_test.go @@ -29,8 +29,8 @@ 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/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/tikv" diff --git a/internal/distributed/querynode/client/client.go b/internal/distributed/querynode/client/client.go index a7e933a5eaca2..4da097c066a90 100644 --- a/internal/distributed/querynode/client/client.go +++ b/internal/distributed/querynode/client/client.go @@ -26,12 +26,12 @@ 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/internal/distributed/utils" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/grpcclient" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/distributed/querynode/client/client_test.go b/internal/distributed/querynode/client/client_test.go index e24a8b59a268e..779382bb36ea9 100644 --- a/internal/distributed/querynode/client/client_test.go +++ b/internal/distributed/querynode/client/client_test.go @@ -24,8 +24,8 @@ import ( "github.com/stretchr/testify/assert" "google.golang.org/grpc" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/util/mock" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/distributed/querynode/service.go b/internal/distributed/querynode/service.go index 5ae035f0a1bf9..5e49e5a39632b 100644 --- a/internal/distributed/querynode/service.go +++ b/internal/distributed/querynode/service.go @@ -32,13 +32,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/internal/distributed/utils" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" qn "github.com/milvus-io/milvus/internal/querynodev2" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/dependency" _ "github.com/milvus-io/milvus/internal/util/grpcclient" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/tracer" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/funcutil" diff --git a/internal/distributed/querynode/service_test.go b/internal/distributed/querynode/service_test.go index bfedc5ad58bee..3ac49214b7694 100644 --- a/internal/distributed/querynode/service_test.go +++ b/internal/distributed/querynode/service_test.go @@ -29,9 +29,9 @@ 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/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/distributed/rootcoord/client/client.go b/internal/distributed/rootcoord/client/client.go index db5fef340c95f..1d06f53ebc4f6 100644 --- a/internal/distributed/rootcoord/client/client.go +++ b/internal/distributed/rootcoord/client/client.go @@ -28,13 +28,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/internal/distributed/utils" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/proxypb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/grpcclient" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/proxypb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/distributed/rootcoord/client/client_test.go b/internal/distributed/rootcoord/client/client_test.go index b70883dcf742f..b580b96761a42 100644 --- a/internal/distributed/rootcoord/client/client_test.go +++ b/internal/distributed/rootcoord/client/client_test.go @@ -29,9 +29,9 @@ import ( "go.uber.org/zap" "google.golang.org/grpc" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/util/mock" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/distributed/rootcoord/service.go b/internal/distributed/rootcoord/service.go index d86b8dcb22799..948d52a9df097 100644 --- a/internal/distributed/rootcoord/service.go +++ b/internal/distributed/rootcoord/service.go @@ -33,15 +33,15 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus/internal/coordinator/coordclient" "github.com/milvus-io/milvus/internal/distributed/utils" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/proxypb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/rootcoord" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/dependency" _ "github.com/milvus-io/milvus/internal/util/grpcclient" streamingserviceinterceptor "github.com/milvus-io/milvus/internal/util/streamingutil/service/interceptor" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/proxypb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/tracer" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/etcd" diff --git a/internal/distributed/rootcoord/service_test.go b/internal/distributed/rootcoord/service_test.go index d9250966cc26e..f475396d70094 100644 --- a/internal/distributed/rootcoord/service_test.go +++ b/internal/distributed/rootcoord/service_test.go @@ -32,11 +32,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/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/rootcoord" "github.com/milvus-io/milvus/internal/types" kvfactory "github.com/milvus-io/milvus/internal/util/dependency/kv" "github.com/milvus-io/milvus/internal/util/sessionutil" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/distributed/streamingnode/service.go b/internal/distributed/streamingnode/service.go index 2e50721ed2d38..a3d3e04268343 100644 --- a/internal/distributed/streamingnode/service.go +++ b/internal/distributed/streamingnode/service.go @@ -47,7 +47,7 @@ import ( streamingserviceinterceptor "github.com/milvus-io/milvus/internal/util/streamingutil/service/interceptor" "github.com/milvus-io/milvus/pkg/kv" "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/tracer" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/funcutil" diff --git a/internal/flushcommon/broker/broker.go b/internal/flushcommon/broker/broker.go index 7e85f7fa76794..41091c632359d 100644 --- a/internal/flushcommon/broker/broker.go +++ b/internal/flushcommon/broker/broker.go @@ -4,8 +4,8 @@ import ( "context" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/flushcommon/broker/datacoord.go b/internal/flushcommon/broker/datacoord.go index 453e3eb4a0fe1..070c88de75fe1 100644 --- a/internal/flushcommon/broker/datacoord.go +++ b/internal/flushcommon/broker/datacoord.go @@ -11,9 +11,9 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/metastore/kv/binlog" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/flushcommon/broker/datacoord_test.go b/internal/flushcommon/broker/datacoord_test.go index ace1c29eb9a55..5b7aca1af9f67 100644 --- a/internal/flushcommon/broker/datacoord_test.go +++ b/internal/flushcommon/broker/datacoord_test.go @@ -14,7 +14,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/tsoutil" diff --git a/internal/flushcommon/broker/mock_broker.go b/internal/flushcommon/broker/mock_broker.go index f5dc156dae911..c55a0317703da 100644 --- a/internal/flushcommon/broker/mock_broker.go +++ b/internal/flushcommon/broker/mock_broker.go @@ -5,7 +5,7 @@ package broker import ( context "context" - datapb "github.com/milvus-io/milvus/internal/proto/datapb" + datapb "github.com/milvus-io/milvus/pkg/proto/datapb" mock "github.com/stretchr/testify/mock" msgpb "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" diff --git a/internal/flushcommon/metacache/actions.go b/internal/flushcommon/metacache/actions.go index 302f1c14c3f50..c6219c6f8c9aa 100644 --- a/internal/flushcommon/metacache/actions.go +++ b/internal/flushcommon/metacache/actions.go @@ -19,9 +19,9 @@ package metacache import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/flushcommon/metacache/meta_cache.go b/internal/flushcommon/metacache/meta_cache.go index cb75b3ddf1773..52c83d553d688 100644 --- a/internal/flushcommon/metacache/meta_cache.go +++ b/internal/flushcommon/metacache/meta_cache.go @@ -25,9 +25,9 @@ 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/flushcommon/metacache/pkoracle" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" ) //go:generate mockery --name=MetaCache --structname=MockMetaCache --output=./ --filename=mock_meta_cache.go --with-expecter --inpackage diff --git a/internal/flushcommon/metacache/meta_cache_test.go b/internal/flushcommon/metacache/meta_cache_test.go index 7f539b7e76a81..a01ed49374af3 100644 --- a/internal/flushcommon/metacache/meta_cache_test.go +++ b/internal/flushcommon/metacache/meta_cache_test.go @@ -25,9 +25,9 @@ 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/flushcommon/metacache/pkoracle" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/flushcommon/metacache/mock_meta_cache.go b/internal/flushcommon/metacache/mock_meta_cache.go index 2476ce4443bd2..1a05e2ca317ae 100644 --- a/internal/flushcommon/metacache/mock_meta_cache.go +++ b/internal/flushcommon/metacache/mock_meta_cache.go @@ -3,7 +3,7 @@ package metacache import ( - datapb "github.com/milvus-io/milvus/internal/proto/datapb" + datapb "github.com/milvus-io/milvus/pkg/proto/datapb" mock "github.com/stretchr/testify/mock" pkoracle "github.com/milvus-io/milvus/internal/flushcommon/metacache/pkoracle" diff --git a/internal/flushcommon/metacache/segment.go b/internal/flushcommon/metacache/segment.go index 4bdadda6177ad..35ca13069c85c 100644 --- a/internal/flushcommon/metacache/segment.go +++ b/internal/flushcommon/metacache/segment.go @@ -20,8 +20,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/flushcommon/metacache/pkoracle" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/pkg/proto/datapb" ) type SegmentInfo struct { diff --git a/internal/flushcommon/metacache/segment_test.go b/internal/flushcommon/metacache/segment_test.go index cc01ae0257ea7..3a0abb23c6fa1 100644 --- a/internal/flushcommon/metacache/segment_test.go +++ b/internal/flushcommon/metacache/segment_test.go @@ -22,7 +22,7 @@ import ( "github.com/stretchr/testify/suite" "github.com/milvus-io/milvus/internal/flushcommon/metacache/pkoracle" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" ) type SegmentSuite struct { diff --git a/internal/flushcommon/pipeline/data_sync_service.go b/internal/flushcommon/pipeline/data_sync_service.go index 4e69cfa82669d..b9f23475b8f02 100644 --- a/internal/flushcommon/pipeline/data_sync_service.go +++ b/internal/flushcommon/pipeline/data_sync_service.go @@ -30,7 +30,6 @@ import ( "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" "github.com/milvus-io/milvus/internal/flushcommon/util" "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/flowgraph" "github.com/milvus-io/milvus/internal/util/streamingutil" @@ -38,6 +37,7 @@ import ( "github.com/milvus-io/milvus/pkg/metrics" "github.com/milvus-io/milvus/pkg/mq/msgdispatcher" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/flushcommon/pipeline/data_sync_service_test.go b/internal/flushcommon/pipeline/data_sync_service_test.go index 1a55231b0e8d8..e8364cd78e16d 100644 --- a/internal/flushcommon/pipeline/data_sync_service_test.go +++ b/internal/flushcommon/pipeline/data_sync_service_test.go @@ -38,13 +38,13 @@ import ( util2 "github.com/milvus-io/milvus/internal/flushcommon/util" "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/mq/msgdispatcher" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/tsoutil" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/flushcommon/pipeline/flow_graph_dd_node.go b/internal/flushcommon/pipeline/flow_graph_dd_node.go index fd5a0b8a78a48..523cfcdb2953e 100644 --- a/internal/flushcommon/pipeline/flow_graph_dd_node.go +++ b/internal/flushcommon/pipeline/flow_graph_dd_node.go @@ -30,13 +30,13 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/datanode/compaction" "github.com/milvus-io/milvus/internal/flushcommon/util" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/streamingnode/server/flusher" "github.com/milvus-io/milvus/internal/util/flowgraph" "github.com/milvus-io/milvus/internal/util/streamingutil" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/streaming/util/message/adaptor" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/flushcommon/pipeline/flow_graph_dd_node_test.go b/internal/flushcommon/pipeline/flow_graph_dd_node_test.go index a7920eaa3acbc..053b5e9df5c3c 100644 --- a/internal/flushcommon/pipeline/flow_graph_dd_node_test.go +++ b/internal/flushcommon/pipeline/flow_graph_dd_node_test.go @@ -29,10 +29,10 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/datanode/compaction" "github.com/milvus-io/milvus/internal/mocks/streamingnode/server/mock_flusher" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/util/flowgraph" "github.com/milvus-io/milvus/pkg/mocks/streaming/util/mock_message" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/message/adaptor" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/flushcommon/pipeline/flow_graph_manager_test.go b/internal/flushcommon/pipeline/flow_graph_manager_test.go index 0566cc437aa5e..3a0649276867a 100644 --- a/internal/flushcommon/pipeline/flow_graph_manager_test.go +++ b/internal/flushcommon/pipeline/flow_graph_manager_test.go @@ -35,12 +35,12 @@ import ( "github.com/milvus-io/milvus/internal/flushcommon/util" "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" "github.com/milvus-io/milvus/internal/json" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/util/metrics" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/mq/msgdispatcher" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/tsoutil" diff --git a/internal/flushcommon/pipeline/testutils_test.go b/internal/flushcommon/pipeline/testutils_test.go index 7a460763f71e7..d9323ca59abea 100644 --- a/internal/flushcommon/pipeline/testutils_test.go +++ b/internal/flushcommon/pipeline/testutils_test.go @@ -32,15 +32,15 @@ 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/metacache/pkoracle" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/etcdpb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/etcdpb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/flushcommon/syncmgr/meta_writer.go b/internal/flushcommon/syncmgr/meta_writer.go index 3f6773c37a831..0fbcd3f841077 100644 --- a/internal/flushcommon/syncmgr/meta_writer.go +++ b/internal/flushcommon/syncmgr/meta_writer.go @@ -10,8 +10,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus/internal/flushcommon/broker" "github.com/milvus-io/milvus/internal/flushcommon/metacache" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/retry" diff --git a/internal/flushcommon/syncmgr/meta_writer_test.go b/internal/flushcommon/syncmgr/meta_writer_test.go index 32890339d2637..35a0aa44e66c4 100644 --- a/internal/flushcommon/syncmgr/meta_writer_test.go +++ b/internal/flushcommon/syncmgr/meta_writer_test.go @@ -11,7 +11,7 @@ import ( "github.com/milvus-io/milvus/internal/flushcommon/broker" "github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/flushcommon/metacache/pkoracle" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/retry" ) diff --git a/internal/flushcommon/syncmgr/options.go b/internal/flushcommon/syncmgr/options.go index 7c84d7099e5ee..57e78c63c14ce 100644 --- a/internal/flushcommon/syncmgr/options.go +++ b/internal/flushcommon/syncmgr/options.go @@ -7,8 +7,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/flushcommon/metacache" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/retry" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/flushcommon/syncmgr/serializer.go b/internal/flushcommon/syncmgr/serializer.go index 25dfd8103f0c0..69f7171b2a368 100644 --- a/internal/flushcommon/syncmgr/serializer.go +++ b/internal/flushcommon/syncmgr/serializer.go @@ -23,8 +23,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/flushcommon/metacache" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/flushcommon/syncmgr/storage_serializer.go b/internal/flushcommon/syncmgr/storage_serializer.go index d45e9755aed49..32379eb0c433f 100644 --- a/internal/flushcommon/syncmgr/storage_serializer.go +++ b/internal/flushcommon/syncmgr/storage_serializer.go @@ -27,11 +27,11 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/flushcommon/metacache" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/etcdpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/etcdpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/timerecord" diff --git a/internal/flushcommon/syncmgr/storage_serializer_test.go b/internal/flushcommon/syncmgr/storage_serializer_test.go index b6d94d574cbcc..6e83c72da428a 100644 --- a/internal/flushcommon/syncmgr/storage_serializer_test.go +++ b/internal/flushcommon/syncmgr/storage_serializer_test.go @@ -33,9 +33,9 @@ import ( "github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/flushcommon/metacache/pkoracle" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/tsoutil" ) diff --git a/internal/flushcommon/syncmgr/sync_manager_test.go b/internal/flushcommon/syncmgr/sync_manager_test.go index 0531f481db8bd..e674ba55b023f 100644 --- a/internal/flushcommon/syncmgr/sync_manager_test.go +++ b/internal/flushcommon/syncmgr/sync_manager_test.go @@ -23,10 +23,10 @@ import ( "github.com/milvus-io/milvus/internal/flushcommon/metacache/pkoracle" "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/config" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/tsoutil" diff --git a/internal/flushcommon/syncmgr/task.go b/internal/flushcommon/syncmgr/task.go index e5c2f913cd0c6..4d66f97530968 100644 --- a/internal/flushcommon/syncmgr/task.go +++ b/internal/flushcommon/syncmgr/task.go @@ -31,11 +31,11 @@ import ( "github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/json" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metautil" "github.com/milvus-io/milvus/pkg/util/metricsinfo" diff --git a/internal/flushcommon/syncmgr/task_test.go b/internal/flushcommon/syncmgr/task_test.go index 29e103b437c15..ef1655fc8c6fe 100644 --- a/internal/flushcommon/syncmgr/task_test.go +++ b/internal/flushcommon/syncmgr/task_test.go @@ -37,9 +37,9 @@ import ( "github.com/milvus-io/milvus/internal/flushcommon/metacache/pkoracle" "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/retry" diff --git a/internal/flushcommon/writebuffer/l0_write_buffer.go b/internal/flushcommon/writebuffer/l0_write_buffer.go index 07237f43f7705..b0834703a50da 100644 --- a/internal/flushcommon/writebuffer/l0_write_buffer.go +++ b/internal/flushcommon/writebuffer/l0_write_buffer.go @@ -13,11 +13,11 @@ import ( "github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/flushcommon/metacache/pkoracle" "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/streamingutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/flushcommon/writebuffer/l0_write_buffer_test.go b/internal/flushcommon/writebuffer/l0_write_buffer_test.go index 53c911544a342..3a83390626100 100644 --- a/internal/flushcommon/writebuffer/l0_write_buffer_test.go +++ b/internal/flushcommon/writebuffer/l0_write_buffer_test.go @@ -17,11 +17,11 @@ import ( "github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/flushcommon/metacache/pkoracle" "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/metrics" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/testutils" "github.com/milvus-io/milvus/pkg/util/tsoutil" diff --git a/internal/flushcommon/writebuffer/write_buffer.go b/internal/flushcommon/writebuffer/write_buffer.go index 07aa582546d18..f5538b2810ac1 100644 --- a/internal/flushcommon/writebuffer/write_buffer.go +++ b/internal/flushcommon/writebuffer/write_buffer.go @@ -16,12 +16,12 @@ import ( "github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/flushcommon/metacache/pkoracle" "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/streamingutil" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/flushcommon/writebuffer/write_buffer_test.go b/internal/flushcommon/writebuffer/write_buffer_test.go index 6f6f20df179a1..55f41f42f9d9d 100644 --- a/internal/flushcommon/writebuffer/write_buffer_test.go +++ b/internal/flushcommon/writebuffer/write_buffer_test.go @@ -14,8 +14,8 @@ import ( "github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/flushcommon/metacache/pkoracle" "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/indexnode/chunk_mgr_factory.go b/internal/indexnode/chunk_mgr_factory.go index 839a05c714a85..98a4c70308bb6 100644 --- a/internal/indexnode/chunk_mgr_factory.go +++ b/internal/indexnode/chunk_mgr_factory.go @@ -4,8 +4,8 @@ import ( "context" "fmt" - "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/pkg/proto/indexpb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/indexnode/chunkmgr_mock.go b/internal/indexnode/chunkmgr_mock.go index a839ae79fcea8..bc470ad0d4a86 100644 --- a/internal/indexnode/chunkmgr_mock.go +++ b/internal/indexnode/chunkmgr_mock.go @@ -11,11 +11,11 @@ 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/proto/etcdpb" - "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/etcdpb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/indexnode/index_test.go b/internal/indexnode/index_test.go index 433cfe8ebd01c..c522851fb5d3f 100644 --- a/internal/indexnode/index_test.go +++ b/internal/indexnode/index_test.go @@ -6,9 +6,9 @@ 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/proto/etcdpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/etcdpb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/indexnode/indexnode.go b/internal/indexnode/indexnode.go index 31617d18276ed..a91a469287f14 100644 --- a/internal/indexnode/indexnode.go +++ b/internal/indexnode/indexnode.go @@ -45,7 +45,6 @@ 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/internal/flushcommon/io" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/internal/util/initcore" @@ -53,6 +52,7 @@ import ( "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/expr" "github.com/milvus-io/milvus/pkg/util/hardware" "github.com/milvus-io/milvus/pkg/util/lifetime" diff --git a/internal/indexnode/indexnode_service.go b/internal/indexnode/indexnode_service.go index 69c96b4162cb5..98b5b8e08fdf7 100644 --- a/internal/indexnode/indexnode_service.go +++ b/internal/indexnode/indexnode_service.go @@ -30,11 +30,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/internal/flushcommon/io" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/indexnode/indexnode_service_test.go b/internal/indexnode/indexnode_service_test.go index 1757b7b835e2f..0033aa5d38a56 100644 --- a/internal/indexnode/indexnode_service_test.go +++ b/internal/indexnode/indexnode_service_test.go @@ -25,9 +25,9 @@ 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/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" ) diff --git a/internal/indexnode/indexnode_test.go b/internal/indexnode/indexnode_test.go index 39bb694ba0a05..8c683ed72a700 100644 --- a/internal/indexnode/indexnode_test.go +++ b/internal/indexnode/indexnode_test.go @@ -30,10 +30,10 @@ 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/metastore/kv/binlog" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/indexnode/task.go b/internal/indexnode/task.go index 003d2621c1252..e659747b8053e 100644 --- a/internal/indexnode/task.go +++ b/internal/indexnode/task.go @@ -20,8 +20,8 @@ import ( "context" "fmt" - "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/pkg/proto/indexpb" ) var ( diff --git a/internal/indexnode/task_analyze.go b/internal/indexnode/task_analyze.go index 796373cccca61..01176d6270222 100644 --- a/internal/indexnode/task_analyze.go +++ b/internal/indexnode/task_analyze.go @@ -24,11 +24,11 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/clusteringpb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/internal/util/analyzecgowrapper" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/clusteringpb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/hardware" "github.com/milvus-io/milvus/pkg/util/metautil" "github.com/milvus-io/milvus/pkg/util/timerecord" diff --git a/internal/indexnode/task_index.go b/internal/indexnode/task_index.go index 87e0f44be8684..4a316f3c909cb 100644 --- a/internal/indexnode/task_index.go +++ b/internal/indexnode/task_index.go @@ -28,15 +28,15 @@ 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/proto/indexcgopb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/indexcgowrapper" "github.com/milvus-io/milvus/internal/util/vecindexmgr" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/indexcgopb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/indexparams" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metautil" diff --git a/internal/indexnode/task_scheduler.go b/internal/indexnode/task_scheduler.go index cccc85595e052..7f6db70427850 100644 --- a/internal/indexnode/task_scheduler.go +++ b/internal/indexnode/task_scheduler.go @@ -26,9 +26,9 @@ import ( "github.com/cockroachdb/errors" "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/indexpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/indexnode/task_scheduler_test.go b/internal/indexnode/task_scheduler_test.go index 36e5b04db3b63..22a9de4d7cbb2 100644 --- a/internal/indexnode/task_scheduler_test.go +++ b/internal/indexnode/task_scheduler_test.go @@ -9,7 +9,7 @@ import ( "github.com/stretchr/testify/assert" - "github.com/milvus-io/milvus/internal/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/indexnode/task_stats.go b/internal/indexnode/task_stats.go index 40c4756877dd5..68c5409b32f57 100644 --- a/internal/indexnode/task_stats.go +++ b/internal/indexnode/task_stats.go @@ -33,13 +33,13 @@ import ( iter "github.com/milvus-io/milvus/internal/datanode/iterators" "github.com/milvus-io/milvus/internal/flushcommon/io" "github.com/milvus-io/milvus/internal/metastore/kv/binlog" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexcgopb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/indexcgowrapper" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexcgopb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/conc" _ "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/metautil" diff --git a/internal/indexnode/task_stats_test.go b/internal/indexnode/task_stats_test.go index 48cde12f4cbb3..6243301578bf4 100644 --- a/internal/indexnode/task_stats_test.go +++ b/internal/indexnode/task_stats_test.go @@ -30,9 +30,9 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/datanode/compaction" "github.com/milvus-io/milvus/internal/flushcommon/io" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/tsoutil" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/indexnode/task_test.go b/internal/indexnode/task_test.go index 354d1d163dc44..9db8a58e03f49 100644 --- a/internal/indexnode/task_test.go +++ b/internal/indexnode/task_test.go @@ -24,12 +24,12 @@ 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/proto/etcdpb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/etcdpb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/metautil" "github.com/milvus-io/milvus/pkg/util/metric" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/indexnode/taskinfo_ops.go b/internal/indexnode/taskinfo_ops.go index 8a0431f76bf42..122a58b011c59 100644 --- a/internal/indexnode/taskinfo_ops.go +++ b/internal/indexnode/taskinfo_ops.go @@ -23,10 +23,10 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" ) type indexTaskInfo struct { diff --git a/internal/indexnode/taskinfo_ops_test.go b/internal/indexnode/taskinfo_ops_test.go index 40fbec46136f3..f286a1b87d6ea 100644 --- a/internal/indexnode/taskinfo_ops_test.go +++ b/internal/indexnode/taskinfo_ops_test.go @@ -22,8 +22,8 @@ import ( "github.com/stretchr/testify/suite" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" ) type statsTaskInfoSuite struct { diff --git a/internal/metastore/catalog.go b/internal/metastore/catalog.go index c7a2042dd73f0..0238cf74550a0 100644 --- a/internal/metastore/catalog.go +++ b/internal/metastore/catalog.go @@ -6,10 +6,10 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/metastore/model" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/metastore/kv/binlog/binlog.go b/internal/metastore/kv/binlog/binlog.go index 126d0a243c66f..a4b1ecf79d324 100644 --- a/internal/metastore/kv/binlog/binlog.go +++ b/internal/metastore/kv/binlog/binlog.go @@ -21,8 +21,8 @@ import ( "strconv" "strings" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metautil" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/metastore/kv/binlog/binlog_test.go b/internal/metastore/kv/binlog/binlog_test.go index 3437ce742540e..174861055d534 100644 --- a/internal/metastore/kv/binlog/binlog_test.go +++ b/internal/metastore/kv/binlog/binlog_test.go @@ -24,8 +24,8 @@ import ( "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metautil" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/metastore/kv/datacoord/kv_catalog.go b/internal/metastore/kv/datacoord/kv_catalog.go index aba3cc4990617..80c00ca13437b 100644 --- a/internal/metastore/kv/datacoord/kv_catalog.go +++ b/internal/metastore/kv/datacoord/kv_catalog.go @@ -34,13 +34,13 @@ import ( "github.com/milvus-io/milvus/internal/metastore" "github.com/milvus-io/milvus/internal/metastore/kv/binlog" "github.com/milvus-io/milvus/internal/metastore/model" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/segmentutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/kv" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/metastore/kv/datacoord/kv_catalog_test.go b/internal/metastore/kv/datacoord/kv_catalog_test.go index 992aeb0830aed..e12a2d2da2ccc 100644 --- a/internal/metastore/kv/datacoord/kv_catalog_test.go +++ b/internal/metastore/kv/datacoord/kv_catalog_test.go @@ -40,9 +40,9 @@ import ( "github.com/milvus-io/milvus/internal/kv/mocks" "github.com/milvus-io/milvus/internal/metastore" "github.com/milvus-io/milvus/internal/metastore/model" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/pkg/kv/predicates" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/metautil" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/metastore/kv/datacoord/util.go b/internal/metastore/kv/datacoord/util.go index 75cc2468db1a8..716285a24dac4 100644 --- a/internal/metastore/kv/datacoord/util.go +++ b/internal/metastore/kv/datacoord/util.go @@ -22,10 +22,10 @@ import ( "go.uber.org/zap" "google.golang.org/protobuf/proto" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/segmentutil" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/metastore/kv/querycoord/kv_catalog.go b/internal/metastore/kv/querycoord/kv_catalog.go index 4d4a7047149c7..ad963e38bc235 100644 --- a/internal/metastore/kv/querycoord/kv_catalog.go +++ b/internal/metastore/kv/querycoord/kv_catalog.go @@ -13,9 +13,9 @@ import ( "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/pkg/kv" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/compressor" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/metastore/kv/querycoord/kv_catalog_test.go b/internal/metastore/kv/querycoord/kv_catalog_test.go index 1c119e838f611..5c2b86d791baa 100644 --- a/internal/metastore/kv/querycoord/kv_catalog_test.go +++ b/internal/metastore/kv/querycoord/kv_catalog_test.go @@ -11,9 +11,9 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/kv/mocks" - "github.com/milvus-io/milvus/internal/proto/querypb" . "github.com/milvus-io/milvus/internal/querycoordv2/params" "github.com/milvus-io/milvus/pkg/kv" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/metastore/kv/rootcoord/kv_catalog.go b/internal/metastore/kv/rootcoord/kv_catalog.go index 4cccde3a8bf0b..b37a301ab8385 100644 --- a/internal/metastore/kv/rootcoord/kv_catalog.go +++ b/internal/metastore/kv/rootcoord/kv_catalog.go @@ -17,11 +17,11 @@ import ( "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/metastore" "github.com/milvus-io/milvus/internal/metastore/model" - pb "github.com/milvus-io/milvus/internal/proto/etcdpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/kv" "github.com/milvus-io/milvus/pkg/log" + pb "github.com/milvus-io/milvus/pkg/proto/etcdpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/crypto" "github.com/milvus-io/milvus/pkg/util/etcd" diff --git a/internal/metastore/kv/rootcoord/kv_catalog_test.go b/internal/metastore/kv/rootcoord/kv_catalog_test.go index dea07983bb501..20a0815c9ca23 100644 --- a/internal/metastore/kv/rootcoord/kv_catalog_test.go +++ b/internal/metastore/kv/rootcoord/kv_catalog_test.go @@ -26,10 +26,10 @@ import ( "github.com/milvus-io/milvus/internal/kv/mocks" "github.com/milvus-io/milvus/internal/metastore" "github.com/milvus-io/milvus/internal/metastore/model" - pb "github.com/milvus-io/milvus/internal/proto/etcdpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + pb "github.com/milvus-io/milvus/pkg/proto/etcdpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/crypto" "github.com/milvus-io/milvus/pkg/util/etcd" diff --git a/internal/metastore/kv/streamingcoord/kv_catalog.go b/internal/metastore/kv/streamingcoord/kv_catalog.go index c0a16a525106e..539e0ca4daaf6 100644 --- a/internal/metastore/kv/streamingcoord/kv_catalog.go +++ b/internal/metastore/kv/streamingcoord/kv_catalog.go @@ -9,7 +9,7 @@ import ( "github.com/milvus-io/milvus/internal/metastore" "github.com/milvus-io/milvus/pkg/kv" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/etcd" ) diff --git a/internal/metastore/kv/streamingcoord/kv_catalog_test.go b/internal/metastore/kv/streamingcoord/kv_catalog_test.go index 215aee3d15ee3..b805ccf05407b 100644 --- a/internal/metastore/kv/streamingcoord/kv_catalog_test.go +++ b/internal/metastore/kv/streamingcoord/kv_catalog_test.go @@ -10,7 +10,7 @@ import ( "github.com/stretchr/testify/mock" "github.com/milvus-io/milvus/pkg/mocks/mock_kv" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) func TestCatalog(t *testing.T) { diff --git a/internal/metastore/kv/streamingnode/kv_catalog.go b/internal/metastore/kv/streamingnode/kv_catalog.go index 2e474738cfdef..7cb546fdf2a6d 100644 --- a/internal/metastore/kv/streamingnode/kv_catalog.go +++ b/internal/metastore/kv/streamingnode/kv_catalog.go @@ -10,7 +10,7 @@ import ( "github.com/milvus-io/milvus/internal/metastore" "github.com/milvus-io/milvus/pkg/kv" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/etcd" ) diff --git a/internal/metastore/kv/streamingnode/kv_catalog_test.go b/internal/metastore/kv/streamingnode/kv_catalog_test.go index 4ebdf5a09ca9b..177726585f3e5 100644 --- a/internal/metastore/kv/streamingnode/kv_catalog_test.go +++ b/internal/metastore/kv/streamingnode/kv_catalog_test.go @@ -9,7 +9,7 @@ import ( "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus/internal/kv/mocks" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) func TestCatalog(t *testing.T) { diff --git a/internal/metastore/mocks/mock_datacoord_catalog.go b/internal/metastore/mocks/mock_datacoord_catalog.go index d10d832b9b1c8..cb92b3652970d 100644 --- a/internal/metastore/mocks/mock_datacoord_catalog.go +++ b/internal/metastore/mocks/mock_datacoord_catalog.go @@ -5,8 +5,8 @@ package mocks import ( context "context" - datapb "github.com/milvus-io/milvus/internal/proto/datapb" - indexpb "github.com/milvus-io/milvus/internal/proto/indexpb" + datapb "github.com/milvus-io/milvus/pkg/proto/datapb" + indexpb "github.com/milvus-io/milvus/pkg/proto/indexpb" metastore "github.com/milvus-io/milvus/internal/metastore" diff --git a/internal/metastore/mocks/mock_querycoord_catalog.go b/internal/metastore/mocks/mock_querycoord_catalog.go index c8368bfcbf31a..70fca13b38e94 100644 --- a/internal/metastore/mocks/mock_querycoord_catalog.go +++ b/internal/metastore/mocks/mock_querycoord_catalog.go @@ -7,7 +7,7 @@ import ( mock "github.com/stretchr/testify/mock" - querypb "github.com/milvus-io/milvus/internal/proto/querypb" + querypb "github.com/milvus-io/milvus/pkg/proto/querypb" ) // QueryCoordCatalog is an autogenerated mock type for the QueryCoordCatalog type diff --git a/internal/metastore/model/alias.go b/internal/metastore/model/alias.go index 1291410896a8a..f0422c8158737 100644 --- a/internal/metastore/model/alias.go +++ b/internal/metastore/model/alias.go @@ -1,6 +1,6 @@ package model -import pb "github.com/milvus-io/milvus/internal/proto/etcdpb" +import pb "github.com/milvus-io/milvus/pkg/proto/etcdpb" type Alias struct { Name string diff --git a/internal/metastore/model/alias_test.go b/internal/metastore/model/alias_test.go index 172f5d6dce3ea..0b6aaac704905 100644 --- a/internal/metastore/model/alias_test.go +++ b/internal/metastore/model/alias_test.go @@ -5,7 +5,7 @@ import ( "github.com/stretchr/testify/assert" - "github.com/milvus-io/milvus/internal/proto/etcdpb" + "github.com/milvus-io/milvus/pkg/proto/etcdpb" ) func TestAlias_Available(t *testing.T) { diff --git a/internal/metastore/model/collection.go b/internal/metastore/model/collection.go index 2a41804f14641..a620c70601b52 100644 --- a/internal/metastore/model/collection.go +++ b/internal/metastore/model/collection.go @@ -5,8 +5,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - pb "github.com/milvus-io/milvus/internal/proto/etcdpb" "github.com/milvus-io/milvus/pkg/common" + pb "github.com/milvus-io/milvus/pkg/proto/etcdpb" ) type Collection struct { diff --git a/internal/metastore/model/collection_test.go b/internal/metastore/model/collection_test.go index 0dfdc59c42be5..ee11b29abf2e4 100644 --- a/internal/metastore/model/collection_test.go +++ b/internal/metastore/model/collection_test.go @@ -7,8 +7,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - pb "github.com/milvus-io/milvus/internal/proto/etcdpb" "github.com/milvus-io/milvus/pkg/common" + pb "github.com/milvus-io/milvus/pkg/proto/etcdpb" ) var ( diff --git a/internal/metastore/model/credential.go b/internal/metastore/model/credential.go index 64e00f8a83392..8f28055f075e9 100644 --- a/internal/metastore/model/credential.go +++ b/internal/metastore/model/credential.go @@ -1,6 +1,6 @@ package model -import "github.com/milvus-io/milvus/internal/proto/internalpb" +import "github.com/milvus-io/milvus/pkg/proto/internalpb" type Credential struct { Username string diff --git a/internal/metastore/model/credential_test.go b/internal/metastore/model/credential_test.go index b7ebdaef2f250..b155b8c8d5f4c 100644 --- a/internal/metastore/model/credential_test.go +++ b/internal/metastore/model/credential_test.go @@ -5,7 +5,7 @@ import ( "github.com/stretchr/testify/assert" - "github.com/milvus-io/milvus/internal/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" ) var ( diff --git a/internal/metastore/model/database.go b/internal/metastore/model/database.go index 95166a65b5de6..5ea4b3077a020 100644 --- a/internal/metastore/model/database.go +++ b/internal/metastore/model/database.go @@ -4,8 +4,8 @@ import ( "time" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - pb "github.com/milvus-io/milvus/internal/proto/etcdpb" "github.com/milvus-io/milvus/pkg/common" + pb "github.com/milvus-io/milvus/pkg/proto/etcdpb" "github.com/milvus-io/milvus/pkg/util" ) diff --git a/internal/metastore/model/database_test.go b/internal/metastore/model/database_test.go index 622674d285171..57df5e60c12e9 100644 --- a/internal/metastore/model/database_test.go +++ b/internal/metastore/model/database_test.go @@ -6,7 +6,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/etcdpb" + "github.com/milvus-io/milvus/pkg/proto/etcdpb" ) var ( diff --git a/internal/metastore/model/index.go b/internal/metastore/model/index.go index cda9537091761..8b16f44105043 100644 --- a/internal/metastore/model/index.go +++ b/internal/metastore/model/index.go @@ -4,7 +4,7 @@ import ( "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" ) type Index struct { diff --git a/internal/metastore/model/index_test.go b/internal/metastore/model/index_test.go index 10cde6c136612..2d7af8cb471d3 100644 --- a/internal/metastore/model/index_test.go +++ b/internal/metastore/model/index_test.go @@ -6,7 +6,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" ) var ( diff --git a/internal/metastore/model/load_info.go b/internal/metastore/model/load_info.go index 1a79a50164995..eb2eba9eaaa8f 100644 --- a/internal/metastore/model/load_info.go +++ b/internal/metastore/model/load_info.go @@ -1,6 +1,6 @@ package model -import "github.com/milvus-io/milvus/internal/proto/querypb" +import "github.com/milvus-io/milvus/pkg/proto/querypb" type CollectionLoadInfo struct { CollectionID int64 diff --git a/internal/metastore/model/partition.go b/internal/metastore/model/partition.go index 4fda387ec4136..8033aa37aa8b8 100644 --- a/internal/metastore/model/partition.go +++ b/internal/metastore/model/partition.go @@ -1,8 +1,8 @@ package model import ( - pb "github.com/milvus-io/milvus/internal/proto/etcdpb" "github.com/milvus-io/milvus/pkg/common" + pb "github.com/milvus-io/milvus/pkg/proto/etcdpb" ) type Partition struct { diff --git a/internal/metastore/model/segment_index.go b/internal/metastore/model/segment_index.go index 8c2e8e3ef3827..0fe7d2a8828a4 100644 --- a/internal/metastore/model/segment_index.go +++ b/internal/metastore/model/segment_index.go @@ -2,8 +2,8 @@ package model import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/indexpb" ) type SegmentIndex struct { diff --git a/internal/metastore/model/segment_index_test.go b/internal/metastore/model/segment_index_test.go index 8ad49de8f4442..d038e14804a1c 100644 --- a/internal/metastore/model/segment_index_test.go +++ b/internal/metastore/model/segment_index_test.go @@ -6,7 +6,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" ) var ( diff --git a/internal/mocks/mock_datacoord.go b/internal/mocks/mock_datacoord.go index 8d7004e4d5190..4a329051b0352 100644 --- a/internal/mocks/mock_datacoord.go +++ b/internal/mocks/mock_datacoord.go @@ -8,11 +8,11 @@ import ( commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" clientv3 "go.etcd.io/etcd/client/v3" - datapb "github.com/milvus-io/milvus/internal/proto/datapb" + datapb "github.com/milvus-io/milvus/pkg/proto/datapb" - indexpb "github.com/milvus-io/milvus/internal/proto/indexpb" + indexpb "github.com/milvus-io/milvus/pkg/proto/indexpb" - internalpb "github.com/milvus-io/milvus/internal/proto/internalpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" diff --git a/internal/mocks/mock_datacoord_client.go b/internal/mocks/mock_datacoord_client.go index 9f42a0f953877..1b9ee181986c5 100644 --- a/internal/mocks/mock_datacoord_client.go +++ b/internal/mocks/mock_datacoord_client.go @@ -7,13 +7,13 @@ import ( commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - datapb "github.com/milvus-io/milvus/internal/proto/datapb" + datapb "github.com/milvus-io/milvus/pkg/proto/datapb" grpc "google.golang.org/grpc" - indexpb "github.com/milvus-io/milvus/internal/proto/indexpb" + indexpb "github.com/milvus-io/milvus/pkg/proto/indexpb" - internalpb "github.com/milvus-io/milvus/internal/proto/internalpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" diff --git a/internal/mocks/mock_datanode.go b/internal/mocks/mock_datanode.go index f03f66c8afb07..0152b7aea858d 100644 --- a/internal/mocks/mock_datanode.go +++ b/internal/mocks/mock_datanode.go @@ -8,9 +8,9 @@ import ( commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" clientv3 "go.etcd.io/etcd/client/v3" - datapb "github.com/milvus-io/milvus/internal/proto/datapb" + datapb "github.com/milvus-io/milvus/pkg/proto/datapb" - internalpb "github.com/milvus-io/milvus/internal/proto/internalpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" diff --git a/internal/mocks/mock_datanode_client.go b/internal/mocks/mock_datanode_client.go index fc493844fba44..1018edb68b643 100644 --- a/internal/mocks/mock_datanode_client.go +++ b/internal/mocks/mock_datanode_client.go @@ -7,11 +7,11 @@ import ( commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - datapb "github.com/milvus-io/milvus/internal/proto/datapb" + datapb "github.com/milvus-io/milvus/pkg/proto/datapb" grpc "google.golang.org/grpc" - internalpb "github.com/milvus-io/milvus/internal/proto/internalpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" diff --git a/internal/mocks/mock_indexnode.go b/internal/mocks/mock_indexnode.go index 31ab4f35729dd..7e3702e92939b 100644 --- a/internal/mocks/mock_indexnode.go +++ b/internal/mocks/mock_indexnode.go @@ -8,13 +8,13 @@ import ( commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" clientv3 "go.etcd.io/etcd/client/v3" - internalpb "github.com/milvus-io/milvus/internal/proto/internalpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" mock "github.com/stretchr/testify/mock" - workerpb "github.com/milvus-io/milvus/internal/proto/workerpb" + workerpb "github.com/milvus-io/milvus/pkg/proto/workerpb" ) // MockIndexNode is an autogenerated mock type for the IndexNodeComponent type diff --git a/internal/mocks/mock_indexnode_client.go b/internal/mocks/mock_indexnode_client.go index 6e9f8bc81bde6..cde97fd139e86 100644 --- a/internal/mocks/mock_indexnode_client.go +++ b/internal/mocks/mock_indexnode_client.go @@ -9,13 +9,13 @@ import ( grpc "google.golang.org/grpc" - internalpb "github.com/milvus-io/milvus/internal/proto/internalpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" mock "github.com/stretchr/testify/mock" - workerpb "github.com/milvus-io/milvus/internal/proto/workerpb" + workerpb "github.com/milvus-io/milvus/pkg/proto/workerpb" ) // MockIndexNodeClient is an autogenerated mock type for the IndexNodeClient type diff --git a/internal/mocks/mock_metastore/mock_StreamingCoordCataLog.go b/internal/mocks/mock_metastore/mock_StreamingCoordCataLog.go index 651554d48b3f3..27e603960ad82 100644 --- a/internal/mocks/mock_metastore/mock_StreamingCoordCataLog.go +++ b/internal/mocks/mock_metastore/mock_StreamingCoordCataLog.go @@ -7,7 +7,7 @@ import ( mock "github.com/stretchr/testify/mock" - streamingpb "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + streamingpb "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) // MockStreamingCoordCataLog is an autogenerated mock type for the StreamingCoordCataLog type diff --git a/internal/mocks/mock_metastore/mock_StreamingNodeCataLog.go b/internal/mocks/mock_metastore/mock_StreamingNodeCataLog.go index b611f79f84c92..ed3f6d6af42d2 100644 --- a/internal/mocks/mock_metastore/mock_StreamingNodeCataLog.go +++ b/internal/mocks/mock_metastore/mock_StreamingNodeCataLog.go @@ -7,7 +7,7 @@ import ( mock "github.com/stretchr/testify/mock" - streamingpb "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + streamingpb "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) // MockStreamingNodeCataLog is an autogenerated mock type for the StreamingNodeCataLog type diff --git a/internal/mocks/mock_proxy.go b/internal/mocks/mock_proxy.go index 28b1f9815489f..6025dd3cc92bb 100644 --- a/internal/mocks/mock_proxy.go +++ b/internal/mocks/mock_proxy.go @@ -10,13 +10,13 @@ import ( federpb "github.com/milvus-io/milvus-proto/go-api/v2/federpb" - internalpb "github.com/milvus-io/milvus/internal/proto/internalpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" mock "github.com/stretchr/testify/mock" - proxypb "github.com/milvus-io/milvus/internal/proto/proxypb" + proxypb "github.com/milvus-io/milvus/pkg/proto/proxypb" types "github.com/milvus-io/milvus/internal/types" ) diff --git a/internal/mocks/mock_proxy_client.go b/internal/mocks/mock_proxy_client.go index 7d84eb171c370..88326888bbf6d 100644 --- a/internal/mocks/mock_proxy_client.go +++ b/internal/mocks/mock_proxy_client.go @@ -9,13 +9,13 @@ import ( grpc "google.golang.org/grpc" - internalpb "github.com/milvus-io/milvus/internal/proto/internalpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" mock "github.com/stretchr/testify/mock" - proxypb "github.com/milvus-io/milvus/internal/proto/proxypb" + proxypb "github.com/milvus-io/milvus/pkg/proto/proxypb" ) // MockProxyClient is an autogenerated mock type for the ProxyClient type diff --git a/internal/mocks/mock_querycoord.go b/internal/mocks/mock_querycoord.go index b5a1caac71094..50a3cb4e54602 100644 --- a/internal/mocks/mock_querycoord.go +++ b/internal/mocks/mock_querycoord.go @@ -8,13 +8,13 @@ import ( commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" clientv3 "go.etcd.io/etcd/client/v3" - internalpb "github.com/milvus-io/milvus/internal/proto/internalpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" mock "github.com/stretchr/testify/mock" - querypb "github.com/milvus-io/milvus/internal/proto/querypb" + querypb "github.com/milvus-io/milvus/pkg/proto/querypb" txnkv "github.com/tikv/client-go/v2/txnkv" diff --git a/internal/mocks/mock_querycoord_client.go b/internal/mocks/mock_querycoord_client.go index 3285f867a6814..9eabd9153000f 100644 --- a/internal/mocks/mock_querycoord_client.go +++ b/internal/mocks/mock_querycoord_client.go @@ -9,13 +9,13 @@ import ( grpc "google.golang.org/grpc" - internalpb "github.com/milvus-io/milvus/internal/proto/internalpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" mock "github.com/stretchr/testify/mock" - querypb "github.com/milvus-io/milvus/internal/proto/querypb" + querypb "github.com/milvus-io/milvus/pkg/proto/querypb" ) // MockQueryCoordClient is an autogenerated mock type for the QueryCoordClient type diff --git a/internal/mocks/mock_querynode.go b/internal/mocks/mock_querynode.go index a8c32577fa5d6..c434ad119e336 100644 --- a/internal/mocks/mock_querynode.go +++ b/internal/mocks/mock_querynode.go @@ -8,13 +8,13 @@ import ( commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" clientv3 "go.etcd.io/etcd/client/v3" - internalpb "github.com/milvus-io/milvus/internal/proto/internalpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" mock "github.com/stretchr/testify/mock" - querypb "github.com/milvus-io/milvus/internal/proto/querypb" + querypb "github.com/milvus-io/milvus/pkg/proto/querypb" ) // MockQueryNode is an autogenerated mock type for the QueryNodeComponent type diff --git a/internal/mocks/mock_querynode_client.go b/internal/mocks/mock_querynode_client.go index 3b3d465610bec..8ed6720d065e1 100644 --- a/internal/mocks/mock_querynode_client.go +++ b/internal/mocks/mock_querynode_client.go @@ -9,13 +9,13 @@ import ( grpc "google.golang.org/grpc" - internalpb "github.com/milvus-io/milvus/internal/proto/internalpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" mock "github.com/stretchr/testify/mock" - querypb "github.com/milvus-io/milvus/internal/proto/querypb" + querypb "github.com/milvus-io/milvus/pkg/proto/querypb" ) // MockQueryNodeClient is an autogenerated mock type for the QueryNodeClient type diff --git a/internal/mocks/mock_rootcoord.go b/internal/mocks/mock_rootcoord.go index 0c261fcc056c4..2993e494d2b6e 100644 --- a/internal/mocks/mock_rootcoord.go +++ b/internal/mocks/mock_rootcoord.go @@ -10,15 +10,15 @@ import ( grpc "google.golang.org/grpc" - internalpb "github.com/milvus-io/milvus/internal/proto/internalpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" mock "github.com/stretchr/testify/mock" - proxypb "github.com/milvus-io/milvus/internal/proto/proxypb" + proxypb "github.com/milvus-io/milvus/pkg/proto/proxypb" - rootcoordpb "github.com/milvus-io/milvus/internal/proto/rootcoordpb" + rootcoordpb "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" txnkv "github.com/tikv/client-go/v2/txnkv" diff --git a/internal/mocks/mock_rootcoord_client.go b/internal/mocks/mock_rootcoord_client.go index 6542d4ea64545..8d88706ce2231 100644 --- a/internal/mocks/mock_rootcoord_client.go +++ b/internal/mocks/mock_rootcoord_client.go @@ -9,15 +9,15 @@ import ( grpc "google.golang.org/grpc" - internalpb "github.com/milvus-io/milvus/internal/proto/internalpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" mock "github.com/stretchr/testify/mock" - proxypb "github.com/milvus-io/milvus/internal/proto/proxypb" + proxypb "github.com/milvus-io/milvus/pkg/proto/proxypb" - rootcoordpb "github.com/milvus-io/milvus/internal/proto/rootcoordpb" + rootcoordpb "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" ) // MockRootCoordClient is an autogenerated mock type for the RootCoordClient type diff --git a/internal/mocks/util/mock_segcore/mock_data.go b/internal/mocks/util/mock_segcore/mock_data.go index 0d7ed4b137626..621209ea00614 100644 --- a/internal/mocks/util/mock_segcore/mock_data.go +++ b/internal/mocks/util/mock_segcore/mock_data.go @@ -34,19 +34,19 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "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/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/etcdpb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/planpb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" storage "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/indexcgowrapper" "github.com/milvus-io/milvus/internal/util/segcore" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/etcdpb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/metautil" "github.com/milvus-io/milvus/pkg/util/metric" diff --git a/internal/parser/planparserv2/check_identical.go b/internal/parser/planparserv2/check_identical.go index faa6efd022f2c..15808f322e39f 100644 --- a/internal/parser/planparserv2/check_identical.go +++ b/internal/parser/planparserv2/check_identical.go @@ -3,7 +3,7 @@ package planparserv2 import ( "reflect" - "github.com/milvus-io/milvus/internal/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" "github.com/milvus-io/milvus/pkg/util/funcutil" ) diff --git a/internal/parser/planparserv2/check_identical_test.go b/internal/parser/planparserv2/check_identical_test.go index 249e9035649c7..7df0bd2e119ed 100644 --- a/internal/parser/planparserv2/check_identical_test.go +++ b/internal/parser/planparserv2/check_identical_test.go @@ -5,7 +5,7 @@ import ( "github.com/stretchr/testify/assert" - "github.com/milvus-io/milvus/internal/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/parser/planparserv2/convert_field_data_to_generic_value.go b/internal/parser/planparserv2/convert_field_data_to_generic_value.go index d00f9dc6c93ee..31482b71b3cf8 100644 --- a/internal/parser/planparserv2/convert_field_data_to_generic_value.go +++ b/internal/parser/planparserv2/convert_field_data_to_generic_value.go @@ -6,7 +6,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/json" - "github.com/milvus-io/milvus/internal/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" ) func convertArrayValue(templateName string, templateValue *schemapb.TemplateArrayValue) (*planpb.GenericValue, error) { diff --git a/internal/parser/planparserv2/convert_field_data_to_generic_value_test.go b/internal/parser/planparserv2/convert_field_data_to_generic_value_test.go index 89a50a7707d60..339d2d8dcbb15 100644 --- a/internal/parser/planparserv2/convert_field_data_to_generic_value_test.go +++ b/internal/parser/planparserv2/convert_field_data_to_generic_value_test.go @@ -7,7 +7,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/json" - "github.com/milvus-io/milvus/internal/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" ) type convertTestcase struct { diff --git a/internal/parser/planparserv2/fill_expression_value.go b/internal/parser/planparserv2/fill_expression_value.go index 8840ac75b08bf..cf9e4dc95be82 100644 --- a/internal/parser/planparserv2/fill_expression_value.go +++ b/internal/parser/planparserv2/fill_expression_value.go @@ -4,7 +4,7 @@ import ( "fmt" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/parser/planparserv2/logical_expr_visitor.go b/internal/parser/planparserv2/logical_expr_visitor.go index ce1d11b7bbb38..92b5920b6a7ef 100644 --- a/internal/parser/planparserv2/logical_expr_visitor.go +++ b/internal/parser/planparserv2/logical_expr_visitor.go @@ -1,6 +1,6 @@ package planparserv2 -import "github.com/milvus-io/milvus/internal/proto/planpb" +import "github.com/milvus-io/milvus/pkg/proto/planpb" type LogicalExprVisitor interface { VisitExpr(expr *planpb.Expr) interface{} diff --git a/internal/parser/planparserv2/node_ret.go b/internal/parser/planparserv2/node_ret.go index 456b190311ff1..e76587de1eb3f 100644 --- a/internal/parser/planparserv2/node_ret.go +++ b/internal/parser/planparserv2/node_ret.go @@ -2,7 +2,7 @@ package planparserv2 import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" ) type ExprWithType struct { diff --git a/internal/parser/planparserv2/operators.go b/internal/parser/planparserv2/operators.go index bf5d0fbd180ad..3d0f2c58a07a4 100644 --- a/internal/parser/planparserv2/operators.go +++ b/internal/parser/planparserv2/operators.go @@ -6,7 +6,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" parser "github.com/milvus-io/milvus/internal/parser/planparserv2/generated" - "github.com/milvus-io/milvus/internal/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" ) var arithExprMap = map[int]planpb.ArithOpType{ diff --git a/internal/parser/planparserv2/parser_visitor.go b/internal/parser/planparserv2/parser_visitor.go index fa7363c39151d..95fa0c4f14945 100644 --- a/internal/parser/planparserv2/parser_visitor.go +++ b/internal/parser/planparserv2/parser_visitor.go @@ -9,7 +9,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" parser "github.com/milvus-io/milvus/internal/parser/planparserv2/generated" - "github.com/milvus-io/milvus/internal/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/parser/planparserv2/pattern_match.go b/internal/parser/planparserv2/pattern_match.go index 1bd6dbe6ec2b2..e74f81f85fd45 100644 --- a/internal/parser/planparserv2/pattern_match.go +++ b/internal/parser/planparserv2/pattern_match.go @@ -1,7 +1,7 @@ package planparserv2 import ( - "github.com/milvus-io/milvus/internal/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" ) var wildcards = map[byte]struct{}{ diff --git a/internal/parser/planparserv2/pattern_match_test.go b/internal/parser/planparserv2/pattern_match_test.go index f0cdd48b9fc0a..fb97de29c7282 100644 --- a/internal/parser/planparserv2/pattern_match_test.go +++ b/internal/parser/planparserv2/pattern_match_test.go @@ -3,7 +3,7 @@ package planparserv2 import ( "testing" - "github.com/milvus-io/milvus/internal/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" ) func Test_hasWildcards(t *testing.T) { diff --git a/internal/parser/planparserv2/plan_parser_v2.go b/internal/parser/planparserv2/plan_parser_v2.go index 0c8e9f73561f4..89ae5344e4f34 100644 --- a/internal/parser/planparserv2/plan_parser_v2.go +++ b/internal/parser/planparserv2/plan_parser_v2.go @@ -11,8 +11,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" planparserv2 "github.com/milvus-io/milvus/internal/parser/planparserv2/generated" - "github.com/milvus-io/milvus/internal/proto/planpb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/planpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/parser/planparserv2/plan_parser_v2_test.go b/internal/parser/planparserv2/plan_parser_v2_test.go index 5642d7e2dd361..b9c9a737c4d73 100644 --- a/internal/parser/planparserv2/plan_parser_v2_test.go +++ b/internal/parser/planparserv2/plan_parser_v2_test.go @@ -12,8 +12,8 @@ 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/proto/planpb" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/planpb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/parser/planparserv2/show_visitor.go b/internal/parser/planparserv2/show_visitor.go index f109d87e021f5..ff8d80394943a 100644 --- a/internal/parser/planparserv2/show_visitor.go +++ b/internal/parser/planparserv2/show_visitor.go @@ -4,8 +4,8 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus/internal/json" - "github.com/milvus-io/milvus/internal/proto/planpb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/planpb" ) type ShowExprVisitor struct{} diff --git a/internal/parser/planparserv2/utils.go b/internal/parser/planparserv2/utils.go index 4cb1e60dbe0df..2dea26cd30b4e 100644 --- a/internal/parser/planparserv2/utils.go +++ b/internal/parser/planparserv2/utils.go @@ -9,7 +9,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/json" - "github.com/milvus-io/milvus/internal/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/parser/planparserv2/utils_test.go b/internal/parser/planparserv2/utils_test.go index 91bfaeba098e2..6902b47526a54 100644 --- a/internal/parser/planparserv2/utils_test.go +++ b/internal/parser/planparserv2/utils_test.go @@ -6,7 +6,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" ) func Test_relationalCompatible(t *testing.T) { diff --git a/internal/proxy/count_reducer.go b/internal/proxy/count_reducer.go index 7c8cdd7e691e4..c1dc96edd504f 100644 --- a/internal/proxy/count_reducer.go +++ b/internal/proxy/count_reducer.go @@ -2,8 +2,8 @@ package proxy import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/util/funcutil" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/proxy/count_reducer_test.go b/internal/proxy/count_reducer_test.go index 4e1fd436777a7..814823cc9aaae 100644 --- a/internal/proxy/count_reducer_test.go +++ b/internal/proxy/count_reducer_test.go @@ -6,8 +6,8 @@ import ( "github.com/stretchr/testify/assert" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/util/funcutil" + "github.com/milvus-io/milvus/pkg/proto/internalpb" ) func Test_cntReducer_Reduce(t *testing.T) { diff --git a/internal/proxy/data_coord_mock_test.go b/internal/proxy/data_coord_mock_test.go index e89e9c838fc7e..0d2f8cdce9a4c 100644 --- a/internal/proxy/data_coord_mock_test.go +++ b/internal/proxy/data_coord_mock_test.go @@ -24,10 +24,10 @@ 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/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/proxy/default_limit_reducer.go b/internal/proxy/default_limit_reducer.go index 0f70f49bd1ccb..ee95d2aa3f355 100644 --- a/internal/proxy/default_limit_reducer.go +++ b/internal/proxy/default_limit_reducer.go @@ -5,8 +5,8 @@ import ( "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/proto/internalpb" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/proxy/http_req_impl.go b/internal/proxy/http_req_impl.go index 0dfcce4d78e93..6b202b09b03cc 100644 --- a/internal/proxy/http_req_impl.go +++ b/internal/proxy/http_req_impl.go @@ -28,10 +28,10 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" mhttp "github.com/milvus-io/milvus/internal/http" "github.com/milvus-io/milvus/internal/json" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proxy/connection" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/dependency" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/proxy/http_req_impl_test.go b/internal/proxy/http_req_impl_test.go index a81199a6e5613..7ccc5f5edd9c3 100644 --- a/internal/proxy/http_req_impl_test.go +++ b/internal/proxy/http_req_impl_test.go @@ -15,8 +15,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proxy/connection" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/proxy/impl.go b/internal/proxy/impl.go index edb5eb65b1b0f..e4734c2cf3c0a 100644 --- a/internal/proxy/impl.go +++ b/internal/proxy/impl.go @@ -41,10 +41,6 @@ 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/http" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/proxypb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proxy/connection" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/hookutil" @@ -54,6 +50,10 @@ import ( "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/proxypb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/crypto" diff --git a/internal/proxy/impl_test.go b/internal/proxy/impl_test.go index ba40366211c62..cdb47be48454f 100644 --- a/internal/proxy/impl_test.go +++ b/internal/proxy/impl_test.go @@ -41,17 +41,17 @@ import ( "github.com/milvus-io/milvus/internal/allocator" mhttp "github.com/milvus-io/milvus/internal/http" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/proxypb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" mqcommon "github.com/milvus-io/milvus/pkg/mq/common" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/proxypb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/proxy/interface_def.go b/internal/proxy/interface_def.go index e9551129bb60c..8766270e35d49 100644 --- a/internal/proxy/interface_def.go +++ b/internal/proxy/interface_def.go @@ -21,7 +21,7 @@ import ( "google.golang.org/grpc" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" ) // use interface tsoAllocator to keep other components testable diff --git a/internal/proxy/lb_balancer.go b/internal/proxy/lb_balancer.go index 3d46cb63c0f19..6cc8aea486cdc 100644 --- a/internal/proxy/lb_balancer.go +++ b/internal/proxy/lb_balancer.go @@ -19,7 +19,7 @@ package proxy import ( "context" - "github.com/milvus-io/milvus/internal/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" ) type LBBalancer interface { diff --git a/internal/proxy/lb_policy.go b/internal/proxy/lb_policy.go index e980d625813f9..d069ddbf917f7 100644 --- a/internal/proxy/lb_policy.go +++ b/internal/proxy/lb_policy.go @@ -23,10 +23,10 @@ import ( "go.uber.org/zap" "golang.org/x/sync/errgroup" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/querycoordv2/params" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/retry" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/proxy/lb_policy_test.go b/internal/proxy/lb_policy_test.go index c17388d40212b..086b50449cd92 100644 --- a/internal/proxy/lb_policy_test.go +++ b/internal/proxy/lb_policy_test.go @@ -33,10 +33,10 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/proxy/look_aside_balancer.go b/internal/proxy/look_aside_balancer.go index 0aa26a78a7599..ac5bcdab3cd94 100644 --- a/internal/proxy/look_aside_balancer.go +++ b/internal/proxy/look_aside_balancer.go @@ -27,8 +27,8 @@ 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/internal/proto/internalpb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/proxy/look_aside_balancer_test.go b/internal/proxy/look_aside_balancer_test.go index 20cc7c51a6892..72cad81f007ab 100644 --- a/internal/proxy/look_aside_balancer_test.go +++ b/internal/proxy/look_aside_balancer_test.go @@ -29,8 +29,8 @@ 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/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/proxy/management.go b/internal/proxy/management.go index 635e516066a45..99bf1ea80820d 100644 --- a/internal/proxy/management.go +++ b/internal/proxy/management.go @@ -25,8 +25,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" management "github.com/milvus-io/milvus/internal/http" "github.com/milvus-io/milvus/internal/json" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/proxy/management_test.go b/internal/proxy/management_test.go index ed652c5383cce..60e8c29907502 100644 --- a/internal/proxy/management_test.go +++ b/internal/proxy/management_test.go @@ -31,8 +31,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" management "github.com/milvus-io/milvus/internal/http" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/proxy/meta_cache.go b/internal/proxy/meta_cache.go index feec9752f78bb..a8216ba7e135f 100644 --- a/internal/proxy/meta_cache.go +++ b/internal/proxy/meta_cache.go @@ -32,13 +32,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/schemapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/conc" diff --git a/internal/proxy/meta_cache_test.go b/internal/proxy/meta_cache_test.go index 344363538892a..ec570d9a78649 100644 --- a/internal/proxy/meta_cache_test.go +++ b/internal/proxy/meta_cache_test.go @@ -35,11 +35,11 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/crypto" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/proxy/metrics_info.go b/internal/proxy/metrics_info.go index 3c31f33617a96..d17c20ac424ef 100644 --- a/internal/proxy/metrics_info.go +++ b/internal/proxy/metrics_info.go @@ -23,8 +23,8 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/hardware" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" diff --git a/internal/proxy/mock_cache.go b/internal/proxy/mock_cache.go index cbd4039d1f760..647bd55eaf755 100644 --- a/internal/proxy/mock_cache.go +++ b/internal/proxy/mock_cache.go @@ -5,7 +5,7 @@ package proxy import ( context "context" - internalpb "github.com/milvus-io/milvus/internal/proto/internalpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" mock "github.com/stretchr/testify/mock" typeutil "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/proxy/mock_lb_balancer.go b/internal/proxy/mock_lb_balancer.go index 4d99213078f23..9a729786e0505 100644 --- a/internal/proxy/mock_lb_balancer.go +++ b/internal/proxy/mock_lb_balancer.go @@ -5,7 +5,7 @@ package proxy import ( context "context" - internalpb "github.com/milvus-io/milvus/internal/proto/internalpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" mock "github.com/stretchr/testify/mock" ) diff --git a/internal/proxy/mock_lb_policy.go b/internal/proxy/mock_lb_policy.go index 44c30bdbef3f0..6492c75e5cd0c 100644 --- a/internal/proxy/mock_lb_policy.go +++ b/internal/proxy/mock_lb_policy.go @@ -5,7 +5,7 @@ package proxy import ( context "context" - internalpb "github.com/milvus-io/milvus/internal/proto/internalpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" mock "github.com/stretchr/testify/mock" ) diff --git a/internal/proxy/mock_test.go b/internal/proxy/mock_test.go index 18da0945d6bd5..8932bcf9c2a16 100644 --- a/internal/proxy/mock_test.go +++ b/internal/proxy/mock_test.go @@ -26,9 +26,9 @@ 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/allocator" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/mq/common" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/proxy/mock_tso_test.go b/internal/proxy/mock_tso_test.go index 131e9ab6881fd..b69029b21787e 100644 --- a/internal/proxy/mock_tso_test.go +++ b/internal/proxy/mock_tso_test.go @@ -9,7 +9,7 @@ import ( mock "github.com/stretchr/testify/mock" - rootcoordpb "github.com/milvus-io/milvus/internal/proto/rootcoordpb" + rootcoordpb "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" ) // mockTimestampAllocator is an autogenerated mock type for the timestampAllocatorInterface type diff --git a/internal/proxy/msg_pack_test.go b/internal/proxy/msg_pack_test.go index 9d199dd166171..6f3d5f90a02b9 100644 --- a/internal/proxy/msg_pack_test.go +++ b/internal/proxy/msg_pack_test.go @@ -30,8 +30,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/testutils" diff --git a/internal/proxy/privilege_interceptor_test.go b/internal/proxy/privilege_interceptor_test.go index fd1aef75da32a..272864e195e7d 100644 --- a/internal/proxy/privilege_interceptor_test.go +++ b/internal/proxy/privilege_interceptor_test.go @@ -11,7 +11,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/proxy/proxy.go b/internal/proxy/proxy.go index 4c1d98773fae5..14876a237fedc 100644 --- a/internal/proxy/proxy.go +++ b/internal/proxy/proxy.go @@ -35,7 +35,6 @@ 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/internal/allocator" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proxy/connection" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/dependency" @@ -45,6 +44,7 @@ import ( "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/expr" "github.com/milvus-io/milvus/pkg/util/logutil" diff --git a/internal/proxy/proxy_rpc_test.go b/internal/proxy/proxy_rpc_test.go index ae49378025535..9cbc44617bc3c 100644 --- a/internal/proxy/proxy_rpc_test.go +++ b/internal/proxy/proxy_rpc_test.go @@ -11,10 +11,10 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" grpcproxyclient "github.com/milvus-io/milvus/internal/distributed/proxy/client" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/proxypb" "github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/proxypb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/proxy/proxy_test.go b/internal/proxy/proxy_test.go index b8eed64bdf20c..2f39ba4265e21 100644 --- a/internal/proxy/proxy_test.go +++ b/internal/proxy/proxy_test.go @@ -54,16 +54,16 @@ import ( rcc "github.com/milvus-io/milvus/internal/distributed/rootcoord/client" "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/proxypb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/util/componentutil" "github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/proxypb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/tracer" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/crypto" diff --git a/internal/proxy/rate_limit_interceptor_test.go b/internal/proxy/rate_limit_interceptor_test.go index 22a5c98c326c9..74ddf4f6c3e37 100644 --- a/internal/proxy/rate_limit_interceptor_test.go +++ b/internal/proxy/rate_limit_interceptor_test.go @@ -28,7 +28,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" - "github.com/milvus-io/milvus/internal/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/proxy/reducer.go b/internal/proxy/reducer.go index 3821047e5156d..857785728df97 100644 --- a/internal/proxy/reducer.go +++ b/internal/proxy/reducer.go @@ -5,8 +5,8 @@ import ( "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/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" ) type milvusReducer interface { diff --git a/internal/proxy/reducer_test.go b/internal/proxy/reducer_test.go index 668a94ce97e3e..e6b3c1377e762 100644 --- a/internal/proxy/reducer_test.go +++ b/internal/proxy/reducer_test.go @@ -6,7 +6,7 @@ import ( "github.com/stretchr/testify/assert" - "github.com/milvus-io/milvus/internal/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" ) func Test_createMilvusReducer(t *testing.T) { diff --git a/internal/proxy/rootcoord_mock_test.go b/internal/proxy/rootcoord_mock_test.go index 513c9412007dc..99c5ad45a5db9 100644 --- a/internal/proxy/rootcoord_mock_test.go +++ b/internal/proxy/rootcoord_mock_test.go @@ -30,12 +30,12 @@ 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/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/proxypb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/proxypb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/proxy/roundrobin_balancer.go b/internal/proxy/roundrobin_balancer.go index e8fe18cf38061..59d28b01060be 100644 --- a/internal/proxy/roundrobin_balancer.go +++ b/internal/proxy/roundrobin_balancer.go @@ -20,7 +20,7 @@ import ( "go.uber.org/atomic" - "github.com/milvus-io/milvus/internal/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/proxy/search_util.go b/internal/proxy/search_util.go index c54b7e66d9fe5..c3a8254827d45 100644 --- a/internal/proxy/search_util.go +++ b/internal/proxy/search_util.go @@ -14,8 +14,8 @@ 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/proto/planpb" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/planpb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/proxy/segment.go b/internal/proxy/segment.go index d6cdcb97e0c9a..a0adb2c01abaf 100644 --- a/internal/proxy/segment.go +++ b/internal/proxy/segment.go @@ -29,9 +29,9 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus/internal/allocator" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/proxy/segment_test.go b/internal/proxy/segment_test.go index eeafe3b3170ef..009d8e522a60e 100644 --- a/internal/proxy/segment_test.go +++ b/internal/proxy/segment_test.go @@ -28,7 +28,7 @@ import ( "google.golang.org/grpc" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/proxy/simple_rate_limiter.go b/internal/proxy/simple_rate_limiter.go index 4881c2a83004f..77fa3430f78f4 100644 --- a/internal/proxy/simple_rate_limiter.go +++ b/internal/proxy/simple_rate_limiter.go @@ -27,12 +27,12 @@ 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/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/proxypb" "github.com/milvus-io/milvus/internal/util/quota" rlinternal "github.com/milvus-io/milvus/internal/util/ratelimitutil" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/proxypb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/ratelimitutil" diff --git a/internal/proxy/simple_rate_limiter_test.go b/internal/proxy/simple_rate_limiter_test.go index cbbe248dc0465..b53e717cd95ef 100644 --- a/internal/proxy/simple_rate_limiter_test.go +++ b/internal/proxy/simple_rate_limiter_test.go @@ -26,9 +26,9 @@ 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/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/proxypb" rlinternal "github.com/milvus-io/milvus/internal/util/ratelimitutil" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/proxypb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/ratelimitutil" diff --git a/internal/proxy/task.go b/internal/proxy/task.go index 786e2171eb1f4..99ca81549699a 100644 --- a/internal/proxy/task.go +++ b/internal/proxy/task.go @@ -30,14 +30,14 @@ 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/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/ctokenizer" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/proxy/task_database.go b/internal/proxy/task_database.go index bee84860d3c04..442ff4fa67cc9 100644 --- a/internal/proxy/task_database.go +++ b/internal/proxy/task_database.go @@ -8,11 +8,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/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/proxy/task_database_test.go b/internal/proxy/task_database_test.go index 3be8c4a8a76e6..73d67c39842a1 100644 --- a/internal/proxy/task_database_test.go +++ b/internal/proxy/task_database_test.go @@ -13,8 +13,8 @@ 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/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/crypto" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/proxy/task_delete.go b/internal/proxy/task_delete.go index 6c0005e6954c5..98b990d09d688 100644 --- a/internal/proxy/task_delete.go +++ b/internal/proxy/task_delete.go @@ -17,9 +17,6 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/parser/planparserv2" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/planpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/exprutil" "github.com/milvus-io/milvus/internal/util/streamingutil" @@ -27,6 +24,9 @@ import ( "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/proxy/task_delete_test.go b/internal/proxy/task_delete_test.go index df5eff3246cb7..e3bb764398108 100644 --- a/internal/proxy/task_delete_test.go +++ b/internal/proxy/task_delete_test.go @@ -17,13 +17,13 @@ import ( "github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/mocks" "github.com/milvus-io/milvus/internal/parser/planparserv2" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/planpb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/util/streamrpc" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/proxy/task_flush.go b/internal/proxy/task_flush.go index 44beeccb72c48..7bc3500dff6e2 100644 --- a/internal/proxy/task_flush.go +++ b/internal/proxy/task_flush.go @@ -24,9 +24,9 @@ import ( "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/proto/datapb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/proxy/task_flush_streaming.go b/internal/proxy/task_flush_streaming.go index 8f122ee2e4fc9..245cab5c0439d 100644 --- a/internal/proxy/task_flush_streaming.go +++ b/internal/proxy/task_flush_streaming.go @@ -27,8 +27,8 @@ 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/distributed/streaming" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/proxy/task_index.go b/internal/proxy/task_index.go index f722842d8b411..5afeae1a17c7d 100644 --- a/internal/proxy/task_index.go +++ b/internal/proxy/task_index.go @@ -26,13 +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/schemapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/indexparamcheck" "github.com/milvus-io/milvus/internal/util/vecindexmgr" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/indexpb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/indexparams" diff --git a/internal/proxy/task_index_test.go b/internal/proxy/task_index_test.go index 780dea5041f4f..f169143d80a61 100644 --- a/internal/proxy/task_index_test.go +++ b/internal/proxy/task_index_test.go @@ -33,10 +33,10 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/util/indexparamcheck" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metric" diff --git a/internal/proxy/task_query.go b/internal/proxy/task_query.go index 9e43e1d3ef42b..3c738eef23d1f 100644 --- a/internal/proxy/task_query.go +++ b/internal/proxy/task_query.go @@ -15,9 +15,6 @@ import ( "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/parser/planparserv2" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/planpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/exprutil" "github.com/milvus-io/milvus/internal/util/reduce" @@ -25,6 +22,9 @@ import ( "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/proxy/task_query_test.go b/internal/proxy/task_query_test.go index 36c3b9ef49dda..244d091075217 100644 --- a/internal/proxy/task_query_test.go +++ b/internal/proxy/task_query_test.go @@ -32,11 +32,11 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/util/reduce" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/proxy/task_scheduler_test.go b/internal/proxy/task_scheduler_test.go index 23f405ac47a41..7c8a0276113dc 100644 --- a/internal/proxy/task_scheduler_test.go +++ b/internal/proxy/task_scheduler_test.go @@ -30,8 +30,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/funcutil" ) diff --git a/internal/proxy/task_search.go b/internal/proxy/task_search.go index ffa7c9b23b8ee..2907fb981e482 100644 --- a/internal/proxy/task_search.go +++ b/internal/proxy/task_search.go @@ -17,14 +17,14 @@ import ( "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/parser/planparserv2" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/planpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/exprutil" "github.com/milvus-io/milvus/internal/util/reduce" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/proxy/task_search_test.go b/internal/proxy/task_search_test.go index 1edf764c8b418..04f5bec9978e8 100644 --- a/internal/proxy/task_search_test.go +++ b/internal/proxy/task_search_test.go @@ -37,13 +37,13 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/planpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/internal/util/reduce" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metric" diff --git a/internal/proxy/task_statistic.go b/internal/proxy/task_statistic.go index 49d176d002702..5a1e9136e24d4 100644 --- a/internal/proxy/task_statistic.go +++ b/internal/proxy/task_statistic.go @@ -12,11 +12,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/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/proxy/task_statistic_test.go b/internal/proxy/task_statistic_test.go index e3d3786cfe97c..bdfc99887847a 100644 --- a/internal/proxy/task_statistic_test.go +++ b/internal/proxy/task_statistic_test.go @@ -28,10 +28,10 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/proxy/task_test.go b/internal/proxy/task_test.go index 019063bbbc9f1..13840898b51dd 100644 --- a/internal/proxy/task_test.go +++ b/internal/proxy/task_test.go @@ -38,13 +38,13 @@ import ( "github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/proxy/timestamp.go b/internal/proxy/timestamp.go index 7078b809c64b9..e368bd3a07da4 100644 --- a/internal/proxy/timestamp.go +++ b/internal/proxy/timestamp.go @@ -23,8 +23,8 @@ import ( "time" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/timerecord" diff --git a/internal/proxy/util.go b/internal/proxy/util.go index 1a5bb46265118..16e01cae8b7f1 100644 --- a/internal/proxy/util.go +++ b/internal/proxy/util.go @@ -35,9 +35,6 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/parser/planparserv2" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/planpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/hookutil" "github.com/milvus-io/milvus/internal/util/indexparamcheck" @@ -45,6 +42,9 @@ import ( "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/contextutil" diff --git a/internal/proxy/util_test.go b/internal/proxy/util_test.go index a4824e7ef11f4..f8dac8db34e15 100644 --- a/internal/proxy/util_test.go +++ b/internal/proxy/util_test.go @@ -37,12 +37,12 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/crypto" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/querycoordv2/.mockery.yaml b/internal/querycoordv2/.mockery.yaml index 11302e864319a..ab9c1ee5d5821 100644 --- a/internal/querycoordv2/.mockery.yaml +++ b/internal/querycoordv2/.mockery.yaml @@ -28,7 +28,7 @@ packages: github.com/milvus-io/milvus/internal/querycoordv2/task: interfaces: Scheduler: - github.com/milvus-io/milvus/internal/proto/querypb: + github.com/milvus-io/milvus/pkg/proto/querypb: interfaces: QueryNodeServer: config: diff --git a/internal/querycoordv2/balance/balance_test.go b/internal/querycoordv2/balance/balance_test.go index 0aaf5f3bb0706..6a7df24e0d4c0 100644 --- a/internal/querycoordv2/balance/balance_test.go +++ b/internal/querycoordv2/balance/balance_test.go @@ -23,11 +23,11 @@ import ( "github.com/stretchr/testify/mock" "github.com/stretchr/testify/suite" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/internal/querycoordv2/session" "github.com/milvus-io/milvus/internal/querycoordv2/task" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querycoordv2/balance/channel_level_score_balancer_test.go b/internal/querycoordv2/balance/channel_level_score_balancer_test.go index 2b8f0ff79b4f4..44dae9a1c9366 100644 --- a/internal/querycoordv2/balance/channel_level_score_balancer_test.go +++ b/internal/querycoordv2/balance/channel_level_score_balancer_test.go @@ -25,8 +25,6 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/metastore/kv/querycoord" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" . "github.com/milvus-io/milvus/internal/querycoordv2/params" "github.com/milvus-io/milvus/internal/querycoordv2/session" @@ -34,6 +32,8 @@ import ( "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/kv" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querycoordv2/balance/multi_target_balancer_test.go b/internal/querycoordv2/balance/multi_target_balancer_test.go index bdf837e3d3b8e..ba3be0661394d 100644 --- a/internal/querycoordv2/balance/multi_target_balancer_test.go +++ b/internal/querycoordv2/balance/multi_target_balancer_test.go @@ -8,8 +8,8 @@ import ( "github.com/samber/lo" "github.com/stretchr/testify/suite" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querycoordv2/balance/rowcount_based_balancer_test.go b/internal/querycoordv2/balance/rowcount_based_balancer_test.go index 628c5159ca9cf..d49ca4c4bd0b5 100644 --- a/internal/querycoordv2/balance/rowcount_based_balancer_test.go +++ b/internal/querycoordv2/balance/rowcount_based_balancer_test.go @@ -27,8 +27,6 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/metastore/kv/querycoord" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" . "github.com/milvus-io/milvus/internal/querycoordv2/params" "github.com/milvus-io/milvus/internal/querycoordv2/session" @@ -36,6 +34,8 @@ import ( "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/kv" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/querycoordv2/balance/score_based_balancer_test.go b/internal/querycoordv2/balance/score_based_balancer_test.go index abb41ffb74830..3bf70e7894407 100644 --- a/internal/querycoordv2/balance/score_based_balancer_test.go +++ b/internal/querycoordv2/balance/score_based_balancer_test.go @@ -26,8 +26,6 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/metastore/kv/querycoord" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" . "github.com/milvus-io/milvus/internal/querycoordv2/params" "github.com/milvus-io/milvus/internal/querycoordv2/session" @@ -35,6 +33,8 @@ import ( "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/kv" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querycoordv2/balance/utils.go b/internal/querycoordv2/balance/utils.go index 9ea06653032f3..f248728e1099c 100644 --- a/internal/querycoordv2/balance/utils.go +++ b/internal/querycoordv2/balance/utils.go @@ -23,10 +23,10 @@ import ( "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/internal/querycoordv2/task" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/querypb" ) const ( diff --git a/internal/querycoordv2/checkers/balance_checker.go b/internal/querycoordv2/checkers/balance_checker.go index 23f7aadb45f46..6664def8c0526 100644 --- a/internal/querycoordv2/checkers/balance_checker.go +++ b/internal/querycoordv2/checkers/balance_checker.go @@ -24,7 +24,6 @@ import ( "github.com/samber/lo" "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/balance" "github.com/milvus-io/milvus/internal/querycoordv2/meta" . "github.com/milvus-io/milvus/internal/querycoordv2/params" @@ -33,6 +32,7 @@ import ( "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querycoordv2/checkers/balance_checker_test.go b/internal/querycoordv2/checkers/balance_checker_test.go index 45e6eb3b9e8af..cdadbecd51468 100644 --- a/internal/querycoordv2/checkers/balance_checker_test.go +++ b/internal/querycoordv2/checkers/balance_checker_test.go @@ -25,8 +25,6 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/metastore/kv/querycoord" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/balance" "github.com/milvus-io/milvus/internal/querycoordv2/meta" . "github.com/milvus-io/milvus/internal/querycoordv2/params" @@ -34,6 +32,8 @@ import ( "github.com/milvus-io/milvus/internal/querycoordv2/task" "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/kv" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querycoordv2/checkers/channel_checker_test.go b/internal/querycoordv2/checkers/channel_checker_test.go index 2012f3185ba17..bf6d72aecdf08 100644 --- a/internal/querycoordv2/checkers/channel_checker_test.go +++ b/internal/querycoordv2/checkers/channel_checker_test.go @@ -26,7 +26,6 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/metastore/kv/querycoord" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/querycoordv2/balance" "github.com/milvus-io/milvus/internal/querycoordv2/meta" . "github.com/milvus-io/milvus/internal/querycoordv2/params" @@ -34,6 +33,7 @@ import ( "github.com/milvus-io/milvus/internal/querycoordv2/task" "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/kv" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querycoordv2/checkers/controller_test.go b/internal/querycoordv2/checkers/controller_test.go index 1caef1db720a1..004ed616654c5 100644 --- a/internal/querycoordv2/checkers/controller_test.go +++ b/internal/querycoordv2/checkers/controller_test.go @@ -27,7 +27,6 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/metastore/kv/querycoord" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/querycoordv2/balance" "github.com/milvus-io/milvus/internal/querycoordv2/meta" . "github.com/milvus-io/milvus/internal/querycoordv2/params" @@ -35,6 +34,7 @@ import ( "github.com/milvus-io/milvus/internal/querycoordv2/task" "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/kv" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querycoordv2/checkers/index_checker.go b/internal/querycoordv2/checkers/index_checker.go index 970ef6fb4b9a1..99fc36abf395b 100644 --- a/internal/querycoordv2/checkers/index_checker.go +++ b/internal/querycoordv2/checkers/index_checker.go @@ -23,15 +23,15 @@ import ( "github.com/samber/lo" "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/internal/querycoordv2/params" "github.com/milvus-io/milvus/internal/querycoordv2/session" "github.com/milvus-io/milvus/internal/querycoordv2/task" "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querycoordv2/checkers/index_checker_test.go b/internal/querycoordv2/checkers/index_checker_test.go index 2a6a7b99f232d..1747698aaed40 100644 --- a/internal/querycoordv2/checkers/index_checker_test.go +++ b/internal/querycoordv2/checkers/index_checker_test.go @@ -26,15 +26,15 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/metastore/kv/querycoord" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/internal/querycoordv2/params" "github.com/milvus-io/milvus/internal/querycoordv2/session" "github.com/milvus-io/milvus/internal/querycoordv2/task" "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/kv" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querycoordv2/checkers/leader_checker.go b/internal/querycoordv2/checkers/leader_checker.go index 320ac743e8073..756e842d8c250 100644 --- a/internal/querycoordv2/checkers/leader_checker.go +++ b/internal/querycoordv2/checkers/leader_checker.go @@ -22,13 +22,13 @@ import ( "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/internal/querycoordv2/session" "github.com/milvus-io/milvus/internal/querycoordv2/task" "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" ) var _ Checker = (*LeaderChecker)(nil) diff --git a/internal/querycoordv2/checkers/leader_checker_test.go b/internal/querycoordv2/checkers/leader_checker_test.go index 2a4a238cefa2c..ff0daaa76e66e 100644 --- a/internal/querycoordv2/checkers/leader_checker_test.go +++ b/internal/querycoordv2/checkers/leader_checker_test.go @@ -26,14 +26,14 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/metastore/kv/querycoord" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" . "github.com/milvus-io/milvus/internal/querycoordv2/params" "github.com/milvus-io/milvus/internal/querycoordv2/session" "github.com/milvus-io/milvus/internal/querycoordv2/task" "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/kv" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querycoordv2/checkers/segment_checker.go b/internal/querycoordv2/checkers/segment_checker.go index ca94cff11e4d0..ca6d108c5c953 100644 --- a/internal/querycoordv2/checkers/segment_checker.go +++ b/internal/querycoordv2/checkers/segment_checker.go @@ -26,8 +26,6 @@ import ( "go.opentelemetry.io/otel/trace" "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/balance" "github.com/milvus-io/milvus/internal/querycoordv2/meta" . "github.com/milvus-io/milvus/internal/querycoordv2/params" @@ -36,6 +34,8 @@ import ( "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/funcutil" ) diff --git a/internal/querycoordv2/checkers/segment_checker_test.go b/internal/querycoordv2/checkers/segment_checker_test.go index 08f586648dfe4..30000c18016a1 100644 --- a/internal/querycoordv2/checkers/segment_checker_test.go +++ b/internal/querycoordv2/checkers/segment_checker_test.go @@ -27,7 +27,6 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/metastore/kv/querycoord" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/querycoordv2/balance" "github.com/milvus-io/milvus/internal/querycoordv2/meta" . "github.com/milvus-io/milvus/internal/querycoordv2/params" @@ -36,6 +35,7 @@ import ( "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/kv" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querycoordv2/dist/dist_controller_test.go b/internal/querycoordv2/dist/dist_controller_test.go index 702a8608977fb..352cfbe1c3186 100644 --- a/internal/querycoordv2/dist/dist_controller_test.go +++ b/internal/querycoordv2/dist/dist_controller_test.go @@ -27,12 +27,12 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/metastore/kv/querycoord" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" . "github.com/milvus-io/milvus/internal/querycoordv2/params" "github.com/milvus-io/milvus/internal/querycoordv2/session" "github.com/milvus-io/milvus/internal/querycoordv2/task" "github.com/milvus-io/milvus/pkg/kv" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/querycoordv2/dist/dist_handler.go b/internal/querycoordv2/dist/dist_handler.go index 828cedc6e5ce3..14dff255d9e08 100644 --- a/internal/querycoordv2/dist/dist_handler.go +++ b/internal/querycoordv2/dist/dist_handler.go @@ -26,14 +26,14 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" . "github.com/milvus-io/milvus/internal/querycoordv2/params" "github.com/milvus-io/milvus/internal/querycoordv2/session" "github.com/milvus-io/milvus/internal/querycoordv2/task" "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/querycoordv2/dist/dist_handler_test.go b/internal/querycoordv2/dist/dist_handler_test.go index c66902bc43309..2e3c6f82ecee0 100644 --- a/internal/querycoordv2/dist/dist_handler_test.go +++ b/internal/querycoordv2/dist/dist_handler_test.go @@ -25,11 +25,11 @@ import ( "github.com/stretchr/testify/mock" "github.com/stretchr/testify/suite" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/internal/querycoordv2/session" "github.com/milvus-io/milvus/internal/querycoordv2/task" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querycoordv2/handlers.go b/internal/querycoordv2/handlers.go index a9c0a51baab00..b6d79c3026c03 100644 --- a/internal/querycoordv2/handlers.go +++ b/internal/querycoordv2/handlers.go @@ -31,12 +31,12 @@ 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/internal/json" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/internal/querycoordv2/session" "github.com/milvus-io/milvus/internal/querycoordv2/task" "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/hardware" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" diff --git a/internal/querycoordv2/handlers_test.go b/internal/querycoordv2/handlers_test.go index e465c81ea407b..2789f644d8cf4 100644 --- a/internal/querycoordv2/handlers_test.go +++ b/internal/querycoordv2/handlers_test.go @@ -25,9 +25,9 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus/internal/json" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/internal/querycoordv2/session" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/metricsinfo" ) diff --git a/internal/querycoordv2/job/job_load.go b/internal/querycoordv2/job/job_load.go index 998335f9e9a41..73815c920f044 100644 --- a/internal/querycoordv2/job/job_load.go +++ b/internal/querycoordv2/job/job_load.go @@ -28,7 +28,6 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/internal/querycoordv2/observers" "github.com/milvus-io/milvus/internal/querycoordv2/session" @@ -37,6 +36,7 @@ import ( "github.com/milvus-io/milvus/pkg/eventlog" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/querycoordv2/job/job_release.go b/internal/querycoordv2/job/job_release.go index 61c95284835fd..515eed97e622b 100644 --- a/internal/querycoordv2/job/job_release.go +++ b/internal/querycoordv2/job/job_release.go @@ -24,8 +24,6 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/proxypb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/checkers" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/internal/querycoordv2/observers" @@ -33,6 +31,8 @@ import ( "github.com/milvus-io/milvus/internal/util/proxyutil" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/proxypb" + "github.com/milvus-io/milvus/pkg/proto/querypb" ) type ReleaseCollectionJob struct { diff --git a/internal/querycoordv2/job/job_sync.go b/internal/querycoordv2/job/job_sync.go index 899cbe56495fc..d4a9511a62123 100644 --- a/internal/querycoordv2/job/job_sync.go +++ b/internal/querycoordv2/job/job_sync.go @@ -23,11 +23,11 @@ import ( "github.com/cockroachdb/errors" "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/internal/querycoordv2/observers" "github.com/milvus-io/milvus/internal/querycoordv2/session" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/querypb" ) type SyncNewCreatedPartitionJob struct { diff --git a/internal/querycoordv2/job/job_test.go b/internal/querycoordv2/job/job_test.go index 7c7e188c9d56e..723268c1ad255 100644 --- a/internal/querycoordv2/job/job_test.go +++ b/internal/querycoordv2/job/job_test.go @@ -32,8 +32,6 @@ import ( "github.com/milvus-io/milvus/internal/metastore" "github.com/milvus-io/milvus/internal/metastore/kv/querycoord" "github.com/milvus-io/milvus/internal/metastore/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/checkers" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/internal/querycoordv2/observers" @@ -42,6 +40,8 @@ import ( "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/internal/util/proxyutil" "github.com/milvus-io/milvus/pkg/kv" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/querycoordv2/job/job_update.go b/internal/querycoordv2/job/job_update.go index 7b5259c83d06c..dcdf0e72401b4 100644 --- a/internal/querycoordv2/job/job_update.go +++ b/internal/querycoordv2/job/job_update.go @@ -22,11 +22,11 @@ import ( "github.com/samber/lo" "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/internal/querycoordv2/observers" "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/querycoordv2/meta/channel_dist_manager.go b/internal/querycoordv2/meta/channel_dist_manager.go index 24e06327c3e54..3900b6ab1138a 100644 --- a/internal/querycoordv2/meta/channel_dist_manager.go +++ b/internal/querycoordv2/meta/channel_dist_manager.go @@ -22,8 +22,8 @@ import ( "github.com/samber/lo" "google.golang.org/protobuf/proto" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/util/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querycoordv2/meta/channel_dist_manager_test.go b/internal/querycoordv2/meta/channel_dist_manager_test.go index 734d665d6f3df..4d7ebf7b0946b 100644 --- a/internal/querycoordv2/meta/channel_dist_manager_test.go +++ b/internal/querycoordv2/meta/channel_dist_manager_test.go @@ -22,8 +22,8 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/suite" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querycoordv2/meta/collection_manager.go b/internal/querycoordv2/meta/collection_manager.go index 980331ff64c80..41e27b22bffdc 100644 --- a/internal/querycoordv2/meta/collection_manager.go +++ b/internal/querycoordv2/meta/collection_manager.go @@ -31,11 +31,11 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/metastore" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/eventlog" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/querycoordv2/meta/collection_manager_test.go b/internal/querycoordv2/meta/collection_manager_test.go index 2a8b5823bffd9..7e361019bda9c 100644 --- a/internal/querycoordv2/meta/collection_manager_test.go +++ b/internal/querycoordv2/meta/collection_manager_test.go @@ -32,11 +32,11 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/metastore" "github.com/milvus-io/milvus/internal/metastore/kv/querycoord" - "github.com/milvus-io/milvus/internal/proto/querypb" . "github.com/milvus-io/milvus/internal/querycoordv2/params" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/kv" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/querycoordv2/meta/coordinator_broker.go b/internal/querycoordv2/meta/coordinator_broker.go index cc84db836bbf3..b9ceab2ea46f6 100644 --- a/internal/querycoordv2/meta/coordinator_broker.go +++ b/internal/querycoordv2/meta/coordinator_broker.go @@ -28,13 +28,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/internal/metastore/kv/binlog" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/querycoordv2/meta/coordinator_broker_test.go b/internal/querycoordv2/meta/coordinator_broker_test.go index 578b4f3713457..70903f353222a 100644 --- a/internal/querycoordv2/meta/coordinator_broker_test.go +++ b/internal/querycoordv2/meta/coordinator_broker_test.go @@ -30,11 +30,11 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querycoordv2/meta/dist_manager_test.go b/internal/querycoordv2/meta/dist_manager_test.go index 3e8f33f4eff26..50b58df19b3d3 100644 --- a/internal/querycoordv2/meta/dist_manager_test.go +++ b/internal/querycoordv2/meta/dist_manager_test.go @@ -7,8 +7,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus/internal/json" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/metricsinfo" ) diff --git a/internal/querycoordv2/meta/leader_view_manager.go b/internal/querycoordv2/meta/leader_view_manager.go index c29171e6f90ae..fbb1ee93ea09d 100644 --- a/internal/querycoordv2/meta/leader_view_manager.go +++ b/internal/querycoordv2/meta/leader_view_manager.go @@ -21,7 +21,7 @@ import ( "github.com/samber/lo" - "github.com/milvus-io/milvus/internal/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querycoordv2/meta/leader_view_manager_test.go b/internal/querycoordv2/meta/leader_view_manager_test.go index c4cc7cec2ec31..5825228aa200d 100644 --- a/internal/querycoordv2/meta/leader_view_manager_test.go +++ b/internal/querycoordv2/meta/leader_view_manager_test.go @@ -26,8 +26,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus/internal/json" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querycoordv2/meta/mock_broker.go b/internal/querycoordv2/meta/mock_broker.go index 23b1b15f28e12..921417bbb5c92 100644 --- a/internal/querycoordv2/meta/mock_broker.go +++ b/internal/querycoordv2/meta/mock_broker.go @@ -5,16 +5,16 @@ package meta import ( context "context" - datapb "github.com/milvus-io/milvus/internal/proto/datapb" - indexpb "github.com/milvus-io/milvus/internal/proto/indexpb" + datapb "github.com/milvus-io/milvus/pkg/proto/datapb" + indexpb "github.com/milvus-io/milvus/pkg/proto/indexpb" milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" mock "github.com/stretchr/testify/mock" - querypb "github.com/milvus-io/milvus/internal/proto/querypb" + querypb "github.com/milvus-io/milvus/pkg/proto/querypb" - rootcoordpb "github.com/milvus-io/milvus/internal/proto/rootcoordpb" + rootcoordpb "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" ) // MockBroker is an autogenerated mock type for the Broker type diff --git a/internal/querycoordv2/meta/mock_target_manager.go b/internal/querycoordv2/meta/mock_target_manager.go index 560c055bf30b3..4fdb9896e95f2 100644 --- a/internal/querycoordv2/meta/mock_target_manager.go +++ b/internal/querycoordv2/meta/mock_target_manager.go @@ -6,7 +6,7 @@ import ( context "context" metastore "github.com/milvus-io/milvus/internal/metastore" - datapb "github.com/milvus-io/milvus/internal/proto/datapb" + datapb "github.com/milvus-io/milvus/pkg/proto/datapb" mock "github.com/stretchr/testify/mock" diff --git a/internal/querycoordv2/meta/replica.go b/internal/querycoordv2/meta/replica.go index 217dfc7144747..8f1ab12b0c5ad 100644 --- a/internal/querycoordv2/meta/replica.go +++ b/internal/querycoordv2/meta/replica.go @@ -3,7 +3,7 @@ package meta import ( "google.golang.org/protobuf/proto" - "github.com/milvus-io/milvus/internal/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querycoordv2/meta/replica_manager.go b/internal/querycoordv2/meta/replica_manager.go index 9145504996205..f204320e969bd 100644 --- a/internal/querycoordv2/meta/replica_manager.go +++ b/internal/querycoordv2/meta/replica_manager.go @@ -27,9 +27,9 @@ import ( "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/metastore" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" diff --git a/internal/querycoordv2/meta/replica_manager_helper_test.go b/internal/querycoordv2/meta/replica_manager_helper_test.go index 6ed9c369207a5..f7049292041c9 100644 --- a/internal/querycoordv2/meta/replica_manager_helper_test.go +++ b/internal/querycoordv2/meta/replica_manager_helper_test.go @@ -5,7 +5,7 @@ import ( "github.com/stretchr/testify/suite" - "github.com/milvus-io/milvus/internal/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querycoordv2/meta/replica_manager_test.go b/internal/querycoordv2/meta/replica_manager_test.go index 7921129590190..2a97ada097a47 100644 --- a/internal/querycoordv2/meta/replica_manager_test.go +++ b/internal/querycoordv2/meta/replica_manager_test.go @@ -32,9 +32,9 @@ import ( "github.com/milvus-io/milvus/internal/metastore" "github.com/milvus-io/milvus/internal/metastore/kv/querycoord" "github.com/milvus-io/milvus/internal/metastore/mocks" - "github.com/milvus-io/milvus/internal/proto/querypb" . "github.com/milvus-io/milvus/internal/querycoordv2/params" "github.com/milvus-io/milvus/pkg/kv" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/querycoordv2/meta/replica_test.go b/internal/querycoordv2/meta/replica_test.go index 31c1194ac023b..2e3804d189c62 100644 --- a/internal/querycoordv2/meta/replica_test.go +++ b/internal/querycoordv2/meta/replica_test.go @@ -5,7 +5,7 @@ import ( "github.com/stretchr/testify/suite" - "github.com/milvus-io/milvus/internal/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querycoordv2/meta/resource_group.go b/internal/querycoordv2/meta/resource_group.go index 4d7a5a43702c8..7cd01afd00209 100644 --- a/internal/querycoordv2/meta/resource_group.go +++ b/internal/querycoordv2/meta/resource_group.go @@ -5,8 +5,8 @@ import ( "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus-proto/go-api/v2/rgpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/session" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querycoordv2/meta/resource_group_test.go b/internal/querycoordv2/meta/resource_group_test.go index ea92e41c1a11c..2d97432f05ea5 100644 --- a/internal/querycoordv2/meta/resource_group_test.go +++ b/internal/querycoordv2/meta/resource_group_test.go @@ -7,8 +7,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/rgpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/session" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querycoordv2/meta/resource_manager.go b/internal/querycoordv2/meta/resource_manager.go index f0696dcecacd9..f24141aa1612a 100644 --- a/internal/querycoordv2/meta/resource_manager.go +++ b/internal/querycoordv2/meta/resource_manager.go @@ -31,10 +31,10 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/rgpb" "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/metastore" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/session" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/querycoordv2/meta/segment_dist_manager.go b/internal/querycoordv2/meta/segment_dist_manager.go index 38cefdcf7ad8f..21d2d24c2ed1f 100644 --- a/internal/querycoordv2/meta/segment_dist_manager.go +++ b/internal/querycoordv2/meta/segment_dist_manager.go @@ -22,9 +22,9 @@ import ( "github.com/samber/lo" "google.golang.org/protobuf/proto" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/util/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/tsoutil" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/querycoordv2/meta/segment_dist_manager_test.go b/internal/querycoordv2/meta/segment_dist_manager_test.go index 43e75041e1382..ca872ff0523d0 100644 --- a/internal/querycoordv2/meta/segment_dist_manager_test.go +++ b/internal/querycoordv2/meta/segment_dist_manager_test.go @@ -23,8 +23,8 @@ import ( "github.com/stretchr/testify/suite" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/metricsinfo" ) diff --git a/internal/querycoordv2/meta/target.go b/internal/querycoordv2/meta/target.go index 60491a389ae05..27eae08be3d0a 100644 --- a/internal/querycoordv2/meta/target.go +++ b/internal/querycoordv2/meta/target.go @@ -22,10 +22,10 @@ import ( "github.com/samber/lo" "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/util/metrics" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querycoordv2/meta/target_manager.go b/internal/querycoordv2/meta/target_manager.go index 924f8eb6ecc18..d76ac1f41fb1d 100644 --- a/internal/querycoordv2/meta/target_manager.go +++ b/internal/querycoordv2/meta/target_manager.go @@ -28,11 +28,11 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/metastore" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/querycoordv2/meta/target_manager_test.go b/internal/querycoordv2/meta/target_manager_test.go index 91958bfcbd60c..a917b7a042bc2 100644 --- a/internal/querycoordv2/meta/target_manager_test.go +++ b/internal/querycoordv2/meta/target_manager_test.go @@ -31,11 +31,11 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/metastore" "github.com/milvus-io/milvus/internal/metastore/kv/querycoord" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" . "github.com/milvus-io/milvus/internal/querycoordv2/params" "github.com/milvus-io/milvus/internal/querycoordv2/session" "github.com/milvus-io/milvus/pkg/kv" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/querycoordv2/mocks/mock_querynode.go b/internal/querycoordv2/mocks/mock_querynode.go index c161c66309ab4..7489be08f34ba 100644 --- a/internal/querycoordv2/mocks/mock_querynode.go +++ b/internal/querycoordv2/mocks/mock_querynode.go @@ -7,13 +7,13 @@ import ( commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - internalpb "github.com/milvus-io/milvus/internal/proto/internalpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" mock "github.com/stretchr/testify/mock" - querypb "github.com/milvus-io/milvus/internal/proto/querypb" + querypb "github.com/milvus-io/milvus/pkg/proto/querypb" ) // MockQueryNodeServer is an autogenerated mock type for the QueryNodeServer type diff --git a/internal/querycoordv2/mocks/querynode.go b/internal/querycoordv2/mocks/querynode.go index 1f05aa4b5fd41..ee1b21b3f5041 100644 --- a/internal/querycoordv2/mocks/querynode.go +++ b/internal/querycoordv2/mocks/querynode.go @@ -29,10 +29,10 @@ import ( "google.golang.org/grpc" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" - "github.com/milvus-io/milvus/internal/proto/querypb" . "github.com/milvus-io/milvus/internal/querycoordv2/params" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querycoordv2/observers/collection_observer.go b/internal/querycoordv2/observers/collection_observer.go index 99f8c2f06a341..4b86e6c2258f7 100644 --- a/internal/querycoordv2/observers/collection_observer.go +++ b/internal/querycoordv2/observers/collection_observer.go @@ -27,8 +27,6 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/proxypb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/checkers" "github.com/milvus-io/milvus/internal/querycoordv2/meta" . "github.com/milvus-io/milvus/internal/querycoordv2/params" @@ -37,6 +35,8 @@ import ( "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/eventlog" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/proxypb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querycoordv2/observers/collection_observer_test.go b/internal/querycoordv2/observers/collection_observer_test.go index 53b2e47a1b711..771b92e53abb4 100644 --- a/internal/querycoordv2/observers/collection_observer_test.go +++ b/internal/querycoordv2/observers/collection_observer_test.go @@ -29,8 +29,6 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/metastore" "github.com/milvus-io/milvus/internal/metastore/kv/querycoord" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/checkers" "github.com/milvus-io/milvus/internal/querycoordv2/meta" . "github.com/milvus-io/milvus/internal/querycoordv2/params" @@ -38,6 +36,8 @@ import ( "github.com/milvus-io/milvus/internal/util/proxyutil" "github.com/milvus-io/milvus/pkg/kv" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/querycoordv2/observers/leader_cache_observer.go b/internal/querycoordv2/observers/leader_cache_observer.go index 7f92d6f0908cb..fc5c7cb94455b 100644 --- a/internal/querycoordv2/observers/leader_cache_observer.go +++ b/internal/querycoordv2/observers/leader_cache_observer.go @@ -23,10 +23,10 @@ import ( "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/proto/proxypb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/util/proxyutil" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/proxypb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querycoordv2/observers/leader_cache_observer_test.go b/internal/querycoordv2/observers/leader_cache_observer_test.go index 665beea8b6834..6d782f30b73f0 100644 --- a/internal/querycoordv2/observers/leader_cache_observer_test.go +++ b/internal/querycoordv2/observers/leader_cache_observer_test.go @@ -26,8 +26,8 @@ import ( "github.com/stretchr/testify/suite" "go.uber.org/atomic" - "github.com/milvus-io/milvus/internal/proto/proxypb" "github.com/milvus-io/milvus/internal/util/proxyutil" + "github.com/milvus-io/milvus/pkg/proto/proxypb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querycoordv2/observers/target_observer.go b/internal/querycoordv2/observers/target_observer.go index 7e5965c63b66b..c4938c453a85f 100644 --- a/internal/querycoordv2/observers/target_observer.go +++ b/internal/querycoordv2/observers/target_observer.go @@ -26,13 +26,13 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/internal/querycoordv2/params" "github.com/milvus-io/milvus/internal/querycoordv2/session" "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/lock" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/querycoordv2/observers/target_observer_test.go b/internal/querycoordv2/observers/target_observer_test.go index 68903bae2c23e..615e182ab430d 100644 --- a/internal/querycoordv2/observers/target_observer_test.go +++ b/internal/querycoordv2/observers/target_observer_test.go @@ -27,14 +27,14 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/metastore/kv/querycoord" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" . "github.com/milvus-io/milvus/internal/querycoordv2/params" "github.com/milvus-io/milvus/internal/querycoordv2/session" "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/kv" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/querycoordv2/ops_service_test.go b/internal/querycoordv2/ops_service_test.go index db56c8ded85d4..7e26f8997392a 100644 --- a/internal/querycoordv2/ops_service_test.go +++ b/internal/querycoordv2/ops_service_test.go @@ -30,8 +30,6 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/metastore" "github.com/milvus-io/milvus/internal/metastore/kv/querycoord" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/balance" "github.com/milvus-io/milvus/internal/querycoordv2/checkers" "github.com/milvus-io/milvus/internal/querycoordv2/dist" @@ -45,6 +43,8 @@ import ( "github.com/milvus-io/milvus/internal/util/proxyutil" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/kv" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" diff --git a/internal/querycoordv2/ops_services.go b/internal/querycoordv2/ops_services.go index d55f734d85e16..135bf481d1b88 100644 --- a/internal/querycoordv2/ops_services.go +++ b/internal/querycoordv2/ops_services.go @@ -24,11 +24,11 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/internal/querycoordv2/session" "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querycoordv2/server.go b/internal/querycoordv2/server.go index 8464d3055fcff..25c7714a69681 100644 --- a/internal/querycoordv2/server.go +++ b/internal/querycoordv2/server.go @@ -42,8 +42,6 @@ import ( "github.com/milvus-io/milvus/internal/kv/tikv" "github.com/milvus-io/milvus/internal/metastore" "github.com/milvus-io/milvus/internal/metastore/kv/querycoord" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/balance" "github.com/milvus-io/milvus/internal/querycoordv2/checkers" "github.com/milvus-io/milvus/internal/querycoordv2/dist" @@ -63,6 +61,8 @@ import ( "github.com/milvus-io/milvus/pkg/kv" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/expr" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/querycoordv2/server_test.go b/internal/querycoordv2/server_test.go index a5b893619c6eb..f3f810ae636d1 100644 --- a/internal/querycoordv2/server_test.go +++ b/internal/querycoordv2/server_test.go @@ -34,9 +34,6 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" coordMocks "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/querycoordv2/checkers" "github.com/milvus-io/milvus/internal/querycoordv2/dist" "github.com/milvus-io/milvus/internal/querycoordv2/meta" @@ -48,6 +45,9 @@ import ( "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/querycoordv2/services.go b/internal/querycoordv2/services.go index 756712cfdfe92..3f91347fc4db2 100644 --- a/internal/querycoordv2/services.go +++ b/internal/querycoordv2/services.go @@ -30,14 +30,14 @@ 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/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/job" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/internal/util/componentutil" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/querycoordv2/services_test.go b/internal/querycoordv2/services_test.go index badf645debf76..77ee08d734fbd 100644 --- a/internal/querycoordv2/services_test.go +++ b/internal/querycoordv2/services_test.go @@ -35,9 +35,6 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/metastore" "github.com/milvus-io/milvus/internal/metastore/kv/querycoord" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/balance" "github.com/milvus-io/milvus/internal/querycoordv2/checkers" "github.com/milvus-io/milvus/internal/querycoordv2/dist" @@ -51,6 +48,9 @@ import ( "github.com/milvus-io/milvus/internal/util/proxyutil" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/kv" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" diff --git a/internal/querycoordv2/session/cluster.go b/internal/querycoordv2/session/cluster.go index 7b6bc316ebe25..b0bd868d1e326 100644 --- a/internal/querycoordv2/session/cluster.go +++ b/internal/querycoordv2/session/cluster.go @@ -29,9 +29,9 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" grpcquerynodeclient "github.com/milvus-io/milvus/internal/distributed/querynode/client" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querycoordv2/session/cluster_test.go b/internal/querycoordv2/session/cluster_test.go index 6ba387249877a..8c25aed9f9109 100644 --- a/internal/querycoordv2/session/cluster_test.go +++ b/internal/querycoordv2/session/cluster_test.go @@ -30,8 +30,8 @@ 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/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/mocks" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querycoordv2/session/mock_cluster.go b/internal/querycoordv2/session/mock_cluster.go index bd7d3b3eae622..377d31b57604e 100644 --- a/internal/querycoordv2/session/mock_cluster.go +++ b/internal/querycoordv2/session/mock_cluster.go @@ -11,7 +11,7 @@ import ( mock "github.com/stretchr/testify/mock" - querypb "github.com/milvus-io/milvus/internal/proto/querypb" + querypb "github.com/milvus-io/milvus/pkg/proto/querypb" ) // MockCluster is an autogenerated mock type for the Cluster type diff --git a/internal/querycoordv2/task/action.go b/internal/querycoordv2/task/action.go index dfbc4c44ddf52..2c3f69c0734c4 100644 --- a/internal/querycoordv2/task/action.go +++ b/internal/querycoordv2/task/action.go @@ -22,8 +22,8 @@ import ( "github.com/samber/lo" "go.uber.org/atomic" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querycoordv2/task/executor.go b/internal/querycoordv2/task/executor.go index b66d26b0f722e..2ab1843dd7577 100644 --- a/internal/querycoordv2/task/executor.go +++ b/internal/querycoordv2/task/executor.go @@ -29,13 +29,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/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" . "github.com/milvus-io/milvus/internal/querycoordv2/params" "github.com/milvus-io/milvus/internal/querycoordv2/session" "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/indexparams" diff --git a/internal/querycoordv2/task/scheduler.go b/internal/querycoordv2/task/scheduler.go index 316f1a552be71..bb2e91139845a 100644 --- a/internal/querycoordv2/task/scheduler.go +++ b/internal/querycoordv2/task/scheduler.go @@ -29,13 +29,13 @@ import ( "go.uber.org/atomic" "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/internal/querycoordv2/session" "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/hardware" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/querycoordv2/task/task.go b/internal/querycoordv2/task/task.go index 0383be60af762..25e64d5be1392 100644 --- a/internal/querycoordv2/task/task.go +++ b/internal/querycoordv2/task/task.go @@ -28,8 +28,8 @@ import ( "go.uber.org/atomic" "github.com/milvus-io/milvus/internal/json" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/querycoordv2/task/task_test.go b/internal/querycoordv2/task/task_test.go index 482282f54f319..9b3b736922a03 100644 --- a/internal/querycoordv2/task/task_test.go +++ b/internal/querycoordv2/task/task_test.go @@ -35,16 +35,16 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/metastore" "github.com/milvus-io/milvus/internal/metastore/kv/querycoord" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" . "github.com/milvus-io/milvus/internal/querycoordv2/params" "github.com/milvus-io/milvus/internal/querycoordv2/session" "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/kv" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" diff --git a/internal/querycoordv2/task/utils.go b/internal/querycoordv2/task/utils.go index c4f4df26e5332..6c4f26673d245 100644 --- a/internal/querycoordv2/task/utils.go +++ b/internal/querycoordv2/task/utils.go @@ -27,12 +27,12 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "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/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querycoordv2/task/utils_test.go b/internal/querycoordv2/task/utils_test.go index f3c3e3c6ad160..61c6651929a80 100644 --- a/internal/querycoordv2/task/utils_test.go +++ b/internal/querycoordv2/task/utils_test.go @@ -26,8 +26,8 @@ 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/proto/querypb" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/querypb" ) type UtilsSuite struct { diff --git a/internal/querycoordv2/utils/meta_test.go b/internal/querycoordv2/utils/meta_test.go index 9b2bbbbdd8051..03bbf58c8b1ec 100644 --- a/internal/querycoordv2/utils/meta_test.go +++ b/internal/querycoordv2/utils/meta_test.go @@ -29,10 +29,10 @@ import ( etcdKV "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/metastore/kv/querycoord" "github.com/milvus-io/milvus/internal/metastore/mocks" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" . "github.com/milvus-io/milvus/internal/querycoordv2/params" "github.com/milvus-io/milvus/internal/querycoordv2/session" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/querycoordv2/utils/test.go b/internal/querycoordv2/utils/test.go index 41b1d57478902..2333874aac0c4 100644 --- a/internal/querycoordv2/utils/test.go +++ b/internal/querycoordv2/utils/test.go @@ -18,9 +18,9 @@ package utils import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querycoordv2/utils/types.go b/internal/querycoordv2/utils/types.go index 511081d73763b..91768acdb95de 100644 --- a/internal/querycoordv2/utils/types.go +++ b/internal/querycoordv2/utils/types.go @@ -23,10 +23,10 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/tsoutil" ) diff --git a/internal/querycoordv2/utils/types_test.go b/internal/querycoordv2/utils/types_test.go index 41743c6882218..6c3afbb52ba7b 100644 --- a/internal/querycoordv2/utils/types_test.go +++ b/internal/querycoordv2/utils/types_test.go @@ -24,7 +24,7 @@ import ( "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/tsoutil" ) diff --git a/internal/querycoordv2/utils/util.go b/internal/querycoordv2/utils/util.go index 455c2b1708133..796fddb8ff0a4 100644 --- a/internal/querycoordv2/utils/util.go +++ b/internal/querycoordv2/utils/util.go @@ -24,11 +24,11 @@ import ( "go.uber.org/multierr" "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/internal/querycoordv2/session" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querycoordv2/utils/util_test.go b/internal/querycoordv2/utils/util_test.go index a111600e193bf..7be3065f542b7 100644 --- a/internal/querycoordv2/utils/util_test.go +++ b/internal/querycoordv2/utils/util_test.go @@ -23,10 +23,10 @@ import ( "github.com/stretchr/testify/mock" "github.com/stretchr/testify/suite" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/internal/querycoordv2/session" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" ) type UtilTestSuite struct { diff --git a/internal/querynodev2/cluster/mock_worker.go b/internal/querynodev2/cluster/mock_worker.go index b9c6f27238f8a..dc59eea0256b9 100644 --- a/internal/querynodev2/cluster/mock_worker.go +++ b/internal/querynodev2/cluster/mock_worker.go @@ -5,10 +5,10 @@ package cluster import ( context "context" - internalpb "github.com/milvus-io/milvus/internal/proto/internalpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" mock "github.com/stretchr/testify/mock" - querypb "github.com/milvus-io/milvus/internal/proto/querypb" + querypb "github.com/milvus-io/milvus/pkg/proto/querypb" streamrpc "github.com/milvus-io/milvus/internal/util/streamrpc" ) diff --git a/internal/querynodev2/cluster/worker.go b/internal/querynodev2/cluster/worker.go index 47752e81df496..472590d00354d 100644 --- a/internal/querynodev2/cluster/worker.go +++ b/internal/querynodev2/cluster/worker.go @@ -25,11 +25,11 @@ import ( "go.uber.org/atomic" "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/streamrpc" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querynodev2/cluster/worker_test.go b/internal/querynodev2/cluster/worker_test.go index d224a14d0ae29..290415a345fd0 100644 --- a/internal/querynodev2/cluster/worker_test.go +++ b/internal/querynodev2/cluster/worker_test.go @@ -34,9 +34,9 @@ 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/mocks" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/util/streamrpc" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/querynodev2/delegator/delegator.go b/internal/querynodev2/delegator/delegator.go index 1e65a31b72cd4..6f7f157fb8b92 100644 --- a/internal/querynodev2/delegator/delegator.go +++ b/internal/querynodev2/delegator/delegator.go @@ -35,8 +35,6 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "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/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querynodev2/cluster" "github.com/milvus-io/milvus/internal/querynodev2/delegator/deletebuffer" "github.com/milvus-io/milvus/internal/querynodev2/pkoracle" @@ -50,6 +48,8 @@ import ( "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/lifetime" diff --git a/internal/querynodev2/delegator/delegator_data.go b/internal/querynodev2/delegator/delegator_data.go index 586f56f2d6794..c0ce3f516d1f8 100644 --- a/internal/querynodev2/delegator/delegator_data.go +++ b/internal/querynodev2/delegator/delegator_data.go @@ -33,10 +33,6 @@ 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/distributed/streaming" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" "github.com/milvus-io/milvus/internal/querynodev2/cluster" "github.com/milvus-io/milvus/internal/querynodev2/delegator/deletebuffer" "github.com/milvus-io/milvus/internal/querynodev2/pkoracle" @@ -48,6 +44,10 @@ import ( "github.com/milvus-io/milvus/pkg/metrics" mqcommon "github.com/milvus-io/milvus/pkg/mq/common" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/message/adaptor" "github.com/milvus-io/milvus/pkg/streaming/util/options" diff --git a/internal/querynodev2/delegator/delegator_data_test.go b/internal/querynodev2/delegator/delegator_data_test.go index f164b9c93ff7a..552d6a93a6930 100644 --- a/internal/querynodev2/delegator/delegator_data_test.go +++ b/internal/querynodev2/delegator/delegator_data_test.go @@ -37,11 +37,6 @@ 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/mocks/util/mock_segcore" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/planpb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" "github.com/milvus-io/milvus/internal/querynodev2/cluster" "github.com/milvus-io/milvus/internal/querynodev2/pkoracle" "github.com/milvus-io/milvus/internal/querynodev2/segments" @@ -51,6 +46,11 @@ import ( "github.com/milvus-io/milvus/internal/util/initcore" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/querynodev2/delegator/delegator_test.go b/internal/querynodev2/delegator/delegator_test.go index 9bfd65242c920..d7cf7cb89639f 100644 --- a/internal/querynodev2/delegator/delegator_test.go +++ b/internal/querynodev2/delegator/delegator_test.go @@ -32,15 +32,15 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "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/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" "github.com/milvus-io/milvus/internal/querynodev2/cluster" "github.com/milvus-io/milvus/internal/querynodev2/segments" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/streamrpc" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metric" diff --git a/internal/querynodev2/delegator/delta_forward.go b/internal/querynodev2/delegator/delta_forward.go index 0c94dec37620c..a42353d4554d8 100644 --- a/internal/querynodev2/delegator/delta_forward.go +++ b/internal/querynodev2/delegator/delta_forward.go @@ -28,8 +28,6 @@ import ( "golang.org/x/sync/errgroup" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querynodev2/cluster" "github.com/milvus-io/milvus/internal/querynodev2/pkoracle" "github.com/milvus-io/milvus/internal/querynodev2/segments" @@ -37,6 +35,8 @@ import ( "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/querynodev2/delegator/delta_forward_test.go b/internal/querynodev2/delegator/delta_forward_test.go index 9f93851a013cd..6dbd07710c0a3 100644 --- a/internal/querynodev2/delegator/delta_forward_test.go +++ b/internal/querynodev2/delegator/delta_forward_test.go @@ -28,15 +28,15 @@ 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/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" "github.com/milvus-io/milvus/internal/querynodev2/cluster" "github.com/milvus-io/milvus/internal/querynodev2/pkoracle" "github.com/milvus-io/milvus/internal/querynodev2/segments" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metric" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/querynodev2/delegator/distribution.go b/internal/querynodev2/delegator/distribution.go index 08fa2d327aa68..64c055433ec56 100644 --- a/internal/querynodev2/delegator/distribution.go +++ b/internal/querynodev2/delegator/distribution.go @@ -23,8 +23,8 @@ import ( "go.uber.org/atomic" "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querynodev2/delegator/idf_oracle.go b/internal/querynodev2/delegator/idf_oracle.go index 3e6c487ddc1a8..ca6aebb17f10f 100644 --- a/internal/querynodev2/delegator/idf_oracle.go +++ b/internal/querynodev2/delegator/idf_oracle.go @@ -25,10 +25,10 @@ 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/proto/datapb" "github.com/milvus-io/milvus/internal/querynodev2/segments" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" ) type IDFOracle interface { diff --git a/internal/querynodev2/delegator/mock_delegator.go b/internal/querynodev2/delegator/mock_delegator.go index 6a348607f0d64..d61bf0a598b87 100644 --- a/internal/querynodev2/delegator/mock_delegator.go +++ b/internal/querynodev2/delegator/mock_delegator.go @@ -5,12 +5,12 @@ package delegator import ( context "context" - internalpb "github.com/milvus-io/milvus/internal/proto/internalpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" mock "github.com/stretchr/testify/mock" msgpb "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" - querypb "github.com/milvus-io/milvus/internal/proto/querypb" + querypb "github.com/milvus-io/milvus/pkg/proto/querypb" streamrpc "github.com/milvus-io/milvus/internal/util/streamrpc" ) diff --git a/internal/querynodev2/delegator/scalar_pruner.go b/internal/querynodev2/delegator/scalar_pruner.go index c2fe95fb836ba..640cf478a0b02 100644 --- a/internal/querynodev2/delegator/scalar_pruner.go +++ b/internal/querynodev2/delegator/scalar_pruner.go @@ -4,8 +4,8 @@ import ( "github.com/bits-and-blooms/bitset" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/planpb" "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/pkg/proto/planpb" ) type EvalCtx struct { diff --git a/internal/querynodev2/delegator/segment_pruner.go b/internal/querynodev2/delegator/segment_pruner.go index 35503e00c8926..e49a50ce17b31 100644 --- a/internal/querynodev2/delegator/segment_pruner.go +++ b/internal/querynodev2/delegator/segment_pruner.go @@ -13,14 +13,14 @@ 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/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/planpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/clustering" "github.com/milvus-io/milvus/internal/util/exprutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" "github.com/milvus-io/milvus/pkg/util/distance" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/querynodev2/delegator/segment_pruner_test.go b/internal/querynodev2/delegator/segment_pruner_test.go index 87698af76e36b..530deeb63d0e7 100644 --- a/internal/querynodev2/delegator/segment_pruner_test.go +++ b/internal/querynodev2/delegator/segment_pruner_test.go @@ -10,10 +10,10 @@ 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/parser/planparserv2" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/clustering" "github.com/milvus-io/milvus/internal/util/testutil" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querynodev2/delegator/types.go b/internal/querynodev2/delegator/types.go index b981cbbfbd63b..c5e7c0e1f4db2 100644 --- a/internal/querynodev2/delegator/types.go +++ b/internal/querynodev2/delegator/types.go @@ -6,7 +6,7 @@ import ( "github.com/cockroachdb/errors" - "github.com/milvus-io/milvus/internal/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querynodev2/delegator/util.go b/internal/querynodev2/delegator/util.go index b9560554e9222..0413d5bbf9d52 100644 --- a/internal/querynodev2/delegator/util.go +++ b/internal/querynodev2/delegator/util.go @@ -7,9 +7,9 @@ import ( "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/planpb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/querynodev2/handlers.go b/internal/querynodev2/handlers.go index 9c3cc113de236..817777606ad26 100644 --- a/internal/querynodev2/handlers.go +++ b/internal/querynodev2/handlers.go @@ -26,9 +26,6 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querynodev2/delegator" "github.com/milvus-io/milvus/internal/querynodev2/segments" "github.com/milvus-io/milvus/internal/querynodev2/tasks" @@ -36,6 +33,9 @@ import ( "github.com/milvus-io/milvus/internal/util/streamrpc" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/querynodev2/handlers_test.go b/internal/querynodev2/handlers_test.go index a311743fca097..78a9c68deb793 100644 --- a/internal/querynodev2/handlers_test.go +++ b/internal/querynodev2/handlers_test.go @@ -25,11 +25,11 @@ import ( "github.com/stretchr/testify/suite" clientv3 "go.etcd.io/etcd/client/v3" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querynodev2/delegator" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/dependency" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querynodev2/local_worker.go b/internal/querynodev2/local_worker.go index 06ab6b26f63ff..4044050d3cf89 100644 --- a/internal/querynodev2/local_worker.go +++ b/internal/querynodev2/local_worker.go @@ -19,10 +19,10 @@ package querynodev2 import ( "context" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querynodev2/cluster" "github.com/milvus-io/milvus/internal/util/streamrpc" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/querynodev2/local_worker_test.go b/internal/querynodev2/local_worker_test.go index 50ed8b4c1d965..af791f6ccbe07 100644 --- a/internal/querynodev2/local_worker_test.go +++ b/internal/querynodev2/local_worker_test.go @@ -28,11 +28,11 @@ 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/mocks/util/mock_segcore" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" "github.com/milvus-io/milvus/internal/querynodev2/segments" "github.com/milvus-io/milvus/internal/util/dependency" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querynodev2/metrics_info_test.go b/internal/querynodev2/metrics_info_test.go index 6966f2b050473..3b9984d6652c7 100644 --- a/internal/querynodev2/metrics_info_test.go +++ b/internal/querynodev2/metrics_info_test.go @@ -24,11 +24,11 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/json" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querynodev2/delegator" "github.com/milvus-io/milvus/internal/querynodev2/pipeline" "github.com/milvus-io/milvus/internal/querynodev2/segments" "github.com/milvus-io/milvus/pkg/mq/msgdispatcher" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/tsoutil" diff --git a/internal/querynodev2/mock_data.go b/internal/querynodev2/mock_data.go index 735decaaa224a..7de63e97a9875 100644 --- a/internal/querynodev2/mock_data.go +++ b/internal/querynodev2/mock_data.go @@ -25,8 +25,8 @@ 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/proto/planpb" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/planpb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querynodev2/pipeline/filter_node_test.go b/internal/querynodev2/pipeline/filter_node_test.go index 738a6e0399e53..022f161d7e370 100644 --- a/internal/querynodev2/pipeline/filter_node_test.go +++ b/internal/querynodev2/pipeline/filter_node_test.go @@ -23,10 +23,10 @@ import ( "github.com/stretchr/testify/mock" "github.com/stretchr/testify/suite" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querynodev2/delegator" "github.com/milvus-io/milvus/internal/querynodev2/segments" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querynodev2/pipeline/insert_node_test.go b/internal/querynodev2/pipeline/insert_node_test.go index c1380db2c28d7..dd98ea2fe8a6a 100644 --- a/internal/querynodev2/pipeline/insert_node_test.go +++ b/internal/querynodev2/pipeline/insert_node_test.go @@ -25,9 +25,9 @@ import ( "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/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querynodev2/delegator" "github.com/milvus-io/milvus/internal/querynodev2/segments" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querynodev2/pipeline/manager_test.go b/internal/querynodev2/pipeline/manager_test.go index 48f09ce3ddfd5..33eaab9b289dd 100644 --- a/internal/querynodev2/pipeline/manager_test.go +++ b/internal/querynodev2/pipeline/manager_test.go @@ -25,11 +25,11 @@ 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/proto/querypb" "github.com/milvus-io/milvus/internal/querynodev2/delegator" "github.com/milvus-io/milvus/internal/querynodev2/segments" "github.com/milvus-io/milvus/pkg/mq/msgdispatcher" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querynodev2/pipeline/pipeline_test.go b/internal/querynodev2/pipeline/pipeline_test.go index dd153527e1c67..4ceb237989e55 100644 --- a/internal/querynodev2/pipeline/pipeline_test.go +++ b/internal/querynodev2/pipeline/pipeline_test.go @@ -28,12 +28,12 @@ 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/mocks/util/mock_segcore" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querynodev2/delegator" "github.com/milvus-io/milvus/internal/querynodev2/segments" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/mq/msgdispatcher" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querynodev2/pipeline/type.go b/internal/querynodev2/pipeline/type.go index 41de59465e4dd..9b822d16f6a06 100644 --- a/internal/querynodev2/pipeline/type.go +++ b/internal/querynodev2/pipeline/type.go @@ -20,11 +20,11 @@ import ( "time" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querynodev2/segments" "github.com/milvus-io/milvus/internal/storage" base "github.com/milvus-io/milvus/internal/util/pipeline" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querynodev2/segments/collection.go b/internal/querynodev2/segments/collection.go index f4ad0f930f599..207f15e2dc8a3 100644 --- a/internal/querynodev2/segments/collection.go +++ b/internal/querynodev2/segments/collection.go @@ -26,13 +26,13 @@ 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/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" "github.com/milvus-io/milvus/internal/util/segcore" "github.com/milvus-io/milvus/internal/util/vecindexmgr" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querynodev2/segments/count_reducer.go b/internal/querynodev2/segments/count_reducer.go index ae484dd54fe14..2da67ca79b1ce 100644 --- a/internal/querynodev2/segments/count_reducer.go +++ b/internal/querynodev2/segments/count_reducer.go @@ -3,10 +3,10 @@ package segments import ( "context" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" "github.com/milvus-io/milvus/internal/util/funcutil" "github.com/milvus-io/milvus/internal/util/segcore" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" ) type cntReducer struct{} diff --git a/internal/querynodev2/segments/count_reducer_test.go b/internal/querynodev2/segments/count_reducer_test.go index 51415cf9c7883..2e1578b29a03a 100644 --- a/internal/querynodev2/segments/count_reducer_test.go +++ b/internal/querynodev2/segments/count_reducer_test.go @@ -7,9 +7,9 @@ import ( "github.com/stretchr/testify/suite" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" "github.com/milvus-io/milvus/internal/util/funcutil" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" ) type InternalCntReducerSuite struct { diff --git a/internal/querynodev2/segments/default_limit_reducer.go b/internal/querynodev2/segments/default_limit_reducer.go index 58ea0a06fba04..38a135c6f779d 100644 --- a/internal/querynodev2/segments/default_limit_reducer.go +++ b/internal/querynodev2/segments/default_limit_reducer.go @@ -4,10 +4,10 @@ import ( "context" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" "github.com/milvus-io/milvus/internal/util/reduce" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" ) type defaultLimitReducer struct { diff --git a/internal/querynodev2/segments/index_attr_cache.go b/internal/querynodev2/segments/index_attr_cache.go index fd7b55843c45c..c58552fc17990 100644 --- a/internal/querynodev2/segments/index_attr_cache.go +++ b/internal/querynodev2/segments/index_attr_cache.go @@ -27,11 +27,11 @@ import ( "fmt" "unsafe" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/util/indexparamcheck" "github.com/milvus-io/milvus/internal/util/vecindexmgr" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/querynodev2/segments/index_attr_cache_test.go b/internal/querynodev2/segments/index_attr_cache_test.go index dc4a11a70ce49..14b05cc999ad8 100644 --- a/internal/querynodev2/segments/index_attr_cache_test.go +++ b/internal/querynodev2/segments/index_attr_cache_test.go @@ -22,10 +22,10 @@ import ( "github.com/stretchr/testify/suite" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/util/indexparamcheck" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querynodev2/segments/load_field_data_info.go b/internal/querynodev2/segments/load_field_data_info.go index 3625c06fc2d17..dd593120371cc 100644 --- a/internal/querynodev2/segments/load_field_data_info.go +++ b/internal/querynodev2/segments/load_field_data_info.go @@ -26,7 +26,7 @@ import ( "context" "unsafe" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" ) type LoadFieldDataInfo struct { diff --git a/internal/querynodev2/segments/load_index_info.go b/internal/querynodev2/segments/load_index_info.go index 07b2e3205cb4f..da60f2d3162c2 100644 --- a/internal/querynodev2/segments/load_index_info.go +++ b/internal/querynodev2/segments/load_index_info.go @@ -34,8 +34,8 @@ import ( "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/cgopb" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/cgopb" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querynodev2/segments/manager.go b/internal/querynodev2/segments/manager.go index 9f085576ff8fc..2c66302fb5621 100644 --- a/internal/querynodev2/segments/manager.go +++ b/internal/querynodev2/segments/manager.go @@ -33,12 +33,12 @@ import ( "golang.org/x/sync/singleflight" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querynodev2/segments/metricsutil" "github.com/milvus-io/milvus/pkg/eventlog" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/cache" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metautil" diff --git a/internal/querynodev2/segments/manager_test.go b/internal/querynodev2/segments/manager_test.go index f9d6ba5a04e3c..d6f215444895d 100644 --- a/internal/querynodev2/segments/manager_test.go +++ b/internal/querynodev2/segments/manager_test.go @@ -11,9 +11,9 @@ 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/mocks/util/mock_segcore" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/util/initcore" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querynodev2/segments/mock_collection_manager.go b/internal/querynodev2/segments/mock_collection_manager.go index 9440d7faee096..1e512316e3eb9 100644 --- a/internal/querynodev2/segments/mock_collection_manager.go +++ b/internal/querynodev2/segments/mock_collection_manager.go @@ -4,10 +4,10 @@ package segments import ( schemapb "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - querypb "github.com/milvus-io/milvus/internal/proto/querypb" + querypb "github.com/milvus-io/milvus/pkg/proto/querypb" mock "github.com/stretchr/testify/mock" - segcorepb "github.com/milvus-io/milvus/internal/proto/segcorepb" + segcorepb "github.com/milvus-io/milvus/pkg/proto/segcorepb" ) // MockCollectionManager is an autogenerated mock type for the CollectionManager type diff --git a/internal/querynodev2/segments/mock_loader.go b/internal/querynodev2/segments/mock_loader.go index d50d52b9c077a..bdefdc91a09e1 100644 --- a/internal/querynodev2/segments/mock_loader.go +++ b/internal/querynodev2/segments/mock_loader.go @@ -7,13 +7,13 @@ import ( commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - datapb "github.com/milvus-io/milvus/internal/proto/datapb" + datapb "github.com/milvus-io/milvus/pkg/proto/datapb" mock "github.com/stretchr/testify/mock" pkoracle "github.com/milvus-io/milvus/internal/querynodev2/pkoracle" - querypb "github.com/milvus-io/milvus/internal/proto/querypb" + querypb "github.com/milvus-io/milvus/pkg/proto/querypb" storage "github.com/milvus-io/milvus/internal/storage" diff --git a/internal/querynodev2/segments/mock_segment.go b/internal/querynodev2/segments/mock_segment.go index 1af3012ed3038..42245cf4d8ba9 100644 --- a/internal/querynodev2/segments/mock_segment.go +++ b/internal/querynodev2/segments/mock_segment.go @@ -7,7 +7,7 @@ import ( commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - datapb "github.com/milvus-io/milvus/internal/proto/datapb" + datapb "github.com/milvus-io/milvus/pkg/proto/datapb" metautil "github.com/milvus-io/milvus/pkg/util/metautil" @@ -15,11 +15,11 @@ import ( msgpb "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" - querypb "github.com/milvus-io/milvus/internal/proto/querypb" + querypb "github.com/milvus-io/milvus/pkg/proto/querypb" segcore "github.com/milvus-io/milvus/internal/util/segcore" - segcorepb "github.com/milvus-io/milvus/internal/proto/segcorepb" + segcorepb "github.com/milvus-io/milvus/pkg/proto/segcorepb" storage "github.com/milvus-io/milvus/internal/storage" ) diff --git a/internal/querynodev2/segments/mock_segment_manager.go b/internal/querynodev2/segments/mock_segment_manager.go index a4598e665d5d7..c5b665e56be81 100644 --- a/internal/querynodev2/segments/mock_segment_manager.go +++ b/internal/querynodev2/segments/mock_segment_manager.go @@ -9,7 +9,7 @@ import ( mock "github.com/stretchr/testify/mock" - querypb "github.com/milvus-io/milvus/internal/proto/querypb" + querypb "github.com/milvus-io/milvus/pkg/proto/querypb" ) // MockSegmentManager is an autogenerated mock type for the SegmentManager type diff --git a/internal/querynodev2/segments/reducer.go b/internal/querynodev2/segments/reducer.go index 8ef7a4102e800..5dc157c330d1b 100644 --- a/internal/querynodev2/segments/reducer.go +++ b/internal/querynodev2/segments/reducer.go @@ -6,11 +6,11 @@ import ( "github.com/samber/lo" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" "github.com/milvus-io/milvus/internal/util/segcore" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querynodev2/segments/reducer_test.go b/internal/querynodev2/segments/reducer_test.go index e52ea51c9ebb5..ed9a98f307fb5 100644 --- a/internal/querynodev2/segments/reducer_test.go +++ b/internal/querynodev2/segments/reducer_test.go @@ -5,8 +5,8 @@ import ( "github.com/stretchr/testify/suite" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" ) type ReducerFactorySuite struct { diff --git a/internal/querynodev2/segments/result.go b/internal/querynodev2/segments/result.go index 8e7adfdab3262..340b28a50f5ab 100644 --- a/internal/querynodev2/segments/result.go +++ b/internal/querynodev2/segments/result.go @@ -27,13 +27,13 @@ import ( "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" "github.com/milvus-io/milvus/internal/util/reduce" "github.com/milvus-io/milvus/internal/util/segcore" typeutil2 "github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/querynodev2/segments/result_sorter.go b/internal/querynodev2/segments/result_sorter.go index f488c37dab69b..32f4ad9dd829c 100644 --- a/internal/querynodev2/segments/result_sorter.go +++ b/internal/querynodev2/segments/result_sorter.go @@ -2,7 +2,7 @@ package segments import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querynodev2/segments/result_test.go b/internal/querynodev2/segments/result_test.go index 60c5332cd9aec..44f141b2e36e7 100644 --- a/internal/querynodev2/segments/result_test.go +++ b/internal/querynodev2/segments/result_test.go @@ -28,11 +28,11 @@ import ( "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/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" "github.com/milvus-io/milvus/internal/util/reduce" "github.com/milvus-io/milvus/internal/util/segcore" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" "github.com/milvus-io/milvus/pkg/util/metric" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/querynodev2/segments/retrieve.go b/internal/querynodev2/segments/retrieve.go index c615dcf79f8e0..40696e2998451 100644 --- a/internal/querynodev2/segments/retrieve.go +++ b/internal/querynodev2/segments/retrieve.go @@ -24,12 +24,12 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" "github.com/milvus-io/milvus/internal/util/streamrpc" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/timerecord" diff --git a/internal/querynodev2/segments/retrieve_test.go b/internal/querynodev2/segments/retrieve_test.go index 0e9193cc1c144..b377df3ee688b 100644 --- a/internal/querynodev2/segments/retrieve_test.go +++ b/internal/querynodev2/segments/retrieve_test.go @@ -26,13 +26,13 @@ import ( "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/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/initcore" "github.com/milvus-io/milvus/internal/util/segcore" "github.com/milvus-io/milvus/internal/util/streamrpc" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querynodev2/segments/search_test.go b/internal/querynodev2/segments/search_test.go index f731b7f9817a7..eac94af7ec591 100644 --- a/internal/querynodev2/segments/search_test.go +++ b/internal/querynodev2/segments/search_test.go @@ -25,10 +25,10 @@ import ( "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/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" storage "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/initcore" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querynodev2/segments/segment.go b/internal/querynodev2/segments/segment.go index 78401e5e73a0c..e2699608d8fc5 100644 --- a/internal/querynodev2/segments/segment.go +++ b/internal/querynodev2/segments/segment.go @@ -43,11 +43,6 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "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/proto/cgopb" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexcgopb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" "github.com/milvus-io/milvus/internal/querycoordv2/params" "github.com/milvus-io/milvus/internal/querynodev2/pkoracle" "github.com/milvus-io/milvus/internal/querynodev2/segments/state" @@ -58,6 +53,11 @@ import ( "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/cgopb" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexcgopb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/indexparams" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/querynodev2/segments/segment_filter.go b/internal/querynodev2/segments/segment_filter.go index 51a334f116969..080fae5841450 100644 --- a/internal/querynodev2/segments/segment_filter.go +++ b/internal/querynodev2/segments/segment_filter.go @@ -17,7 +17,7 @@ package segments import ( - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/metautil" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querynodev2/segments/segment_interface.go b/internal/querynodev2/segments/segment_interface.go index 400886ccd5edf..816d8b89f557f 100644 --- a/internal/querynodev2/segments/segment_interface.go +++ b/internal/querynodev2/segments/segment_interface.go @@ -20,11 +20,11 @@ import ( "context" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/segcore" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" "github.com/milvus-io/milvus/pkg/util/metautil" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querynodev2/segments/segment_l0.go b/internal/querynodev2/segments/segment_l0.go index cab1f64b7645a..a889e9f1909ac 100644 --- a/internal/querynodev2/segments/segment_l0.go +++ b/internal/querynodev2/segments/segment_l0.go @@ -23,12 +23,12 @@ import ( "github.com/samber/lo" "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" storage "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/segcore" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querynodev2/segments/segment_loader.go b/internal/querynodev2/segments/segment_loader.go index 390f4a460d6ef..f7ae5c0712000 100644 --- a/internal/querynodev2/segments/segment_loader.go +++ b/internal/querynodev2/segments/segment_loader.go @@ -42,14 +42,14 @@ 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/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querynodev2/pkoracle" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/segcore" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/contextutil" diff --git a/internal/querynodev2/segments/segment_loader_test.go b/internal/querynodev2/segments/segment_loader_test.go index 0c888c73b5ba7..8a388194579db 100644 --- a/internal/querynodev2/segments/segment_loader_test.go +++ b/internal/querynodev2/segments/segment_loader_test.go @@ -31,12 +31,12 @@ 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/mocks/util/mock_segcore" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/indexparamcheck" "github.com/milvus-io/milvus/internal/util/initcore" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/contextutil" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/querynodev2/segments/segment_test.go b/internal/querynodev2/segments/segment_test.go index 42188cc42f74c..41442abf16691 100644 --- a/internal/querynodev2/segments/segment_test.go +++ b/internal/querynodev2/segments/segment_test.go @@ -13,10 +13,10 @@ import ( "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/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" storage "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/initcore" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/querynodev2/segments/utils.go b/internal/querynodev2/segments/utils.go index 3c1108e937a92..64f7dd518a32e 100644 --- a/internal/querynodev2/segments/utils.go +++ b/internal/querynodev2/segments/utils.go @@ -23,9 +23,6 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/params" "github.com/milvus-io/milvus/internal/querynodev2/segments/metricsutil" "github.com/milvus-io/milvus/internal/storage" @@ -34,6 +31,9 @@ import ( "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/contextutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/querynodev2/segments/utils_test.go b/internal/querynodev2/segments/utils_test.go index 881068eb3eb19..28f4b609e502b 100644 --- a/internal/querynodev2/segments/utils_test.go +++ b/internal/querynodev2/segments/utils_test.go @@ -7,9 +7,9 @@ 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/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querynodev2/server_test.go b/internal/querynodev2/server_test.go index a6ea597b1875c..e955df19320ef 100644 --- a/internal/querynodev2/server_test.go +++ b/internal/querynodev2/server_test.go @@ -34,11 +34,11 @@ import ( "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/internal/mocks/util/searchutil/mock_optimizers" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querynodev2/segments" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/dependency" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/querynodev2/services.go b/internal/querynodev2/services.go index 446f5fded4fa0..3f5af3f7d0745 100644 --- a/internal/querynodev2/services.go +++ b/internal/querynodev2/services.go @@ -33,11 +33,6 @@ import ( "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/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" "github.com/milvus-io/milvus/internal/querynodev2/delegator" "github.com/milvus-io/milvus/internal/querynodev2/segments" "github.com/milvus-io/milvus/internal/querynodev2/tasks" @@ -47,6 +42,11 @@ import ( "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/funcutil" diff --git a/internal/querynodev2/services_test.go b/internal/querynodev2/services_test.go index 290413c11b4df..a28367ec54895 100644 --- a/internal/querynodev2/services_test.go +++ b/internal/querynodev2/services_test.go @@ -39,10 +39,6 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/mocks/util/mock_segcore" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querynodev2/delegator" "github.com/milvus-io/milvus/internal/querynodev2/segments" "github.com/milvus-io/milvus/internal/storage" @@ -51,6 +47,10 @@ import ( "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/funcutil" diff --git a/internal/querynodev2/tasks/query_stream_task.go b/internal/querynodev2/tasks/query_stream_task.go index cdb3c3c2e99ba..348b414df2aa1 100644 --- a/internal/querynodev2/tasks/query_stream_task.go +++ b/internal/querynodev2/tasks/query_stream_task.go @@ -3,12 +3,12 @@ package tasks import ( "context" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querynodev2/segments" "github.com/milvus-io/milvus/internal/util/searchutil/scheduler" "github.com/milvus-io/milvus/internal/util/segcore" "github.com/milvus-io/milvus/internal/util/streamrpc" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" ) var _ scheduler.Task = &QueryStreamTask{} diff --git a/internal/querynodev2/tasks/query_task.go b/internal/querynodev2/tasks/query_task.go index 7099e83defc22..d84e192182f8e 100644 --- a/internal/querynodev2/tasks/query_task.go +++ b/internal/querynodev2/tasks/query_task.go @@ -11,13 +11,13 @@ import ( "go.opentelemetry.io/otel/trace" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" "github.com/milvus-io/milvus/internal/querynodev2/segments" "github.com/milvus-io/milvus/internal/util/searchutil/scheduler" "github.com/milvus-io/milvus/internal/util/segcore" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/timerecord" diff --git a/internal/querynodev2/tasks/search_task.go b/internal/querynodev2/tasks/search_task.go index 9c16c13253dfd..5056ac8889ec2 100644 --- a/internal/querynodev2/tasks/search_task.go +++ b/internal/querynodev2/tasks/search_task.go @@ -17,13 +17,13 @@ import ( "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querynodev2/segments" "github.com/milvus-io/milvus/internal/util/searchutil/scheduler" "github.com/milvus-io/milvus/internal/util/segcore" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/rootcoord/alter_collection_task.go b/internal/rootcoord/alter_collection_task.go index 97dddc29312cf..735284e559aad 100644 --- a/internal/rootcoord/alter_collection_task.go +++ b/internal/rootcoord/alter_collection_task.go @@ -28,11 +28,11 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/metastore/model" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/util/proxyutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/rootcoord/alter_database_task.go b/internal/rootcoord/alter_database_task.go index 6d21e841b3355..9be65b104fb38 100644 --- a/internal/rootcoord/alter_database_task.go +++ b/internal/rootcoord/alter_database_task.go @@ -27,12 +27,12 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/metastore/model" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/util/proxyutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/rootcoord/alter_database_task_test.go b/internal/rootcoord/alter_database_task_test.go index 47b66e176b4e7..e38f292183775 100644 --- a/internal/rootcoord/alter_database_task_test.go +++ b/internal/rootcoord/alter_database_task_test.go @@ -27,10 +27,10 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus/internal/metastore/model" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" mockrootcoord "github.com/milvus-io/milvus/internal/rootcoord/mocks" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/funcutil" ) diff --git a/internal/rootcoord/broker.go b/internal/rootcoord/broker.go index 94a3bcfe88a7c..8a74af1f65974 100644 --- a/internal/rootcoord/broker.go +++ b/internal/rootcoord/broker.go @@ -27,10 +27,10 @@ import ( "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/metastore/model" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/rootcoord/broker_test.go b/internal/rootcoord/broker_test.go index 4d4102d560d43..e099e3025b156 100644 --- a/internal/rootcoord/broker_test.go +++ b/internal/rootcoord/broker_test.go @@ -28,10 +28,10 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus/internal/metastore/model" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" - pb "github.com/milvus-io/milvus/internal/proto/etcdpb" - "github.com/milvus-io/milvus/internal/proto/indexpb" mockrootcoord "github.com/milvus-io/milvus/internal/rootcoord/mocks" + "github.com/milvus-io/milvus/pkg/proto/datapb" + pb "github.com/milvus-io/milvus/pkg/proto/etcdpb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/rootcoord/constrant_test.go b/internal/rootcoord/constrant_test.go index 6e8bb9e734ce0..f4c60d3d3e9fa 100644 --- a/internal/rootcoord/constrant_test.go +++ b/internal/rootcoord/constrant_test.go @@ -25,8 +25,8 @@ import ( "github.com/milvus-io/milvus/internal/metastore/mocks" "github.com/milvus-io/milvus/internal/metastore/model" - pb "github.com/milvus-io/milvus/internal/proto/etcdpb" mocktso "github.com/milvus-io/milvus/internal/tso/mocks" + pb "github.com/milvus-io/milvus/pkg/proto/etcdpb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/rootcoord/create_collection_task.go b/internal/rootcoord/create_collection_task.go index 00f1804127aa4..99f386f28e8ac 100644 --- a/internal/rootcoord/create_collection_task.go +++ b/internal/rootcoord/create_collection_task.go @@ -32,12 +32,12 @@ import ( "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" - pb "github.com/milvus-io/milvus/internal/proto/etcdpb" "github.com/milvus-io/milvus/internal/util/proxyutil" "github.com/milvus-io/milvus/internal/util/streamingutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" ms "github.com/milvus-io/milvus/pkg/mq/msgstream" + pb "github.com/milvus-io/milvus/pkg/proto/etcdpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/message/adaptor" "github.com/milvus-io/milvus/pkg/util/commonpbutil" diff --git a/internal/rootcoord/create_collection_task_test.go b/internal/rootcoord/create_collection_task_test.go index 62d44562cce0d..a27af1cf7a61a 100644 --- a/internal/rootcoord/create_collection_task_test.go +++ b/internal/rootcoord/create_collection_task_test.go @@ -32,9 +32,9 @@ import ( "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/metastore/model" - "github.com/milvus-io/milvus/internal/proto/datapb" mockrootcoord "github.com/milvus-io/milvus/internal/rootcoord/mocks" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/rootcoord/create_db_task.go b/internal/rootcoord/create_db_task.go index 5db6dbeef3f9d..ea926100a5910 100644 --- a/internal/rootcoord/create_db_task.go +++ b/internal/rootcoord/create_db_task.go @@ -21,7 +21,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus/internal/metastore/model" - "github.com/milvus-io/milvus/internal/proto/etcdpb" + "github.com/milvus-io/milvus/pkg/proto/etcdpb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/rootcoord/create_partition_task.go b/internal/rootcoord/create_partition_task.go index 7d76720c8006e..e26cf068d4d66 100644 --- a/internal/rootcoord/create_partition_task.go +++ b/internal/rootcoord/create_partition_task.go @@ -25,10 +25,10 @@ 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/internal/metastore/model" - pb "github.com/milvus-io/milvus/internal/proto/etcdpb" "github.com/milvus-io/milvus/internal/util/proxyutil" "github.com/milvus-io/milvus/internal/util/streamingutil" "github.com/milvus-io/milvus/pkg/log" + pb "github.com/milvus-io/milvus/pkg/proto/etcdpb" ) type createPartitionTask struct { diff --git a/internal/rootcoord/create_partition_task_test.go b/internal/rootcoord/create_partition_task_test.go index bf830fd04de75..14b6c97783d5d 100644 --- a/internal/rootcoord/create_partition_task_test.go +++ b/internal/rootcoord/create_partition_task_test.go @@ -26,8 +26,8 @@ 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/internal/metastore/model" - "github.com/milvus-io/milvus/internal/proto/etcdpb" mockrootcoord "github.com/milvus-io/milvus/internal/rootcoord/mocks" + "github.com/milvus-io/milvus/pkg/proto/etcdpb" "github.com/milvus-io/milvus/pkg/util/funcutil" ) diff --git a/internal/rootcoord/describe_db_task.go b/internal/rootcoord/describe_db_task.go index d711316b95238..7f064e3164f37 100644 --- a/internal/rootcoord/describe_db_task.go +++ b/internal/rootcoord/describe_db_task.go @@ -19,7 +19,7 @@ package rootcoord import ( "context" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/rootcoord/describe_db_task_test.go b/internal/rootcoord/describe_db_task_test.go index 2d9b8a4f5ef32..71fb8b4615638 100644 --- a/internal/rootcoord/describe_db_task_test.go +++ b/internal/rootcoord/describe_db_task_test.go @@ -25,8 +25,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus/internal/metastore/model" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" mockrootcoord "github.com/milvus-io/milvus/internal/rootcoord/mocks" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util" ) diff --git a/internal/rootcoord/drop_collection_task.go b/internal/rootcoord/drop_collection_task.go index 795842a51b609..3f231eef37a9a 100644 --- a/internal/rootcoord/drop_collection_task.go +++ b/internal/rootcoord/drop_collection_task.go @@ -26,9 +26,9 @@ 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/internal/metastore/model" - pb "github.com/milvus-io/milvus/internal/proto/etcdpb" "github.com/milvus-io/milvus/internal/util/proxyutil" "github.com/milvus-io/milvus/pkg/log" + pb "github.com/milvus-io/milvus/pkg/proto/etcdpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/rootcoord/drop_partition_task.go b/internal/rootcoord/drop_partition_task.go index 648b3c74ddd6f..0fa7037bbf1e2 100644 --- a/internal/rootcoord/drop_partition_task.go +++ b/internal/rootcoord/drop_partition_task.go @@ -25,10 +25,10 @@ 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/internal/metastore/model" - pb "github.com/milvus-io/milvus/internal/proto/etcdpb" "github.com/milvus-io/milvus/internal/util/proxyutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + pb "github.com/milvus-io/milvus/pkg/proto/etcdpb" ) type dropPartitionTask struct { diff --git a/internal/rootcoord/expire_cache.go b/internal/rootcoord/expire_cache.go index ba296d206b06e..31bdaa4b520fb 100644 --- a/internal/rootcoord/expire_cache.go +++ b/internal/rootcoord/expire_cache.go @@ -19,8 +19,8 @@ package rootcoord import ( "context" - "github.com/milvus-io/milvus/internal/proto/proxypb" "github.com/milvus-io/milvus/internal/util/proxyutil" + "github.com/milvus-io/milvus/pkg/proto/proxypb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/rootcoord/expire_cache_test.go b/internal/rootcoord/expire_cache_test.go index 12a50099b37ff..8ab0513fbee10 100644 --- a/internal/rootcoord/expire_cache_test.go +++ b/internal/rootcoord/expire_cache_test.go @@ -22,8 +22,8 @@ import ( "github.com/stretchr/testify/assert" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/proxypb" "github.com/milvus-io/milvus/internal/util/proxyutil" + "github.com/milvus-io/milvus/pkg/proto/proxypb" ) func Test_expireCacheConfig_apply(t *testing.T) { diff --git a/internal/rootcoord/garbage_collector_test.go b/internal/rootcoord/garbage_collector_test.go index f49da0e00c04a..563d95c4b1d48 100644 --- a/internal/rootcoord/garbage_collector_test.go +++ b/internal/rootcoord/garbage_collector_test.go @@ -30,11 +30,11 @@ import ( "github.com/milvus-io/milvus/internal/metastore/model" "github.com/milvus-io/milvus/internal/mocks" "github.com/milvus-io/milvus/internal/mocks/distributed/mock_streaming" - "github.com/milvus-io/milvus/internal/proto/querypb" mockrootcoord "github.com/milvus-io/milvus/internal/rootcoord/mocks" mocktso "github.com/milvus-io/milvus/internal/tso/mocks" "github.com/milvus-io/milvus/internal/util/streamingutil" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/rootcoord/meta_table.go b/internal/rootcoord/meta_table.go index 9159caac0af81..1de392ff1f792 100644 --- a/internal/rootcoord/meta_table.go +++ b/internal/rootcoord/meta_table.go @@ -29,13 +29,13 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus/internal/metastore" "github.com/milvus-io/milvus/internal/metastore/model" - pb "github.com/milvus-io/milvus/internal/proto/etcdpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/tso" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + pb "github.com/milvus-io/milvus/pkg/proto/etcdpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/contextutil" "github.com/milvus-io/milvus/pkg/util/funcutil" diff --git a/internal/rootcoord/meta_table_test.go b/internal/rootcoord/meta_table_test.go index a143cbcf97431..1513dd9a3ae6e 100644 --- a/internal/rootcoord/meta_table_test.go +++ b/internal/rootcoord/meta_table_test.go @@ -32,10 +32,10 @@ import ( "github.com/milvus-io/milvus/internal/metastore/kv/rootcoord" "github.com/milvus-io/milvus/internal/metastore/mocks" "github.com/milvus-io/milvus/internal/metastore/model" - pb "github.com/milvus-io/milvus/internal/proto/etcdpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" mocktso "github.com/milvus-io/milvus/internal/tso/mocks" "github.com/milvus-io/milvus/pkg/common" + pb "github.com/milvus-io/milvus/pkg/proto/etcdpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/rootcoord/mock_test.go b/internal/rootcoord/mock_test.go index b3b6429452097..b7c1fa4c8e383 100644 --- a/internal/rootcoord/mock_test.go +++ b/internal/rootcoord/mock_test.go @@ -31,12 +31,6 @@ import ( "github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/metastore/model" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/datapb" - pb "github.com/milvus-io/milvus/internal/proto/etcdpb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/proxypb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/tso" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/dependency" @@ -44,6 +38,12 @@ import ( "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/datapb" + pb "github.com/milvus-io/milvus/pkg/proto/etcdpb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/proxypb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/rootcoord/mocks/meta_table.go b/internal/rootcoord/mocks/meta_table.go index cb617224a9eec..f6b60d53a001e 100644 --- a/internal/rootcoord/mocks/meta_table.go +++ b/internal/rootcoord/mocks/meta_table.go @@ -5,8 +5,8 @@ package mockrootcoord import ( context "context" - etcdpb "github.com/milvus-io/milvus/internal/proto/etcdpb" - internalpb "github.com/milvus-io/milvus/internal/proto/internalpb" + etcdpb "github.com/milvus-io/milvus/pkg/proto/etcdpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" @@ -14,7 +14,7 @@ import ( model "github.com/milvus-io/milvus/internal/metastore/model" - rootcoordpb "github.com/milvus-io/milvus/internal/proto/rootcoordpb" + rootcoordpb "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" ) // IMetaTable is an autogenerated mock type for the IMetaTable type diff --git a/internal/rootcoord/quota_center.go b/internal/rootcoord/quota_center.go index 98e42f5e24676..9f88d785ba1cc 100644 --- a/internal/rootcoord/quota_center.go +++ b/internal/rootcoord/quota_center.go @@ -33,8 +33,6 @@ 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/internal/metastore/model" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/proxypb" "github.com/milvus-io/milvus/internal/tso" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/proxyutil" @@ -44,6 +42,8 @@ import ( "github.com/milvus-io/milvus/pkg/config" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/proxypb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/rootcoord/quota_center_test.go b/internal/rootcoord/quota_center_test.go index 15b306d582a35..7ec4ea9482660 100644 --- a/internal/rootcoord/quota_center_test.go +++ b/internal/rootcoord/quota_center_test.go @@ -36,12 +36,12 @@ import ( "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/metastore/model" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/internalpb" mockrootcoord "github.com/milvus-io/milvus/internal/rootcoord/mocks" "github.com/milvus-io/milvus/internal/util/proxyutil" interalratelimitutil "github.com/milvus-io/milvus/internal/util/ratelimitutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/rootcoord/rbac_task.go b/internal/rootcoord/rbac_task.go index af5efbc007fa0..8d826cc62afdd 100644 --- a/internal/rootcoord/rbac_task.go +++ b/internal/rootcoord/rbac_task.go @@ -27,9 +27,9 @@ import ( "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" - "github.com/milvus-io/milvus/internal/proto/proxypb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/proxypb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/rootcoord/root_coord.go b/internal/rootcoord/root_coord.go index ecdba34ee7f7d..7839e358da477 100644 --- a/internal/rootcoord/root_coord.go +++ b/internal/rootcoord/root_coord.go @@ -45,10 +45,6 @@ import ( "github.com/milvus-io/milvus/internal/metastore" kvmetestore "github.com/milvus-io/milvus/internal/metastore/kv/rootcoord" "github.com/milvus-io/milvus/internal/metastore/model" - pb "github.com/milvus-io/milvus/internal/proto/etcdpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/proxypb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" streamingcoord "github.com/milvus-io/milvus/internal/streamingcoord/server" tso2 "github.com/milvus-io/milvus/internal/tso" "github.com/milvus-io/milvus/internal/types" @@ -61,6 +57,10 @@ import ( "github.com/milvus-io/milvus/pkg/kv" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + pb "github.com/milvus-io/milvus/pkg/proto/etcdpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/proxypb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/crypto" diff --git a/internal/rootcoord/root_coord_test.go b/internal/rootcoord/root_coord_test.go index 9df0d5dd1bdec..402a6b2242914 100644 --- a/internal/rootcoord/root_coord_test.go +++ b/internal/rootcoord/root_coord_test.go @@ -33,15 +33,15 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus/internal/metastore/model" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/etcdpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/proxypb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" mockrootcoord "github.com/milvus-io/milvus/internal/rootcoord/mocks" "github.com/milvus-io/milvus/internal/util/dependency" kvfactory "github.com/milvus-io/milvus/internal/util/dependency/kv" "github.com/milvus-io/milvus/internal/util/proxyutil" "github.com/milvus-io/milvus/internal/util/sessionutil" + "github.com/milvus-io/milvus/pkg/proto/etcdpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/proxypb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/funcutil" diff --git a/internal/rootcoord/step.go b/internal/rootcoord/step.go index 795dc46be2b23..4cfce37bece96 100644 --- a/internal/rootcoord/step.go +++ b/internal/rootcoord/step.go @@ -28,9 +28,9 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/distributed/streaming" "github.com/milvus-io/milvus/internal/metastore/model" - pb "github.com/milvus-io/milvus/internal/proto/etcdpb" "github.com/milvus-io/milvus/internal/util/proxyutil" "github.com/milvus-io/milvus/pkg/log" + pb "github.com/milvus-io/milvus/pkg/proto/etcdpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/util/commonpbutil" ) diff --git a/internal/rootcoord/task_test.go b/internal/rootcoord/task_test.go index 043885eab9e58..1ecb5bea53bb1 100644 --- a/internal/rootcoord/task_test.go +++ b/internal/rootcoord/task_test.go @@ -28,8 +28,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus/internal/metastore/model" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" mockrootcoord "github.com/milvus-io/milvus/internal/rootcoord/mocks" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" ) func TestLockerKey(t *testing.T) { diff --git a/internal/rootcoord/timestamp_bench_test.go b/internal/rootcoord/timestamp_bench_test.go index e8526af5de338..bf47c68634f11 100644 --- a/internal/rootcoord/timestamp_bench_test.go +++ b/internal/rootcoord/timestamp_bench_test.go @@ -24,10 +24,10 @@ import ( clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/tso" "github.com/milvus-io/milvus/internal/util/tsoutil" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/rootcoord/timeticksync.go b/internal/rootcoord/timeticksync.go index 6c2b38331a310..dafbf25ccea8a 100644 --- a/internal/rootcoord/timeticksync.go +++ b/internal/rootcoord/timeticksync.go @@ -26,12 +26,12 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/streamingutil" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/timerecord" "github.com/milvus-io/milvus/pkg/util/tsoutil" diff --git a/internal/rootcoord/timeticksync_test.go b/internal/rootcoord/timeticksync_test.go index 2320471c87bf2..2b92b328e83d3 100644 --- a/internal/rootcoord/timeticksync_test.go +++ b/internal/rootcoord/timeticksync_test.go @@ -24,9 +24,9 @@ import ( "github.com/stretchr/testify/assert" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/internal/util/sessionutil" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/storage/binlog_iterator_test.go b/internal/storage/binlog_iterator_test.go index 343e7a835f5df..99f5a51e34331 100644 --- a/internal/storage/binlog_iterator_test.go +++ b/internal/storage/binlog_iterator_test.go @@ -25,8 +25,8 @@ 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/proto/etcdpb" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/etcdpb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/storage/data_codec.go b/internal/storage/data_codec.go index b7086387e0acd..5fbe1e039e27e 100644 --- a/internal/storage/data_codec.go +++ b/internal/storage/data_codec.go @@ -26,8 +26,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/json" - "github.com/milvus-io/milvus/internal/proto/etcdpb" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/etcdpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metautil" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/storage/data_codec_test.go b/internal/storage/data_codec_test.go index 614167b2bf1d8..679ce136986d5 100644 --- a/internal/storage/data_codec_test.go +++ b/internal/storage/data_codec_test.go @@ -27,9 +27,9 @@ 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/proto/etcdpb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/etcdpb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/storage/data_sorter_test.go b/internal/storage/data_sorter_test.go index 1ca05285f9a50..12f0e72d4e083 100644 --- a/internal/storage/data_sorter_test.go +++ b/internal/storage/data_sorter_test.go @@ -23,7 +23,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/etcdpb" + "github.com/milvus-io/milvus/pkg/proto/etcdpb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/storage/field_value.go b/internal/storage/field_value.go index f9d59d89dcb89..b5b746f6bcd04 100644 --- a/internal/storage/field_value.go +++ b/internal/storage/field_value.go @@ -23,8 +23,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/json" - "github.com/milvus-io/milvus/internal/proto/planpb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/planpb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/storage/print_binlog_test.go b/internal/storage/print_binlog_test.go index 5c302965d0e32..5ac813f8a1b41 100644 --- a/internal/storage/print_binlog_test.go +++ b/internal/storage/print_binlog_test.go @@ -28,8 +28,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "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/proto/etcdpb" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/etcdpb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/tsoutil" "github.com/milvus-io/milvus/pkg/util/uniquegenerator" diff --git a/internal/storage/utils.go b/internal/storage/utils.go index 5e444027dcc48..d348eef5385f2 100644 --- a/internal/storage/utils.go +++ b/internal/storage/utils.go @@ -34,10 +34,10 @@ 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/proto/segcorepb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/streamingcoord/client/assignment/assignment_impl.go b/internal/streamingcoord/client/assignment/assignment_impl.go index 134517d46ea15..4d18614310b86 100644 --- a/internal/streamingcoord/client/assignment/assignment_impl.go +++ b/internal/streamingcoord/client/assignment/assignment_impl.go @@ -11,7 +11,7 @@ import ( "github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc" "github.com/milvus-io/milvus/internal/util/streamingutil/status" "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/syncutil" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/streamingcoord/client/assignment/assignment_test.go b/internal/streamingcoord/client/assignment/assignment_test.go index 60f05873e8c54..9e0be7a9fc339 100644 --- a/internal/streamingcoord/client/assignment/assignment_test.go +++ b/internal/streamingcoord/client/assignment/assignment_test.go @@ -13,7 +13,7 @@ import ( "github.com/milvus-io/milvus/internal/mocks/util/streamingutil/service/mock_lazygrpc" "github.com/milvus-io/milvus/internal/util/streamingutil/status" "github.com/milvus-io/milvus/pkg/mocks/streaming/proto/mock_streamingpb" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/streamingcoord/client/assignment/discoverer.go b/internal/streamingcoord/client/assignment/discoverer.go index b9f92f27e5e0b..357d0ab5bba68 100644 --- a/internal/streamingcoord/client/assignment/discoverer.go +++ b/internal/streamingcoord/client/assignment/discoverer.go @@ -6,7 +6,7 @@ import ( "github.com/milvus-io/milvus/internal/util/streamingutil/status" "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/streamingcoord/client/broadcast/broadcast_impl.go b/internal/streamingcoord/client/broadcast/broadcast_impl.go index b6296748d1eba..5c5c825b3059d 100644 --- a/internal/streamingcoord/client/broadcast/broadcast_impl.go +++ b/internal/streamingcoord/client/broadcast/broadcast_impl.go @@ -4,8 +4,8 @@ import ( "context" "github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc" - "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/messagespb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/types" ) diff --git a/internal/streamingcoord/client/client.go b/internal/streamingcoord/client/client.go index 07f0937360bfd..79ef36053d8d9 100644 --- a/internal/streamingcoord/client/client.go +++ b/internal/streamingcoord/client/client.go @@ -18,7 +18,7 @@ import ( "github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc" "github.com/milvus-io/milvus/internal/util/streamingutil/service/resolver" "github.com/milvus-io/milvus/internal/util/streamingutil/util" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/tracer" diff --git a/internal/streamingcoord/server/balancer/balancer_test.go b/internal/streamingcoord/server/balancer/balancer_test.go index f0a738044c9b5..b794527ca7ff6 100644 --- a/internal/streamingcoord/server/balancer/balancer_test.go +++ b/internal/streamingcoord/server/balancer/balancer_test.go @@ -13,7 +13,7 @@ import ( "github.com/milvus-io/milvus/internal/streamingcoord/server/balancer" _ "github.com/milvus-io/milvus/internal/streamingcoord/server/balancer/policy" "github.com/milvus-io/milvus/internal/streamingcoord/server/resource" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/streamingcoord/server/balancer/channel/manager.go b/internal/streamingcoord/server/balancer/channel/manager.go index 9d77fe45b1b74..dd8bd8163d44c 100644 --- a/internal/streamingcoord/server/balancer/channel/manager.go +++ b/internal/streamingcoord/server/balancer/channel/manager.go @@ -7,7 +7,7 @@ import ( "github.com/cockroachdb/errors" "github.com/milvus-io/milvus/internal/streamingcoord/server/resource" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/syncutil" diff --git a/internal/streamingcoord/server/balancer/channel/manager_test.go b/internal/streamingcoord/server/balancer/channel/manager_test.go index bbb01d0280445..0d948d8792b9f 100644 --- a/internal/streamingcoord/server/balancer/channel/manager_test.go +++ b/internal/streamingcoord/server/balancer/channel/manager_test.go @@ -10,7 +10,7 @@ import ( "github.com/milvus-io/milvus/internal/mocks/mock_metastore" "github.com/milvus-io/milvus/internal/streamingcoord/server/resource" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/streamingcoord/server/balancer/channel/metrics.go b/internal/streamingcoord/server/balancer/channel/metrics.go index 767c60fa075b9..9d9c01d9ff257 100644 --- a/internal/streamingcoord/server/balancer/channel/metrics.go +++ b/internal/streamingcoord/server/balancer/channel/metrics.go @@ -6,7 +6,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/milvus-io/milvus/pkg/metrics" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/streamingcoord/server/balancer/channel/pchannel.go b/internal/streamingcoord/server/balancer/channel/pchannel.go index 0c832f4b937f1..90fcafc56aa7e 100644 --- a/internal/streamingcoord/server/balancer/channel/pchannel.go +++ b/internal/streamingcoord/server/balancer/channel/pchannel.go @@ -3,7 +3,7 @@ package channel import ( "google.golang.org/protobuf/proto" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/types" ) diff --git a/internal/streamingcoord/server/balancer/channel/pchannel_test.go b/internal/streamingcoord/server/balancer/channel/pchannel_test.go index 72ee907d9ccc9..b5feda9261808 100644 --- a/internal/streamingcoord/server/balancer/channel/pchannel_test.go +++ b/internal/streamingcoord/server/balancer/channel/pchannel_test.go @@ -5,7 +5,7 @@ import ( "github.com/stretchr/testify/assert" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/types" ) diff --git a/internal/streamingcoord/server/broadcaster/broadcaster_impl.go b/internal/streamingcoord/server/broadcaster/broadcaster_impl.go index 2da0e0679f907..8019ac9c779d4 100644 --- a/internal/streamingcoord/server/broadcaster/broadcaster_impl.go +++ b/internal/streamingcoord/server/broadcaster/broadcaster_impl.go @@ -10,8 +10,8 @@ import ( "github.com/milvus-io/milvus/internal/streamingcoord/server/resource" "github.com/milvus-io/milvus/internal/util/streamingutil/status" "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/messagespb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/contextutil" diff --git a/internal/streamingcoord/server/broadcaster/broadcaster_test.go b/internal/streamingcoord/server/broadcaster/broadcaster_test.go index 624535f1c8755..de0cc86b3b887 100644 --- a/internal/streamingcoord/server/broadcaster/broadcaster_test.go +++ b/internal/streamingcoord/server/broadcaster/broadcaster_test.go @@ -18,8 +18,8 @@ import ( "github.com/milvus-io/milvus/internal/streamingcoord/server/resource" internaltypes "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/idalloc" - "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/messagespb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/walimplstest" diff --git a/internal/streamingcoord/server/broadcaster/task.go b/internal/streamingcoord/server/broadcaster/task.go index 52a2b0e77d0c6..fff789e200729 100644 --- a/internal/streamingcoord/server/broadcaster/task.go +++ b/internal/streamingcoord/server/broadcaster/task.go @@ -9,7 +9,7 @@ import ( "github.com/milvus-io/milvus/internal/streamingcoord/server/resource" "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/syncutil" diff --git a/internal/streamingcoord/server/server.go b/internal/streamingcoord/server/server.go index f465d1b4b6cfc..67726bcf01e06 100644 --- a/internal/streamingcoord/server/server.go +++ b/internal/streamingcoord/server/server.go @@ -14,7 +14,7 @@ import ( "github.com/milvus-io/milvus/internal/util/streamingutil" "github.com/milvus-io/milvus/internal/util/streamingutil/util" "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/syncutil" ) diff --git a/internal/streamingcoord/server/service/assignment.go b/internal/streamingcoord/server/service/assignment.go index 8c26cb1996205..e5b22e79780cf 100644 --- a/internal/streamingcoord/server/service/assignment.go +++ b/internal/streamingcoord/server/service/assignment.go @@ -6,7 +6,7 @@ import ( "github.com/milvus-io/milvus/internal/streamingcoord/server/balancer" "github.com/milvus-io/milvus/internal/streamingcoord/server/service/discover" "github.com/milvus-io/milvus/pkg/metrics" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/syncutil" ) diff --git a/internal/streamingcoord/server/service/broadcast.go b/internal/streamingcoord/server/service/broadcast.go index 6d192615e32d4..2bdcf7296fae8 100644 --- a/internal/streamingcoord/server/service/broadcast.go +++ b/internal/streamingcoord/server/service/broadcast.go @@ -4,7 +4,7 @@ import ( "context" "github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/util/syncutil" ) diff --git a/internal/streamingcoord/server/service/discover/discover_grpc_server_helper.go b/internal/streamingcoord/server/service/discover/discover_grpc_server_helper.go index dadbf63e345e5..11129a7a9c5ce 100644 --- a/internal/streamingcoord/server/service/discover/discover_grpc_server_helper.go +++ b/internal/streamingcoord/server/service/discover/discover_grpc_server_helper.go @@ -1,7 +1,7 @@ package discover import ( - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/streamingcoord/server/service/discover/discover_server.go b/internal/streamingcoord/server/service/discover/discover_server.go index d4aaa48b4ebf2..5c82f297e2f29 100644 --- a/internal/streamingcoord/server/service/discover/discover_server.go +++ b/internal/streamingcoord/server/service/discover/discover_server.go @@ -10,7 +10,7 @@ import ( "github.com/milvus-io/milvus/internal/streamingcoord/server/balancer" "github.com/milvus-io/milvus/internal/streamingcoord/server/resource" "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/types" ) diff --git a/internal/streamingcoord/server/service/discover/discover_server_test.go b/internal/streamingcoord/server/service/discover/discover_server_test.go index a308028b55282..c4c4a94dde711 100644 --- a/internal/streamingcoord/server/service/discover/discover_server_test.go +++ b/internal/streamingcoord/server/service/discover/discover_server_test.go @@ -10,7 +10,7 @@ import ( "github.com/milvus-io/milvus/internal/mocks/streamingcoord/server/mock_balancer" "github.com/milvus-io/milvus/internal/streamingcoord/server/resource" "github.com/milvus-io/milvus/pkg/mocks/streaming/proto/mock_streamingpb" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/streamingnode/client/handler/consumer/consumer_impl.go b/internal/streamingnode/client/handler/consumer/consumer_impl.go index b880f7064a8d8..fdd333d2364ea 100644 --- a/internal/streamingnode/client/handler/consumer/consumer_impl.go +++ b/internal/streamingnode/client/handler/consumer/consumer_impl.go @@ -12,7 +12,7 @@ import ( "github.com/milvus-io/milvus/internal/util/streamingutil/service/contextutil" "github.com/milvus-io/milvus/internal/util/streamingutil/status" "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/options" "github.com/milvus-io/milvus/pkg/streaming/util/types" diff --git a/internal/streamingnode/client/handler/consumer/consumer_test.go b/internal/streamingnode/client/handler/consumer/consumer_test.go index 8481beea30260..86ce942d74120 100644 --- a/internal/streamingnode/client/handler/consumer/consumer_test.go +++ b/internal/streamingnode/client/handler/consumer/consumer_test.go @@ -11,8 +11,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/pkg/mocks/streaming/proto/mock_streamingpb" - "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/messagespb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/options" "github.com/milvus-io/milvus/pkg/streaming/util/types" diff --git a/internal/streamingnode/client/handler/handler_client.go b/internal/streamingnode/client/handler/handler_client.go index f386be11e2cb2..c04912f2aa1c5 100644 --- a/internal/streamingnode/client/handler/handler_client.go +++ b/internal/streamingnode/client/handler/handler_client.go @@ -17,7 +17,7 @@ import ( streamingserviceinterceptor "github.com/milvus-io/milvus/internal/util/streamingutil/service/interceptor" "github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc" "github.com/milvus-io/milvus/internal/util/streamingutil/service/resolver" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/options" "github.com/milvus-io/milvus/pkg/streaming/util/types" diff --git a/internal/streamingnode/client/handler/handler_client_impl.go b/internal/streamingnode/client/handler/handler_client_impl.go index d2a52f66fad4f..e8266af323a59 100644 --- a/internal/streamingnode/client/handler/handler_client_impl.go +++ b/internal/streamingnode/client/handler/handler_client_impl.go @@ -16,7 +16,7 @@ import ( "github.com/milvus-io/milvus/internal/util/streamingutil/service/resolver" "github.com/milvus-io/milvus/internal/util/streamingutil/status" "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/streamingnode/client/handler/handler_client_test.go b/internal/streamingnode/client/handler/handler_client_test.go index 3aa571e142cfe..b6afbbe1fc2f6 100644 --- a/internal/streamingnode/client/handler/handler_client_test.go +++ b/internal/streamingnode/client/handler/handler_client_test.go @@ -18,7 +18,7 @@ import ( "github.com/milvus-io/milvus/internal/util/streamingutil/status" "github.com/milvus-io/milvus/pkg/mocks/streaming/proto/mock_streamingpb" "github.com/milvus-io/milvus/pkg/mocks/streaming/util/mock_types" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/options" "github.com/milvus-io/milvus/pkg/streaming/util/types" diff --git a/internal/streamingnode/client/handler/producer/produce_grpc_client.go b/internal/streamingnode/client/handler/producer/produce_grpc_client.go index 5b562851fcc92..1606bb5a802f4 100644 --- a/internal/streamingnode/client/handler/producer/produce_grpc_client.go +++ b/internal/streamingnode/client/handler/producer/produce_grpc_client.go @@ -1,8 +1,8 @@ package producer import ( - "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/messagespb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" ) diff --git a/internal/streamingnode/client/handler/producer/producer_impl.go b/internal/streamingnode/client/handler/producer/producer_impl.go index 54ec3224d0f02..9c84df18429c0 100644 --- a/internal/streamingnode/client/handler/producer/producer_impl.go +++ b/internal/streamingnode/client/handler/producer/producer_impl.go @@ -12,7 +12,7 @@ import ( "github.com/milvus-io/milvus/internal/util/streamingutil/service/contextutil" "github.com/milvus-io/milvus/internal/util/streamingutil/status" "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/streamingnode/client/handler/producer/producer_test.go b/internal/streamingnode/client/handler/producer/producer_test.go index bea7eda13da53..54e9fa394bfdb 100644 --- a/internal/streamingnode/client/handler/producer/producer_test.go +++ b/internal/streamingnode/client/handler/producer/producer_test.go @@ -10,8 +10,8 @@ import ( "github.com/stretchr/testify/mock" "github.com/milvus-io/milvus/pkg/mocks/streaming/proto/mock_streamingpb" - "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/messagespb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/walimplstest" diff --git a/internal/streamingnode/client/manager/manager_client.go b/internal/streamingnode/client/manager/manager_client.go index 1f26c532f3b5b..31459ef9068d4 100644 --- a/internal/streamingnode/client/manager/manager_client.go +++ b/internal/streamingnode/client/manager/manager_client.go @@ -15,7 +15,7 @@ import ( streamingserviceinterceptor "github.com/milvus-io/milvus/internal/util/streamingutil/service/interceptor" "github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc" "github.com/milvus-io/milvus/internal/util/streamingutil/service/resolver" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/tracer" "github.com/milvus-io/milvus/pkg/util/interceptor" diff --git a/internal/streamingnode/client/manager/manager_client_impl.go b/internal/streamingnode/client/manager/manager_client_impl.go index 98e352315525a..625d943661d05 100644 --- a/internal/streamingnode/client/manager/manager_client_impl.go +++ b/internal/streamingnode/client/manager/manager_client_impl.go @@ -14,7 +14,7 @@ import ( "github.com/milvus-io/milvus/internal/util/streamingutil/service/resolver" "github.com/milvus-io/milvus/internal/util/streamingutil/status" "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/streamingnode/client/manager/manager_test.go b/internal/streamingnode/client/manager/manager_test.go index 3f9a4451d89ce..fb80a4597842d 100644 --- a/internal/streamingnode/client/manager/manager_test.go +++ b/internal/streamingnode/client/manager/manager_test.go @@ -18,7 +18,7 @@ import ( "github.com/milvus-io/milvus/internal/util/streamingutil/service/contextutil" "github.com/milvus-io/milvus/internal/util/streamingutil/service/discoverer" "github.com/milvus-io/milvus/pkg/mocks/streaming/proto/mock_streamingpb" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/streamingnode/server/flusher/flusherimpl/channel_lifetime.go b/internal/streamingnode/server/flusher/flusherimpl/channel_lifetime.go index 51965267f56fb..5201b9c7653c2 100644 --- a/internal/streamingnode/server/flusher/flusherimpl/channel_lifetime.go +++ b/internal/streamingnode/server/flusher/flusherimpl/channel_lifetime.go @@ -27,12 +27,12 @@ import ( "github.com/milvus-io/milvus/internal/flushcommon/pipeline" "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/streamingnode/server/resource" "github.com/milvus-io/milvus/internal/streamingnode/server/wal" adaptor2 "github.com/milvus-io/milvus/internal/streamingnode/server/wal/adaptor" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/segment/stats" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/streaming/util/message/adaptor" "github.com/milvus-io/milvus/pkg/streaming/util/options" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/streamingnode/server/flusher/flusherimpl/flusher_impl.go b/internal/streamingnode/server/flusher/flusherimpl/flusher_impl.go index a5c417b64b212..677896997ce16 100644 --- a/internal/streamingnode/server/flusher/flusherimpl/flusher_impl.go +++ b/internal/streamingnode/server/flusher/flusherimpl/flusher_impl.go @@ -29,13 +29,13 @@ import ( "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" "github.com/milvus-io/milvus/internal/flushcommon/util" "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/streamingnode/server/flusher" "github.com/milvus-io/milvus/internal/streamingnode/server/resource" "github.com/milvus-io/milvus/internal/streamingnode/server/wal" "github.com/milvus-io/milvus/internal/util/idalloc" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/streamingnode/server/flusher/flusherimpl/flusher_impl_test.go b/internal/streamingnode/server/flusher/flusherimpl/flusher_impl_test.go index f4f0116231962..05f3a2f704f32 100644 --- a/internal/streamingnode/server/flusher/flusherimpl/flusher_impl_test.go +++ b/internal/streamingnode/server/flusher/flusherimpl/flusher_impl_test.go @@ -32,13 +32,13 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/mocks" "github.com/milvus-io/milvus/internal/mocks/streamingnode/server/mock_wal" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/streamingnode/server/flusher" "github.com/milvus-io/milvus/internal/streamingnode/server/resource" "github.com/milvus-io/milvus/internal/streamingnode/server/wal" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/syncutil" diff --git a/internal/streamingnode/server/server.go b/internal/streamingnode/server/server.go index 8956d8d78eaac..a5ce256e9ca43 100644 --- a/internal/streamingnode/server/server.go +++ b/internal/streamingnode/server/server.go @@ -10,7 +10,7 @@ import ( "github.com/milvus-io/milvus/internal/streamingnode/server/walmanager" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" _ "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/kafka" _ "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/pulsar" _ "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/rmq" diff --git a/internal/streamingnode/server/service/handler.go b/internal/streamingnode/server/service/handler.go index d9bed7e20d536..c67e3c6023476 100644 --- a/internal/streamingnode/server/service/handler.go +++ b/internal/streamingnode/server/service/handler.go @@ -4,7 +4,7 @@ import ( "github.com/milvus-io/milvus/internal/streamingnode/server/service/handler/consumer" "github.com/milvus-io/milvus/internal/streamingnode/server/service/handler/producer" "github.com/milvus-io/milvus/internal/streamingnode/server/walmanager" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) var _ HandlerService = (*handlerServiceImpl)(nil) diff --git a/internal/streamingnode/server/service/handler/consumer/consume_grpc_server_helper.go b/internal/streamingnode/server/service/handler/consumer/consume_grpc_server_helper.go index ea6ee649a3fbd..c55a449d6a87d 100644 --- a/internal/streamingnode/server/service/handler/consumer/consume_grpc_server_helper.go +++ b/internal/streamingnode/server/service/handler/consumer/consume_grpc_server_helper.go @@ -1,6 +1,6 @@ package consumer -import "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" +import "github.com/milvus-io/milvus/pkg/proto/streamingpb" // consumeGrpcServerHelper is a wrapped consumer server of log messages. type consumeGrpcServerHelper struct { diff --git a/internal/streamingnode/server/service/handler/consumer/consume_server.go b/internal/streamingnode/server/service/handler/consumer/consume_server.go index 45cf3db8c7b3e..1d6d9cab3d49f 100644 --- a/internal/streamingnode/server/service/handler/consumer/consume_server.go +++ b/internal/streamingnode/server/service/handler/consumer/consume_server.go @@ -12,8 +12,8 @@ import ( "github.com/milvus-io/milvus/internal/util/streamingutil/service/contextutil" "github.com/milvus-io/milvus/internal/util/streamingutil/status" "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/messagespb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/types" ) diff --git a/internal/streamingnode/server/service/handler/consumer/consume_server_test.go b/internal/streamingnode/server/service/handler/consumer/consume_server_test.go index 1b4ae840714b6..e61b66dcb8a4c 100644 --- a/internal/streamingnode/server/service/handler/consumer/consume_server_test.go +++ b/internal/streamingnode/server/service/handler/consumer/consume_server_test.go @@ -19,7 +19,7 @@ import ( "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/mocks/streaming/proto/mock_streamingpb" "github.com/milvus-io/milvus/pkg/mocks/streaming/util/mock_message" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/walimplstest" diff --git a/internal/streamingnode/server/service/handler/producer/produce_grpc_server_helper.go b/internal/streamingnode/server/service/handler/producer/produce_grpc_server_helper.go index cfdec40193296..4f78e9ebef5dc 100644 --- a/internal/streamingnode/server/service/handler/producer/produce_grpc_server_helper.go +++ b/internal/streamingnode/server/service/handler/producer/produce_grpc_server_helper.go @@ -1,7 +1,7 @@ package producer import ( - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) // produceGrpcServerHelper is a wrapped producer server of log messages. diff --git a/internal/streamingnode/server/service/handler/producer/produce_server.go b/internal/streamingnode/server/service/handler/producer/produce_server.go index 4075efe80882f..a14a5778b268f 100644 --- a/internal/streamingnode/server/service/handler/producer/produce_server.go +++ b/internal/streamingnode/server/service/handler/producer/produce_server.go @@ -13,7 +13,7 @@ import ( "github.com/milvus-io/milvus/internal/util/streamingutil/service/contextutil" "github.com/milvus-io/milvus/internal/util/streamingutil/status" "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/types" ) diff --git a/internal/streamingnode/server/service/handler/producer/produce_server_test.go b/internal/streamingnode/server/service/handler/producer/produce_server_test.go index 9603bc6ad1fb1..d517de56f7a52 100644 --- a/internal/streamingnode/server/service/handler/producer/produce_server_test.go +++ b/internal/streamingnode/server/service/handler/producer/produce_server_test.go @@ -21,8 +21,8 @@ import ( "github.com/milvus-io/milvus/internal/util/streamingutil/service/contextutil" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/mocks/streaming/proto/mock_streamingpb" - "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/messagespb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/walimplstest" diff --git a/internal/streamingnode/server/service/manager.go b/internal/streamingnode/server/service/manager.go index a439911171f00..4942a0cb844e7 100644 --- a/internal/streamingnode/server/service/manager.go +++ b/internal/streamingnode/server/service/manager.go @@ -4,7 +4,7 @@ import ( "context" "github.com/milvus-io/milvus/internal/streamingnode/server/walmanager" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/types" ) diff --git a/internal/streamingnode/server/wal/adaptor/wal_adaptor_test.go b/internal/streamingnode/server/wal/adaptor/wal_adaptor_test.go index 85c3365af4543..cdc82156881d8 100644 --- a/internal/streamingnode/server/wal/adaptor/wal_adaptor_test.go +++ b/internal/streamingnode/server/wal/adaptor/wal_adaptor_test.go @@ -19,7 +19,7 @@ import ( "github.com/milvus-io/milvus/internal/util/streamingutil/status" "github.com/milvus-io/milvus/pkg/mocks/streaming/mock_walimpls" "github.com/milvus-io/milvus/pkg/mocks/streaming/util/mock_message" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/streaming/walimpls" diff --git a/internal/streamingnode/server/wal/adaptor/wal_test.go b/internal/streamingnode/server/wal/adaptor/wal_test.go index 8a222b04b35c0..3b49bf1933882 100644 --- a/internal/streamingnode/server/wal/adaptor/wal_test.go +++ b/internal/streamingnode/server/wal/adaptor/wal_test.go @@ -18,13 +18,13 @@ import ( "github.com/milvus-io/milvus/internal/mocks" "github.com/milvus-io/milvus/internal/mocks/mock_metastore" "github.com/milvus-io/milvus/internal/mocks/streamingnode/server/mock_flusher" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/streamingnode/server/resource" "github.com/milvus-io/milvus/internal/streamingnode/server/wal" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/registry" internaltypes "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/idalloc" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/options" "github.com/milvus-io/milvus/pkg/streaming/util/types" diff --git a/internal/streamingnode/server/wal/interceptors/segment/manager/partition_manager.go b/internal/streamingnode/server/wal/interceptors/segment/manager/partition_manager.go index 59be40f37c39a..d815d645bad33 100644 --- a/internal/streamingnode/server/wal/interceptors/segment/manager/partition_manager.go +++ b/internal/streamingnode/server/wal/interceptors/segment/manager/partition_manager.go @@ -8,14 +8,14 @@ import ( "github.com/samber/lo" "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/streamingnode/server/resource" "github.com/milvus-io/milvus/internal/streamingnode/server/wal" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/segment/policy" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/metricsutil" "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/messagespb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/streamingnode/server/wal/interceptors/segment/manager/partition_managers.go b/internal/streamingnode/server/wal/interceptors/segment/manager/partition_managers.go index 02ec9996e58e1..2673853bf8f42 100644 --- a/internal/streamingnode/server/wal/interceptors/segment/manager/partition_managers.go +++ b/internal/streamingnode/server/wal/interceptors/segment/manager/partition_managers.go @@ -6,14 +6,14 @@ import ( "github.com/cockroachdb/errors" "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/streamingnode/server/resource" "github.com/milvus-io/milvus/internal/streamingnode/server/wal" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/segment/policy" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/metricsutil" "github.com/milvus-io/milvus/internal/util/streamingutil/status" "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/syncutil" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/streamingnode/server/wal/interceptors/segment/manager/pchannel_manager.go b/internal/streamingnode/server/wal/interceptors/segment/manager/pchannel_manager.go index e942ffae35c55..d2393d728182a 100644 --- a/internal/streamingnode/server/wal/interceptors/segment/manager/pchannel_manager.go +++ b/internal/streamingnode/server/wal/interceptors/segment/manager/pchannel_manager.go @@ -6,14 +6,14 @@ import ( "github.com/cockroachdb/errors" "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/streamingnode/server/resource" "github.com/milvus-io/milvus/internal/streamingnode/server/wal" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/segment/inspector" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/segment/stats" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/metricsutil" "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/syncutil" diff --git a/internal/streamingnode/server/wal/interceptors/segment/manager/pchannel_manager_test.go b/internal/streamingnode/server/wal/interceptors/segment/manager/pchannel_manager_test.go index 4497551c2bc58..6a4f7a11ec607 100644 --- a/internal/streamingnode/server/wal/interceptors/segment/manager/pchannel_manager_test.go +++ b/internal/streamingnode/server/wal/interceptors/segment/manager/pchannel_manager_test.go @@ -12,8 +12,6 @@ import ( "github.com/milvus-io/milvus/internal/mocks" "github.com/milvus-io/milvus/internal/mocks/mock_metastore" "github.com/milvus-io/milvus/internal/mocks/streamingnode/server/mock_wal" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/streamingnode/server/resource" "github.com/milvus-io/milvus/internal/streamingnode/server/wal" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/segment/inspector" @@ -21,7 +19,9 @@ import ( "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/txn" internaltypes "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/idalloc" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/rmq" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/internal/streamingnode/server/wal/interceptors/segment/manager/seal_queue.go b/internal/streamingnode/server/wal/interceptors/segment/manager/seal_queue.go index d80422324fa1f..e3cb540d4fa9a 100644 --- a/internal/streamingnode/server/wal/interceptors/segment/manager/seal_queue.go +++ b/internal/streamingnode/server/wal/interceptors/segment/manager/seal_queue.go @@ -10,7 +10,7 @@ import ( "github.com/milvus-io/milvus/internal/streamingnode/server/wal" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/metricsutil" "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/util/syncutil" ) diff --git a/internal/streamingnode/server/wal/interceptors/segment/manager/segment_manager.go b/internal/streamingnode/server/wal/interceptors/segment/manager/segment_manager.go index 796dbe8034101..986cb44b94ee5 100644 --- a/internal/streamingnode/server/wal/interceptors/segment/manager/segment_manager.go +++ b/internal/streamingnode/server/wal/interceptors/segment/manager/segment_manager.go @@ -14,7 +14,7 @@ import ( "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/segment/stats" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/metricsutil" "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/types" ) diff --git a/internal/streamingnode/server/wal/interceptors/segment/segment_assign_interceptor.go b/internal/streamingnode/server/wal/interceptors/segment/segment_assign_interceptor.go index 3975791e86fa9..72849cf5920cc 100644 --- a/internal/streamingnode/server/wal/interceptors/segment/segment_assign_interceptor.go +++ b/internal/streamingnode/server/wal/interceptors/segment/segment_assign_interceptor.go @@ -16,7 +16,7 @@ import ( "github.com/milvus-io/milvus/internal/streamingnode/server/wal/utility" "github.com/milvus-io/milvus/internal/util/streamingutil/status" "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb" + "github.com/milvus-io/milvus/pkg/proto/messagespb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/util/syncutil" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/streamingnode/server/wal/interceptors/segment/stats/stats.go b/internal/streamingnode/server/wal/interceptors/segment/stats/stats.go index 11a57fc1c0868..1a210e6f10570 100644 --- a/internal/streamingnode/server/wal/interceptors/segment/stats/stats.go +++ b/internal/streamingnode/server/wal/interceptors/segment/stats/stats.go @@ -3,7 +3,7 @@ package stats import ( "time" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) // SegmentStats is the usage stats of a segment. diff --git a/internal/streamingnode/server/wal/interceptors/timetick/ack/ack_test.go b/internal/streamingnode/server/wal/interceptors/timetick/ack/ack_test.go index 0ba11fd88b499..86ce5eed5385c 100644 --- a/internal/streamingnode/server/wal/interceptors/timetick/ack/ack_test.go +++ b/internal/streamingnode/server/wal/interceptors/timetick/ack/ack_test.go @@ -14,10 +14,10 @@ import ( "google.golang.org/grpc" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/streamingnode/server/resource" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/metricsutil" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/walimplstest" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/streamingnode/server/wal/interceptors/txn/session_test.go b/internal/streamingnode/server/wal/interceptors/txn/session_test.go index e8331240ae4d9..5a973a5c406b0 100644 --- a/internal/streamingnode/server/wal/interceptors/txn/session_test.go +++ b/internal/streamingnode/server/wal/interceptors/txn/session_test.go @@ -11,7 +11,7 @@ import ( "github.com/milvus-io/milvus/internal/streamingnode/server/resource" "github.com/milvus-io/milvus/internal/util/streamingutil/status" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/streamingnode/server/wal/metricsutil/segment.go b/internal/streamingnode/server/wal/metricsutil/segment.go index f0d429f1a1d5b..de00973867a5f 100644 --- a/internal/streamingnode/server/wal/metricsutil/segment.go +++ b/internal/streamingnode/server/wal/metricsutil/segment.go @@ -4,7 +4,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/milvus-io/milvus/pkg/metrics" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/streamingnode/server/walmanager/manager_impl_test.go b/internal/streamingnode/server/walmanager/manager_impl_test.go index cdaa931e3c51d..1c709b81c05ad 100644 --- a/internal/streamingnode/server/walmanager/manager_impl_test.go +++ b/internal/streamingnode/server/walmanager/manager_impl_test.go @@ -14,7 +14,7 @@ import ( "github.com/milvus-io/milvus/internal/streamingnode/server/wal" internaltypes "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/streamingutil/status" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/syncutil" diff --git a/internal/streamingnode/server/walmanager/wal_state_pair_test.go b/internal/streamingnode/server/walmanager/wal_state_pair_test.go index 347d15b5fe938..e2a0d3cdd0523 100644 --- a/internal/streamingnode/server/walmanager/wal_state_pair_test.go +++ b/internal/streamingnode/server/walmanager/wal_state_pair_test.go @@ -9,7 +9,7 @@ import ( "github.com/milvus-io/milvus/internal/mocks/streamingnode/server/mock_wal" "github.com/milvus-io/milvus/internal/util/streamingutil/status" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/types" ) diff --git a/internal/types/types.go b/internal/types/types.go index fb48de650e3ec..2d52a8e6dcbf7 100644 --- a/internal/types/types.go +++ b/internal/types/types.go @@ -26,13 +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/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/proxypb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" - "github.com/milvus-io/milvus/internal/proto/workerpb" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/proxypb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" ) // Limiter defines the interface to perform request rate limiting. diff --git a/internal/util/analyzecgowrapper/analyze.go b/internal/util/analyzecgowrapper/analyze.go index ad57fc3f718cb..0d106a0830298 100644 --- a/internal/util/analyzecgowrapper/analyze.go +++ b/internal/util/analyzecgowrapper/analyze.go @@ -32,8 +32,8 @@ import ( "go.uber.org/zap" "google.golang.org/protobuf/proto" - "github.com/milvus-io/milvus/internal/proto/clusteringpb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/clusteringpb" ) type CodecAnalyze interface { diff --git a/internal/util/componentutil/componentutil_test.go b/internal/util/componentutil/componentutil_test.go index 107dc7258ab04..7f6a5b2734f5c 100644 --- a/internal/util/componentutil/componentutil_test.go +++ b/internal/util/componentutil/componentutil_test.go @@ -27,7 +27,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" - "github.com/milvus-io/milvus/internal/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/funcutil" ) diff --git a/internal/util/exprutil/expr_checker.go b/internal/util/exprutil/expr_checker.go index 6744138968860..0f97de44ccf9a 100644 --- a/internal/util/exprutil/expr_checker.go +++ b/internal/util/exprutil/expr_checker.go @@ -7,7 +7,7 @@ import ( "github.com/cockroachdb/errors" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" ) type KeyType int64 diff --git a/internal/util/exprutil/expr_checker_test.go b/internal/util/exprutil/expr_checker_test.go index 4f500de954e0c..ff5a3a0f5b501 100644 --- a/internal/util/exprutil/expr_checker_test.go +++ b/internal/util/exprutil/expr_checker_test.go @@ -9,9 +9,9 @@ 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/parser/planparserv2" - "github.com/milvus-io/milvus/internal/proto/planpb" "github.com/milvus-io/milvus/internal/util/testutil" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/planpb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/util/funcutil/count_util.go b/internal/util/funcutil/count_util.go index f00b3c430d896..ae2b370aa123a 100644 --- a/internal/util/funcutil/count_util.go +++ b/internal/util/funcutil/count_util.go @@ -6,8 +6,8 @@ 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/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" ) func CntOfInternalResult(res *internalpb.RetrieveResults) (int64, error) { diff --git a/internal/util/funcutil/count_util_test.go b/internal/util/funcutil/count_util_test.go index db4c430d72b5d..062d3f21aa77e 100644 --- a/internal/util/funcutil/count_util_test.go +++ b/internal/util/funcutil/count_util_test.go @@ -6,8 +6,8 @@ import ( "github.com/stretchr/testify/assert" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" ) func TestCntOfInternalResult(t *testing.T) { diff --git a/internal/util/grpcclient/client_test.go b/internal/util/grpcclient/client_test.go index 37a0fd4318f96..b89b7459d5466 100644 --- a/internal/util/grpcclient/client_test.go +++ b/internal/util/grpcclient/client_test.go @@ -38,9 +38,9 @@ import ( "google.golang.org/grpc/status" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/streamrpc" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/util/grpcclient/local_grpc_client_test.go b/internal/util/grpcclient/local_grpc_client_test.go index bcd59e62a9f65..8500b2508ed24 100644 --- a/internal/util/grpcclient/local_grpc_client_test.go +++ b/internal/util/grpcclient/local_grpc_client_test.go @@ -6,7 +6,7 @@ import ( "github.com/stretchr/testify/assert" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" ) type mockRootCoordServer struct { diff --git a/internal/util/idalloc/allocator_test.go b/internal/util/idalloc/allocator_test.go index 305c917c5be74..46dd366ed71ed 100644 --- a/internal/util/idalloc/allocator_test.go +++ b/internal/util/idalloc/allocator_test.go @@ -10,8 +10,8 @@ import ( "google.golang.org/grpc" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/syncutil" ) diff --git a/internal/util/idalloc/basic_allocator.go b/internal/util/idalloc/basic_allocator.go index ad49c5192a03b..4b768f323c9e5 100644 --- a/internal/util/idalloc/basic_allocator.go +++ b/internal/util/idalloc/basic_allocator.go @@ -8,8 +8,8 @@ import ( "github.com/cockroachdb/errors" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/syncutil" diff --git a/internal/util/idalloc/basic_allocator_test.go b/internal/util/idalloc/basic_allocator_test.go index 549f78cc00d8b..9f11a8b64e13d 100644 --- a/internal/util/idalloc/basic_allocator_test.go +++ b/internal/util/idalloc/basic_allocator_test.go @@ -12,8 +12,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/syncutil" ) diff --git a/internal/util/idalloc/test_mock_root_coord_client.go b/internal/util/idalloc/test_mock_root_coord_client.go index 2d3ed8bec65cd..19a8c42402daa 100644 --- a/internal/util/idalloc/test_mock_root_coord_client.go +++ b/internal/util/idalloc/test_mock_root_coord_client.go @@ -14,7 +14,7 @@ import ( "google.golang.org/grpc" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/tsoutil" ) diff --git a/internal/util/importutilv2/binlog/l0_reader.go b/internal/util/importutilv2/binlog/l0_reader.go index 15874ce6cad9b..eb794548f3026 100644 --- a/internal/util/importutilv2/binlog/l0_reader.go +++ b/internal/util/importutilv2/binlog/l0_reader.go @@ -24,9 +24,9 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/util/importutilv2/binlog/l0_reader_test.go b/internal/util/importutilv2/binlog/l0_reader_test.go index dbbd63dd4fa34..eeae66b05b04c 100644 --- a/internal/util/importutilv2/binlog/l0_reader_test.go +++ b/internal/util/importutilv2/binlog/l0_reader_test.go @@ -27,8 +27,8 @@ import ( "github.com/stretchr/testify/mock" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/pkg/proto/internalpb" ) func TestL0Reader_NewL0Reader(t *testing.T) { diff --git a/internal/util/importutilv2/reader.go b/internal/util/importutilv2/reader.go index 971158a833a35..8324250ba2ac6 100644 --- a/internal/util/importutilv2/reader.go +++ b/internal/util/importutilv2/reader.go @@ -20,13 +20,13 @@ import ( "context" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/importutilv2/binlog" "github.com/milvus-io/milvus/internal/util/importutilv2/csv" "github.com/milvus-io/milvus/internal/util/importutilv2/json" "github.com/milvus-io/milvus/internal/util/importutilv2/numpy" "github.com/milvus-io/milvus/internal/util/importutilv2/parquet" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/util/importutilv2/util.go b/internal/util/importutilv2/util.go index b187a541f7d7d..ae7bdf6693e8c 100644 --- a/internal/util/importutilv2/util.go +++ b/internal/util/importutilv2/util.go @@ -22,7 +22,7 @@ import ( "github.com/samber/lo" - "github.com/milvus-io/milvus/internal/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/util/indexcgowrapper/build_index_info.go b/internal/util/indexcgowrapper/build_index_info.go index 755a7b93d2ff1..41838647d04e2 100644 --- a/internal/util/indexcgowrapper/build_index_info.go +++ b/internal/util/indexcgowrapper/build_index_info.go @@ -31,8 +31,8 @@ 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/proto/indexcgopb" - "github.com/milvus-io/milvus/internal/proto/indexpb" + "github.com/milvus-io/milvus/pkg/proto/indexcgopb" + "github.com/milvus-io/milvus/pkg/proto/indexpb" ) type BuildIndexInfo struct { diff --git a/internal/util/indexcgowrapper/codec_index_test.go b/internal/util/indexcgowrapper/codec_index_test.go index b9398ac8f6151..3445115363071 100644 --- a/internal/util/indexcgowrapper/codec_index_test.go +++ b/internal/util/indexcgowrapper/codec_index_test.go @@ -10,9 +10,9 @@ import ( "github.com/stretchr/testify/assert" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/indexpb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/metric" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/util/indexcgowrapper/index.go b/internal/util/indexcgowrapper/index.go index c87a3801feeae..d126ab79bd72b 100644 --- a/internal/util/indexcgowrapper/index.go +++ b/internal/util/indexcgowrapper/index.go @@ -21,9 +21,9 @@ 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/proto/indexcgopb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/indexcgopb" ) type Blob = storage.Blob diff --git a/internal/util/indexparamcheck/vector_index_checker.go b/internal/util/indexparamcheck/vector_index_checker.go index 4d25e8a63fe50..877c15e558de8 100644 --- a/internal/util/indexparamcheck/vector_index_checker.go +++ b/internal/util/indexparamcheck/vector_index_checker.go @@ -17,9 +17,9 @@ 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/proto/indexcgopb" "github.com/milvus-io/milvus/internal/util/vecindexmgr" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/indexcgopb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/util/metrics/utils.go b/internal/util/metrics/utils.go index e3493773ce1d8..d4a1fb1e8e21c 100644 --- a/internal/util/metrics/utils.go +++ b/internal/util/metrics/utils.go @@ -5,7 +5,7 @@ import ( "github.com/samber/lo" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/metricsinfo" ) diff --git a/internal/util/mock/grpc_datanode_client.go b/internal/util/mock/grpc_datanode_client.go index 13ae355738d80..9aa4573ba4a87 100644 --- a/internal/util/mock/grpc_datanode_client.go +++ b/internal/util/mock/grpc_datanode_client.go @@ -23,8 +23,8 @@ 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/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" ) var _ datapb.DataNodeClient = &GrpcDataNodeClient{} diff --git a/internal/util/mock/grpc_querycoord_client.go b/internal/util/mock/grpc_querycoord_client.go index eb1932c8df0b0..de245b19334f0 100644 --- a/internal/util/mock/grpc_querycoord_client.go +++ b/internal/util/mock/grpc_querycoord_client.go @@ -23,8 +23,8 @@ 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/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" ) // Check if GrpcQueryCoordClient implements proto.GrpcQueryCoordClient diff --git a/internal/util/mock/grpc_querynode_client.go b/internal/util/mock/grpc_querynode_client.go index dadfb3157897d..1a2e2e0a7e2d4 100644 --- a/internal/util/mock/grpc_querynode_client.go +++ b/internal/util/mock/grpc_querynode_client.go @@ -23,9 +23,9 @@ 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/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/util/streamrpc" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" ) var _ querypb.QueryNodeClient = &GrpcQueryNodeClient{} diff --git a/internal/util/mock/grpc_rootcoord_client.go b/internal/util/mock/grpc_rootcoord_client.go index abf856d0be9c0..b19e9ebc948cd 100644 --- a/internal/util/mock/grpc_rootcoord_client.go +++ b/internal/util/mock/grpc_rootcoord_client.go @@ -23,10 +23,10 @@ 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/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/proxypb" - "github.com/milvus-io/milvus/internal/proto/rootcoordpb" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/proxypb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/uniquegenerator" ) diff --git a/internal/util/proxyutil/mock_proxy_client_manager.go b/internal/util/proxyutil/mock_proxy_client_manager.go index 9b6e2be16d88a..e2cf971e27db8 100644 --- a/internal/util/proxyutil/mock_proxy_client_manager.go +++ b/internal/util/proxyutil/mock_proxy_client_manager.go @@ -8,7 +8,7 @@ import ( milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" mock "github.com/stretchr/testify/mock" - proxypb "github.com/milvus-io/milvus/internal/proto/proxypb" + proxypb "github.com/milvus-io/milvus/pkg/proto/proxypb" sessionutil "github.com/milvus-io/milvus/internal/util/sessionutil" diff --git a/internal/util/proxyutil/proxy_client_manager.go b/internal/util/proxyutil/proxy_client_manager.go index 76018dda5b2d4..f5a98b9cd8bea 100644 --- a/internal/util/proxyutil/proxy_client_manager.go +++ b/internal/util/proxyutil/proxy_client_manager.go @@ -28,11 +28,11 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" grpcproxyclient "github.com/milvus-io/milvus/internal/distributed/proxy/client" - "github.com/milvus-io/milvus/internal/proto/proxypb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/proxypb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" diff --git a/internal/util/proxyutil/proxy_client_manager_test.go b/internal/util/proxyutil/proxy_client_manager_test.go index 1dde818704cfa..03ba4e8a5ee4d 100644 --- a/internal/util/proxyutil/proxy_client_manager_test.go +++ b/internal/util/proxyutil/proxy_client_manager_test.go @@ -29,9 +29,9 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/proxypb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/sessionutil" + "github.com/milvus-io/milvus/pkg/proto/proxypb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/util/quota/quota_constant.go b/internal/util/quota/quota_constant.go index 7f36800bfe8b2..94b1a31610cfb 100644 --- a/internal/util/quota/quota_constant.go +++ b/internal/util/quota/quota_constant.go @@ -24,10 +24,10 @@ import ( "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/pkg/config" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/util/quota/quota_constant_test.go b/internal/util/quota/quota_constant_test.go index f8476bdf11e5c..4179c519416e4 100644 --- a/internal/util/quota/quota_constant_test.go +++ b/internal/util/quota/quota_constant_test.go @@ -24,7 +24,7 @@ import ( "github.com/stretchr/testify/assert" - "github.com/milvus-io/milvus/internal/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/util/ratelimitutil/rate_limiter_tree.go b/internal/util/ratelimitutil/rate_limiter_tree.go index 083b42ae392b9..5ae9c0b9db23c 100644 --- a/internal/util/ratelimitutil/rate_limiter_tree.go +++ b/internal/util/ratelimitutil/rate_limiter_tree.go @@ -23,8 +23,8 @@ 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/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/proxypb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/proxypb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/ratelimitutil" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/util/ratelimitutil/rate_limiter_tree_test.go b/internal/util/ratelimitutil/rate_limiter_tree_test.go index 383cf07a81906..ab56ce8a6e9be 100644 --- a/internal/util/ratelimitutil/rate_limiter_tree_test.go +++ b/internal/util/ratelimitutil/rate_limiter_tree_test.go @@ -26,8 +26,8 @@ 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/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/proxypb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/proxypb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/ratelimitutil" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/util/searchutil/optimizers/query_hook.go b/internal/util/searchutil/optimizers/query_hook.go index 7ed96cabb38a0..0bed21940acba 100644 --- a/internal/util/searchutil/optimizers/query_hook.go +++ b/internal/util/searchutil/optimizers/query_hook.go @@ -7,11 +7,11 @@ import ( "go.uber.org/zap" "google.golang.org/protobuf/proto" - "github.com/milvus-io/milvus/internal/proto/planpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/util/searchutil/optimizers/query_hook_test.go b/internal/util/searchutil/optimizers/query_hook_test.go index a0abd38f7dff0..1f405b9bb2834 100644 --- a/internal/util/searchutil/optimizers/query_hook_test.go +++ b/internal/util/searchutil/optimizers/query_hook_test.go @@ -9,10 +9,10 @@ import ( "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus/internal/mocks/util/searchutil/mock_optimizers" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/planpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/util/searchutil/scheduler/mock_task_test.go b/internal/util/searchutil/scheduler/mock_task_test.go index 9f334cb2426c9..eae4d2f7ed3fc 100644 --- a/internal/util/searchutil/scheduler/mock_task_test.go +++ b/internal/util/searchutil/scheduler/mock_task_test.go @@ -5,7 +5,7 @@ import ( "math/rand" "time" - "github.com/milvus-io/milvus/internal/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/timerecord" ) diff --git a/internal/util/searchutil/scheduler/tasks.go b/internal/util/searchutil/scheduler/tasks.go index cadb78472965e..f221eb518b749 100644 --- a/internal/util/searchutil/scheduler/tasks.go +++ b/internal/util/searchutil/scheduler/tasks.go @@ -1,6 +1,6 @@ package scheduler -import "github.com/milvus-io/milvus/internal/proto/internalpb" +import "github.com/milvus-io/milvus/pkg/proto/internalpb" const ( schedulePolicyNameFIFO = "fifo" diff --git a/internal/util/segcore/collection.go b/internal/util/segcore/collection.go index 23835a377fb1a..100e28e537f90 100644 --- a/internal/util/segcore/collection.go +++ b/internal/util/segcore/collection.go @@ -15,7 +15,7 @@ import ( "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" ) // CreateCCollectionRequest is a request to create a CCollection. diff --git a/internal/util/segcore/plan.go b/internal/util/segcore/plan.go index 54779caf58213..516807aad7668 100644 --- a/internal/util/segcore/plan.go +++ b/internal/util/segcore/plan.go @@ -31,7 +31,7 @@ import ( "github.com/cockroachdb/errors" - "github.com/milvus-io/milvus/internal/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/util/segcore/plan_test.go b/internal/util/segcore/plan_test.go index 4fad9c83ae3df..01893348e8ad0 100644 --- a/internal/util/segcore/plan_test.go +++ b/internal/util/segcore/plan_test.go @@ -24,10 +24,10 @@ import ( "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/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/planpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/util/segcore" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/paramtable" ) diff --git a/internal/util/segcore/reduce_test.go b/internal/util/segcore/reduce_test.go index 8d7e21e6c2358..64eb33b63cc26 100644 --- a/internal/util/segcore/reduce_test.go +++ b/internal/util/segcore/reduce_test.go @@ -31,13 +31,13 @@ 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/mocks/util/mock_segcore" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/planpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/initcore" "github.com/milvus-io/milvus/internal/util/segcore" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/testutils" diff --git a/internal/util/segcore/requests.go b/internal/util/segcore/requests.go index 1800c0fcc5b4f..5a1a49d490a62 100644 --- a/internal/util/segcore/requests.go +++ b/internal/util/segcore/requests.go @@ -11,9 +11,9 @@ import ( "github.com/cockroachdb/errors" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/util/segcore/requests_test.go b/internal/util/segcore/requests_test.go index 87f1500161bcd..94104234469de 100644 --- a/internal/util/segcore/requests_test.go +++ b/internal/util/segcore/requests_test.go @@ -5,7 +5,7 @@ import ( "github.com/stretchr/testify/assert" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" ) func TestLoadFieldDataRequest(t *testing.T) { diff --git a/internal/util/segcore/responses.go b/internal/util/segcore/responses.go index f2405cbcf2569..ffafac08e7f25 100644 --- a/internal/util/segcore/responses.go +++ b/internal/util/segcore/responses.go @@ -9,7 +9,7 @@ package segcore import "C" import ( - "github.com/milvus-io/milvus/internal/proto/segcorepb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" ) type SearchResult struct { diff --git a/internal/util/segcore/segment_test.go b/internal/util/segcore/segment_test.go index bca8c579f90a3..fd82fd728f18f 100644 --- a/internal/util/segcore/segment_test.go +++ b/internal/util/segcore/segment_test.go @@ -10,11 +10,11 @@ import ( "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/internal/proto/planpb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/initcore" "github.com/milvus-io/milvus/internal/util/segcore" + "github.com/milvus-io/milvus/pkg/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/util/segmentutil/utils.go b/internal/util/segmentutil/utils.go index 3f183ef32b067..3e883cce8cc6d 100644 --- a/internal/util/segmentutil/utils.go +++ b/internal/util/segmentutil/utils.go @@ -3,8 +3,8 @@ package segmentutil import ( "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" ) // ReCalcRowCount re-calculates number of rows of `oldSeg` based on its bin log count, and correct its value in its diff --git a/internal/util/streamingutil/service/contextutil/create_consumer.go b/internal/util/streamingutil/service/contextutil/create_consumer.go index dad56718035ff..ffe61b3a6ad51 100644 --- a/internal/util/streamingutil/service/contextutil/create_consumer.go +++ b/internal/util/streamingutil/service/contextutil/create_consumer.go @@ -9,7 +9,7 @@ import ( "google.golang.org/grpc/metadata" "google.golang.org/protobuf/proto" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) const ( diff --git a/internal/util/streamingutil/service/contextutil/create_consumer_test.go b/internal/util/streamingutil/service/contextutil/create_consumer_test.go index f1410e526c0f0..c9ae1218c7cde 100644 --- a/internal/util/streamingutil/service/contextutil/create_consumer_test.go +++ b/internal/util/streamingutil/service/contextutil/create_consumer_test.go @@ -8,7 +8,7 @@ import ( "github.com/stretchr/testify/assert" "google.golang.org/grpc/metadata" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) func TestWithCreateConsumer(t *testing.T) { diff --git a/internal/util/streamingutil/service/contextutil/create_producer.go b/internal/util/streamingutil/service/contextutil/create_producer.go index 53770c81a9321..599553e61a7a3 100644 --- a/internal/util/streamingutil/service/contextutil/create_producer.go +++ b/internal/util/streamingutil/service/contextutil/create_producer.go @@ -9,7 +9,7 @@ import ( "google.golang.org/grpc/metadata" "google.golang.org/protobuf/proto" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) const ( diff --git a/internal/util/streamingutil/service/contextutil/create_producer_test.go b/internal/util/streamingutil/service/contextutil/create_producer_test.go index 1347345430dec..514143ea2bb97 100644 --- a/internal/util/streamingutil/service/contextutil/create_producer_test.go +++ b/internal/util/streamingutil/service/contextutil/create_producer_test.go @@ -8,7 +8,7 @@ import ( "github.com/stretchr/testify/assert" "google.golang.org/grpc/metadata" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) func TestWithCreateProducer(t *testing.T) { diff --git a/internal/util/streamingutil/service/interceptor/client.go b/internal/util/streamingutil/service/interceptor/client.go index 60b5631340459..1c95aa46fcaac 100644 --- a/internal/util/streamingutil/service/interceptor/client.go +++ b/internal/util/streamingutil/service/interceptor/client.go @@ -7,7 +7,7 @@ import ( "google.golang.org/grpc" "github.com/milvus-io/milvus/internal/util/streamingutil/status" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) // NewStreamingServiceUnaryClientInterceptor returns a new unary client interceptor for error handling. diff --git a/internal/util/streamingutil/service/interceptor/server.go b/internal/util/streamingutil/service/interceptor/server.go index e9881d73d953c..cd3943e3d513f 100644 --- a/internal/util/streamingutil/service/interceptor/server.go +++ b/internal/util/streamingutil/service/interceptor/server.go @@ -7,7 +7,7 @@ import ( "google.golang.org/grpc" "github.com/milvus-io/milvus/internal/util/streamingutil/status" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) // NewStreamingServiceUnaryServerInterceptor returns a new unary server interceptor for error handling, metric... diff --git a/internal/util/streamingutil/status/client_stream_wrapper_test.go b/internal/util/streamingutil/status/client_stream_wrapper_test.go index 703e1d87e4b24..3efd8bab3638d 100644 --- a/internal/util/streamingutil/status/client_stream_wrapper_test.go +++ b/internal/util/streamingutil/status/client_stream_wrapper_test.go @@ -8,7 +8,7 @@ import ( "github.com/stretchr/testify/mock" "github.com/milvus-io/milvus/internal/mocks/google.golang.org/mock_grpc" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) func TestClientStreamWrapper(t *testing.T) { diff --git a/internal/util/streamingutil/status/rpc_error.go b/internal/util/streamingutil/status/rpc_error.go index 4a4da36131806..9fb13275da2db 100644 --- a/internal/util/streamingutil/status/rpc_error.go +++ b/internal/util/streamingutil/status/rpc_error.go @@ -9,7 +9,7 @@ import ( "google.golang.org/grpc/codes" "google.golang.org/grpc/status" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) var streamingErrorToGRPCStatus = map[streamingpb.StreamingCode]codes.Code{ diff --git a/internal/util/streamingutil/status/rpc_error_test.go b/internal/util/streamingutil/status/rpc_error_test.go index 9f64ceee358e5..05f6b2a44e38a 100644 --- a/internal/util/streamingutil/status/rpc_error_test.go +++ b/internal/util/streamingutil/status/rpc_error_test.go @@ -8,7 +8,7 @@ import ( "github.com/stretchr/testify/assert" "google.golang.org/grpc/codes" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) func TestStreamingStatus(t *testing.T) { diff --git a/internal/util/streamingutil/status/streaming_error.go b/internal/util/streamingutil/status/streaming_error.go index 8e1fc9d155c3b..08b2263e3478d 100644 --- a/internal/util/streamingutil/status/streaming_error.go +++ b/internal/util/streamingutil/status/streaming_error.go @@ -6,7 +6,7 @@ import ( "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" ) diff --git a/internal/util/streamingutil/status/streaming_error_test.go b/internal/util/streamingutil/status/streaming_error_test.go index d6bb406c0cd06..04ea16931349c 100644 --- a/internal/util/streamingutil/status/streaming_error_test.go +++ b/internal/util/streamingutil/status/streaming_error_test.go @@ -5,7 +5,7 @@ import ( "github.com/stretchr/testify/assert" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) func TestStreamingError(t *testing.T) { diff --git a/internal/util/streamrpc/streamer.go b/internal/util/streamrpc/streamer.go index a9ca407dc22f8..768eb7a82d9ba 100644 --- a/internal/util/streamrpc/streamer.go +++ b/internal/util/streamrpc/streamer.go @@ -9,7 +9,7 @@ import ( "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/internal/util/streamrpc/streamer_test.go b/internal/util/streamrpc/streamer_test.go index 8377158549b42..4deff15f73287 100644 --- a/internal/util/streamrpc/streamer_test.go +++ b/internal/util/streamrpc/streamer_test.go @@ -26,7 +26,7 @@ import ( "github.com/stretchr/testify/suite" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" ) type ResultCacheServerSuite struct { diff --git a/internal/util/typeutil/hash.go b/internal/util/typeutil/hash.go index 8815768336593..b977592ea3040 100644 --- a/internal/util/typeutil/hash.go +++ b/internal/util/typeutil/hash.go @@ -4,7 +4,7 @@ import ( "github.com/cockroachdb/errors" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/planpb" + "github.com/milvus-io/milvus/pkg/proto/planpb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/util/typeutil/result_helper_test.go b/internal/util/typeutil/result_helper_test.go index b1d3cec646ee0..3385ecad4c396 100644 --- a/internal/util/typeutil/result_helper_test.go +++ b/internal/util/typeutil/result_helper_test.go @@ -8,9 +8,9 @@ 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/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/util/typeutil/retrieve_result.go b/internal/util/typeutil/retrieve_result.go index ff77f1ca64b89..8f809284d4199 100644 --- a/internal/util/typeutil/retrieve_result.go +++ b/internal/util/typeutil/retrieve_result.go @@ -3,8 +3,8 @@ package typeutil import ( "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/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/segcorepb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/segcorepb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/internal/util/wrappers/qn_wrapper.go b/internal/util/wrappers/qn_wrapper.go index def2e64f01bef..09ac9166e9578 100644 --- a/internal/util/wrappers/qn_wrapper.go +++ b/internal/util/wrappers/qn_wrapper.go @@ -23,10 +23,10 @@ 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/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/streamrpc" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" ) type qnServerWrapper struct { diff --git a/internal/util/wrappers/qn_wrapper_test.go b/internal/util/wrappers/qn_wrapper_test.go index 9299e6305882a..505908c954306 100644 --- a/internal/util/wrappers/qn_wrapper_test.go +++ b/internal/util/wrappers/qn_wrapper_test.go @@ -25,9 +25,9 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus/internal/mocks" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" ) diff --git a/pkg/.mockery_pkg.yaml b/pkg/.mockery_pkg.yaml index b151a3b5203e7..124f721fa14da 100644 --- a/pkg/.mockery_pkg.yaml +++ b/pkg/.mockery_pkg.yaml @@ -28,7 +28,7 @@ packages: interfaces: AssignmentDiscoverWatcher: AssignmentRebalanceTrigger: - github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb: + github.com/milvus-io/milvus/pkg/proto/streamingpb: interfaces: StreamingNodeHandlerService_ConsumeServer: StreamingNodeHandlerService_ProduceServer: diff --git a/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingCoordAssignmentServiceClient.go b/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingCoordAssignmentServiceClient.go index 904bd2b08a3cc..0ade6137b72f0 100644 --- a/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingCoordAssignmentServiceClient.go +++ b/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingCoordAssignmentServiceClient.go @@ -9,7 +9,7 @@ import ( mock "github.com/stretchr/testify/mock" - streamingpb "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + streamingpb "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) // MockStreamingCoordAssignmentServiceClient is an autogenerated mock type for the StreamingCoordAssignmentServiceClient type diff --git a/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingCoordAssignmentService_AssignmentDiscoverClient.go b/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingCoordAssignmentService_AssignmentDiscoverClient.go index a9e201ab88e35..0506483c53bed 100644 --- a/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingCoordAssignmentService_AssignmentDiscoverClient.go +++ b/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingCoordAssignmentService_AssignmentDiscoverClient.go @@ -8,7 +8,7 @@ import ( mock "github.com/stretchr/testify/mock" metadata "google.golang.org/grpc/metadata" - streamingpb "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + streamingpb "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) // MockStreamingCoordAssignmentService_AssignmentDiscoverClient is an autogenerated mock type for the StreamingCoordAssignmentService_AssignmentDiscoverClient type diff --git a/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingCoordAssignmentService_AssignmentDiscoverServer.go b/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingCoordAssignmentService_AssignmentDiscoverServer.go index 2b130b409df23..89f8cd2356ab3 100644 --- a/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingCoordAssignmentService_AssignmentDiscoverServer.go +++ b/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingCoordAssignmentService_AssignmentDiscoverServer.go @@ -8,7 +8,7 @@ import ( mock "github.com/stretchr/testify/mock" metadata "google.golang.org/grpc/metadata" - streamingpb "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + streamingpb "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) // MockStreamingCoordAssignmentService_AssignmentDiscoverServer is an autogenerated mock type for the StreamingCoordAssignmentService_AssignmentDiscoverServer type diff --git a/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingNodeHandlerServiceClient.go b/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingNodeHandlerServiceClient.go index 54b30eddfec95..94fa20d06e8f6 100644 --- a/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingNodeHandlerServiceClient.go +++ b/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingNodeHandlerServiceClient.go @@ -9,7 +9,7 @@ import ( mock "github.com/stretchr/testify/mock" - streamingpb "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + streamingpb "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) // MockStreamingNodeHandlerServiceClient is an autogenerated mock type for the StreamingNodeHandlerServiceClient type diff --git a/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingNodeHandlerService_ConsumeClient.go b/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingNodeHandlerService_ConsumeClient.go index 335f43af0c5e5..85e2eb8bac1f4 100644 --- a/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingNodeHandlerService_ConsumeClient.go +++ b/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingNodeHandlerService_ConsumeClient.go @@ -8,7 +8,7 @@ import ( mock "github.com/stretchr/testify/mock" metadata "google.golang.org/grpc/metadata" - streamingpb "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + streamingpb "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) // MockStreamingNodeHandlerService_ConsumeClient is an autogenerated mock type for the StreamingNodeHandlerService_ConsumeClient type diff --git a/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingNodeHandlerService_ConsumeServer.go b/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingNodeHandlerService_ConsumeServer.go index fa6c99ddbfd25..fddf76ff73ae2 100644 --- a/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingNodeHandlerService_ConsumeServer.go +++ b/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingNodeHandlerService_ConsumeServer.go @@ -8,7 +8,7 @@ import ( mock "github.com/stretchr/testify/mock" metadata "google.golang.org/grpc/metadata" - streamingpb "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + streamingpb "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) // MockStreamingNodeHandlerService_ConsumeServer is an autogenerated mock type for the StreamingNodeHandlerService_ConsumeServer type diff --git a/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingNodeHandlerService_ProduceClient.go b/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingNodeHandlerService_ProduceClient.go index 784e313b26462..164b616ce5257 100644 --- a/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingNodeHandlerService_ProduceClient.go +++ b/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingNodeHandlerService_ProduceClient.go @@ -8,7 +8,7 @@ import ( mock "github.com/stretchr/testify/mock" metadata "google.golang.org/grpc/metadata" - streamingpb "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + streamingpb "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) // MockStreamingNodeHandlerService_ProduceClient is an autogenerated mock type for the StreamingNodeHandlerService_ProduceClient type diff --git a/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingNodeHandlerService_ProduceServer.go b/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingNodeHandlerService_ProduceServer.go index 7126e7a3ca005..6ce0e317dd22f 100644 --- a/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingNodeHandlerService_ProduceServer.go +++ b/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingNodeHandlerService_ProduceServer.go @@ -8,7 +8,7 @@ import ( mock "github.com/stretchr/testify/mock" metadata "google.golang.org/grpc/metadata" - streamingpb "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + streamingpb "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) // MockStreamingNodeHandlerService_ProduceServer is an autogenerated mock type for the StreamingNodeHandlerService_ProduceServer type diff --git a/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingNodeManagerServiceClient.go b/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingNodeManagerServiceClient.go index e278c6588c0e7..e5409c192811b 100644 --- a/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingNodeManagerServiceClient.go +++ b/pkg/mocks/streaming/proto/mock_streamingpb/mock_StreamingNodeManagerServiceClient.go @@ -9,7 +9,7 @@ import ( mock "github.com/stretchr/testify/mock" - streamingpb "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + streamingpb "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) // MockStreamingNodeManagerServiceClient is an autogenerated mock type for the StreamingNodeManagerServiceClient type diff --git a/pkg/mq/mqimpl/rocksmq/client/streaming.go b/pkg/mq/mqimpl/rocksmq/client/streaming.go index c317bb26d9ed9..60c372d17b338 100644 --- a/pkg/mq/mqimpl/rocksmq/client/streaming.go +++ b/pkg/mq/mqimpl/rocksmq/client/streaming.go @@ -8,7 +8,7 @@ import ( "github.com/milvus-io/milvus/pkg/mq/common" "github.com/milvus-io/milvus/pkg/mq/mqimpl/rocksmq/server" - "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb" + "github.com/milvus-io/milvus/pkg/proto/messagespb" ) var ( diff --git a/internal/proto/OWNERS b/pkg/proto/OWNERS similarity index 93% rename from internal/proto/OWNERS rename to pkg/proto/OWNERS index 2ddcc3f22e056..0582e3b213473 100644 --- a/internal/proto/OWNERS +++ b/pkg/proto/OWNERS @@ -18,7 +18,7 @@ filters: approvers: - maintainers - "{data_coord|index_coord|internal|proxy|query_coord|root_coord}\\.proto": + "{data_coord|index_coord|internal|proxy|query_coord|root_coord|streaming|messages}\\.proto": labels: - area/internal-api approvers: diff --git a/internal/proto/cgo_msg.proto b/pkg/proto/cgo_msg.proto similarity index 88% rename from internal/proto/cgo_msg.proto rename to pkg/proto/cgo_msg.proto index 6ccfca9e9630b..7b7a516d9da96 100644 --- a/internal/proto/cgo_msg.proto +++ b/pkg/proto/cgo_msg.proto @@ -1,7 +1,7 @@ syntax = "proto3"; package milvus.proto.cgo; -option go_package="github.com/milvus-io/milvus/internal/proto/cgopb"; +option go_package="github.com/milvus-io/milvus/pkg/proto/cgopb"; import "schema.proto"; diff --git a/pkg/proto/cgopb/cgo_msg.pb.go b/pkg/proto/cgopb/cgo_msg.pb.go new file mode 100644 index 0000000000000..91c1105e30aa8 --- /dev/null +++ b/pkg/proto/cgopb/cgo_msg.pb.go @@ -0,0 +1,302 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.33.0 +// protoc v3.21.4 +// source: cgo_msg.proto + +package cgopb + +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" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type LoadIndexInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionID int64 `protobuf:"varint,2,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + SegmentID int64 `protobuf:"varint,3,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + Field *schemapb.FieldSchema `protobuf:"bytes,5,opt,name=field,proto3" json:"field,omitempty"` + EnableMmap bool `protobuf:"varint,6,opt,name=enable_mmap,json=enableMmap,proto3" json:"enable_mmap,omitempty"` + MmapDirPath string `protobuf:"bytes,7,opt,name=mmap_dir_path,json=mmapDirPath,proto3" json:"mmap_dir_path,omitempty"` + IndexID int64 `protobuf:"varint,8,opt,name=indexID,proto3" json:"indexID,omitempty"` + IndexBuildID int64 `protobuf:"varint,9,opt,name=index_buildID,json=indexBuildID,proto3" json:"index_buildID,omitempty"` + IndexVersion int64 `protobuf:"varint,10,opt,name=index_version,json=indexVersion,proto3" json:"index_version,omitempty"` + IndexParams map[string]string `protobuf:"bytes,11,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + IndexFiles []string `protobuf:"bytes,12,rep,name=index_files,json=indexFiles,proto3" json:"index_files,omitempty"` + Uri string `protobuf:"bytes,13,opt,name=uri,proto3" json:"uri,omitempty"` + IndexStoreVersion int64 `protobuf:"varint,14,opt,name=index_store_version,json=indexStoreVersion,proto3" json:"index_store_version,omitempty"` + IndexEngineVersion int32 `protobuf:"varint,15,opt,name=index_engine_version,json=indexEngineVersion,proto3" json:"index_engine_version,omitempty"` + IndexFileSize int64 `protobuf:"varint,16,opt,name=index_file_size,json=indexFileSize,proto3" json:"index_file_size,omitempty"` +} + +func (x *LoadIndexInfo) Reset() { + *x = LoadIndexInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_cgo_msg_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *LoadIndexInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LoadIndexInfo) ProtoMessage() {} + +func (x *LoadIndexInfo) ProtoReflect() protoreflect.Message { + mi := &file_cgo_msg_proto_msgTypes[0] + 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 LoadIndexInfo.ProtoReflect.Descriptor instead. +func (*LoadIndexInfo) Descriptor() ([]byte, []int) { + return file_cgo_msg_proto_rawDescGZIP(), []int{0} +} + +func (x *LoadIndexInfo) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *LoadIndexInfo) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *LoadIndexInfo) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *LoadIndexInfo) GetField() *schemapb.FieldSchema { + if x != nil { + return x.Field + } + return nil +} + +func (x *LoadIndexInfo) GetEnableMmap() bool { + if x != nil { + return x.EnableMmap + } + return false +} + +func (x *LoadIndexInfo) GetMmapDirPath() string { + if x != nil { + return x.MmapDirPath + } + return "" +} + +func (x *LoadIndexInfo) GetIndexID() int64 { + if x != nil { + return x.IndexID + } + return 0 +} + +func (x *LoadIndexInfo) GetIndexBuildID() int64 { + if x != nil { + return x.IndexBuildID + } + return 0 +} + +func (x *LoadIndexInfo) GetIndexVersion() int64 { + if x != nil { + return x.IndexVersion + } + return 0 +} + +func (x *LoadIndexInfo) GetIndexParams() map[string]string { + if x != nil { + return x.IndexParams + } + return nil +} + +func (x *LoadIndexInfo) GetIndexFiles() []string { + if x != nil { + return x.IndexFiles + } + return nil +} + +func (x *LoadIndexInfo) GetUri() string { + if x != nil { + return x.Uri + } + return "" +} + +func (x *LoadIndexInfo) GetIndexStoreVersion() int64 { + if x != nil { + return x.IndexStoreVersion + } + return 0 +} + +func (x *LoadIndexInfo) GetIndexEngineVersion() int32 { + if x != nil { + return x.IndexEngineVersion + } + return 0 +} + +func (x *LoadIndexInfo) GetIndexFileSize() int64 { + if x != nil { + return x.IndexFileSize + } + return 0 +} + +var File_cgo_msg_proto protoreflect.FileDescriptor + +var file_cgo_msg_proto_rawDesc = []byte{ + 0x0a, 0x0d, 0x63, 0x67, 0x6f, 0x5f, 0x6d, 0x73, 0x67, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, + 0x10, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x67, + 0x6f, 0x1a, 0x0c, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, + 0xa6, 0x05, 0x0a, 0x0d, 0x4c, 0x6f, 0x61, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, + 0x6f, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x36, 0x0a, 0x05, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x12, 0x1f, 0x0a, + 0x0b, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6d, 0x6d, 0x61, 0x70, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0a, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x4d, 0x6d, 0x61, 0x70, 0x12, 0x22, + 0x0a, 0x0d, 0x6d, 0x6d, 0x61, 0x70, 0x5f, 0x64, 0x69, 0x72, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, + 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6d, 0x6d, 0x61, 0x70, 0x44, 0x69, 0x72, 0x50, 0x61, + 0x74, 0x68, 0x12, 0x18, 0x0a, 0x07, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x18, 0x08, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x07, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x12, 0x23, 0x0a, 0x0d, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x09, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0c, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, + 0x44, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x56, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x53, 0x0a, 0x0c, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, + 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x0b, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x67, 0x6f, 0x2e, + 0x4c, 0x6f, 0x61, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x49, 0x6e, + 0x64, 0x65, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0b, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x0c, 0x20, 0x03, 0x28, 0x09, + 0x52, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x12, 0x10, 0x0a, 0x03, + 0x75, 0x72, 0x69, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x69, 0x12, 0x2e, + 0x0a, 0x13, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x5f, 0x76, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x03, 0x52, 0x11, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x53, 0x74, 0x6f, 0x72, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x30, + 0x0a, 0x14, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x5f, 0x76, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x05, 0x52, 0x12, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x45, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x12, 0x26, 0x0a, 0x0f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x73, + 0x69, 0x7a, 0x65, 0x18, 0x10, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x46, 0x69, 0x6c, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x1a, 0x3e, 0x0a, 0x10, 0x49, 0x6e, 0x64, 0x65, + 0x78, 0x50, 0x61, 0x72, 0x61, 0x6d, 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, 0x42, 0x2d, 0x5a, 0x2b, 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, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2f, 0x63, 0x67, 0x6f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_cgo_msg_proto_rawDescOnce sync.Once + file_cgo_msg_proto_rawDescData = file_cgo_msg_proto_rawDesc +) + +func file_cgo_msg_proto_rawDescGZIP() []byte { + file_cgo_msg_proto_rawDescOnce.Do(func() { + file_cgo_msg_proto_rawDescData = protoimpl.X.CompressGZIP(file_cgo_msg_proto_rawDescData) + }) + return file_cgo_msg_proto_rawDescData +} + +var file_cgo_msg_proto_msgTypes = make([]protoimpl.MessageInfo, 2) +var file_cgo_msg_proto_goTypes = []interface{}{ + (*LoadIndexInfo)(nil), // 0: milvus.proto.cgo.LoadIndexInfo + nil, // 1: milvus.proto.cgo.LoadIndexInfo.IndexParamsEntry + (*schemapb.FieldSchema)(nil), // 2: milvus.proto.schema.FieldSchema +} +var file_cgo_msg_proto_depIdxs = []int32{ + 2, // 0: milvus.proto.cgo.LoadIndexInfo.field:type_name -> milvus.proto.schema.FieldSchema + 1, // 1: milvus.proto.cgo.LoadIndexInfo.index_params:type_name -> milvus.proto.cgo.LoadIndexInfo.IndexParamsEntry + 2, // [2:2] is the sub-list for method output_type + 2, // [2:2] is the sub-list for method input_type + 2, // [2:2] is the sub-list for extension type_name + 2, // [2:2] is the sub-list for extension extendee + 0, // [0:2] is the sub-list for field type_name +} + +func init() { file_cgo_msg_proto_init() } +func file_cgo_msg_proto_init() { + if File_cgo_msg_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_cgo_msg_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*LoadIndexInfo); 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{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_cgo_msg_proto_rawDesc, + NumEnums: 0, + NumMessages: 2, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_cgo_msg_proto_goTypes, + DependencyIndexes: file_cgo_msg_proto_depIdxs, + MessageInfos: file_cgo_msg_proto_msgTypes, + }.Build() + File_cgo_msg_proto = out.File + file_cgo_msg_proto_rawDesc = nil + file_cgo_msg_proto_goTypes = nil + file_cgo_msg_proto_depIdxs = nil +} diff --git a/internal/proto/clustering.proto b/pkg/proto/clustering.proto similarity index 94% rename from internal/proto/clustering.proto rename to pkg/proto/clustering.proto index 6c41cb3b8c321..eef47b534f800 100644 --- a/internal/proto/clustering.proto +++ b/pkg/proto/clustering.proto @@ -1,7 +1,7 @@ syntax = "proto3"; package milvus.proto.clustering; -option go_package = "github.com/milvus-io/milvus/internal/proto/clusteringpb"; +option go_package = "github.com/milvus-io/milvus/pkg/proto/clusteringpb"; import "schema.proto"; // Synchronously modify StorageConfig in index_coord.proto/index_cgo_msg.proto file diff --git a/pkg/proto/clusteringpb/clustering.pb.go b/pkg/proto/clusteringpb/clustering.pb.go new file mode 100644 index 0000000000000..cf0d771aadbf4 --- /dev/null +++ b/pkg/proto/clusteringpb/clustering.pb.go @@ -0,0 +1,741 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.33.0 +// protoc v3.21.4 +// source: clustering.proto + +package clusteringpb + +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" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// Synchronously modify StorageConfig in index_coord.proto/index_cgo_msg.proto file +type StorageConfig struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Address string `protobuf:"bytes,1,opt,name=address,proto3" json:"address,omitempty"` + AccessKeyID string `protobuf:"bytes,2,opt,name=access_keyID,json=accessKeyID,proto3" json:"access_keyID,omitempty"` + SecretAccessKey string `protobuf:"bytes,3,opt,name=secret_access_key,json=secretAccessKey,proto3" json:"secret_access_key,omitempty"` + UseSSL bool `protobuf:"varint,4,opt,name=useSSL,proto3" json:"useSSL,omitempty"` + BucketName string `protobuf:"bytes,5,opt,name=bucket_name,json=bucketName,proto3" json:"bucket_name,omitempty"` + RootPath string `protobuf:"bytes,6,opt,name=root_path,json=rootPath,proto3" json:"root_path,omitempty"` + UseIAM bool `protobuf:"varint,7,opt,name=useIAM,proto3" json:"useIAM,omitempty"` + IAMEndpoint string `protobuf:"bytes,8,opt,name=IAMEndpoint,proto3" json:"IAMEndpoint,omitempty"` + StorageType string `protobuf:"bytes,9,opt,name=storage_type,json=storageType,proto3" json:"storage_type,omitempty"` + UseVirtualHost bool `protobuf:"varint,10,opt,name=use_virtual_host,json=useVirtualHost,proto3" json:"use_virtual_host,omitempty"` + Region string `protobuf:"bytes,11,opt,name=region,proto3" json:"region,omitempty"` + CloudProvider string `protobuf:"bytes,12,opt,name=cloud_provider,json=cloudProvider,proto3" json:"cloud_provider,omitempty"` + RequestTimeoutMs int64 `protobuf:"varint,13,opt,name=request_timeout_ms,json=requestTimeoutMs,proto3" json:"request_timeout_ms,omitempty"` + SslCACert string `protobuf:"bytes,14,opt,name=sslCACert,proto3" json:"sslCACert,omitempty"` + GcpCredentialJSON string `protobuf:"bytes,15,opt,name=GcpCredentialJSON,proto3" json:"GcpCredentialJSON,omitempty"` +} + +func (x *StorageConfig) Reset() { + *x = StorageConfig{} + if protoimpl.UnsafeEnabled { + mi := &file_clustering_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StorageConfig) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StorageConfig) ProtoMessage() {} + +func (x *StorageConfig) ProtoReflect() protoreflect.Message { + mi := &file_clustering_proto_msgTypes[0] + 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 StorageConfig.ProtoReflect.Descriptor instead. +func (*StorageConfig) Descriptor() ([]byte, []int) { + return file_clustering_proto_rawDescGZIP(), []int{0} +} + +func (x *StorageConfig) GetAddress() string { + if x != nil { + return x.Address + } + return "" +} + +func (x *StorageConfig) GetAccessKeyID() string { + if x != nil { + return x.AccessKeyID + } + return "" +} + +func (x *StorageConfig) GetSecretAccessKey() string { + if x != nil { + return x.SecretAccessKey + } + return "" +} + +func (x *StorageConfig) GetUseSSL() bool { + if x != nil { + return x.UseSSL + } + return false +} + +func (x *StorageConfig) GetBucketName() string { + if x != nil { + return x.BucketName + } + return "" +} + +func (x *StorageConfig) GetRootPath() string { + if x != nil { + return x.RootPath + } + return "" +} + +func (x *StorageConfig) GetUseIAM() bool { + if x != nil { + return x.UseIAM + } + return false +} + +func (x *StorageConfig) GetIAMEndpoint() string { + if x != nil { + return x.IAMEndpoint + } + return "" +} + +func (x *StorageConfig) GetStorageType() string { + if x != nil { + return x.StorageType + } + return "" +} + +func (x *StorageConfig) GetUseVirtualHost() bool { + if x != nil { + return x.UseVirtualHost + } + return false +} + +func (x *StorageConfig) GetRegion() string { + if x != nil { + return x.Region + } + return "" +} + +func (x *StorageConfig) GetCloudProvider() string { + if x != nil { + return x.CloudProvider + } + return "" +} + +func (x *StorageConfig) GetRequestTimeoutMs() int64 { + if x != nil { + return x.RequestTimeoutMs + } + return 0 +} + +func (x *StorageConfig) GetSslCACert() string { + if x != nil { + return x.SslCACert + } + return "" +} + +func (x *StorageConfig) GetGcpCredentialJSON() string { + if x != nil { + return x.GcpCredentialJSON + } + return "" +} + +type InsertFiles struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + InsertFiles []string `protobuf:"bytes,1,rep,name=insert_files,json=insertFiles,proto3" json:"insert_files,omitempty"` +} + +func (x *InsertFiles) Reset() { + *x = InsertFiles{} + if protoimpl.UnsafeEnabled { + mi := &file_clustering_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *InsertFiles) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*InsertFiles) ProtoMessage() {} + +func (x *InsertFiles) ProtoReflect() protoreflect.Message { + mi := &file_clustering_proto_msgTypes[1] + 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 InsertFiles.ProtoReflect.Descriptor instead. +func (*InsertFiles) Descriptor() ([]byte, []int) { + return file_clustering_proto_rawDescGZIP(), []int{1} +} + +func (x *InsertFiles) GetInsertFiles() []string { + if x != nil { + return x.InsertFiles + } + return nil +} + +type AnalyzeInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ClusterID string `protobuf:"bytes,1,opt,name=clusterID,proto3" json:"clusterID,omitempty"` + BuildID int64 `protobuf:"varint,2,opt,name=buildID,proto3" json:"buildID,omitempty"` + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionID int64 `protobuf:"varint,4,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + SegmentID int64 `protobuf:"varint,5,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + Version int64 `protobuf:"varint,6,opt,name=version,proto3" json:"version,omitempty"` + Dim int64 `protobuf:"varint,7,opt,name=dim,proto3" json:"dim,omitempty"` + NumClusters int64 `protobuf:"varint,8,opt,name=num_clusters,json=numClusters,proto3" json:"num_clusters,omitempty"` + TrainSize int64 `protobuf:"varint,9,opt,name=train_size,json=trainSize,proto3" json:"train_size,omitempty"` + MinClusterRatio float64 `protobuf:"fixed64,10,opt,name=min_cluster_ratio,json=minClusterRatio,proto3" json:"min_cluster_ratio,omitempty"` // min_cluster_size / avg_cluster_size < min_cluster_ratio, is skew + MaxClusterRatio float64 `protobuf:"fixed64,11,opt,name=max_cluster_ratio,json=maxClusterRatio,proto3" json:"max_cluster_ratio,omitempty"` // max_cluster_size / avg_cluster_size > max_cluster_ratio, is skew + MaxClusterSize int64 `protobuf:"varint,12,opt,name=max_cluster_size,json=maxClusterSize,proto3" json:"max_cluster_size,omitempty"` + InsertFiles map[int64]*InsertFiles `protobuf:"bytes,13,rep,name=insert_files,json=insertFiles,proto3" json:"insert_files,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + NumRows map[int64]int64 `protobuf:"bytes,14,rep,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + FieldSchema *schemapb.FieldSchema `protobuf:"bytes,15,opt,name=field_schema,json=fieldSchema,proto3" json:"field_schema,omitempty"` + StorageConfig *StorageConfig `protobuf:"bytes,16,opt,name=storage_config,json=storageConfig,proto3" json:"storage_config,omitempty"` +} + +func (x *AnalyzeInfo) Reset() { + *x = AnalyzeInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_clustering_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AnalyzeInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AnalyzeInfo) ProtoMessage() {} + +func (x *AnalyzeInfo) ProtoReflect() protoreflect.Message { + mi := &file_clustering_proto_msgTypes[2] + 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 AnalyzeInfo.ProtoReflect.Descriptor instead. +func (*AnalyzeInfo) Descriptor() ([]byte, []int) { + return file_clustering_proto_rawDescGZIP(), []int{2} +} + +func (x *AnalyzeInfo) GetClusterID() string { + if x != nil { + return x.ClusterID + } + return "" +} + +func (x *AnalyzeInfo) GetBuildID() int64 { + if x != nil { + return x.BuildID + } + return 0 +} + +func (x *AnalyzeInfo) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *AnalyzeInfo) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *AnalyzeInfo) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *AnalyzeInfo) GetVersion() int64 { + if x != nil { + return x.Version + } + return 0 +} + +func (x *AnalyzeInfo) GetDim() int64 { + if x != nil { + return x.Dim + } + return 0 +} + +func (x *AnalyzeInfo) GetNumClusters() int64 { + if x != nil { + return x.NumClusters + } + return 0 +} + +func (x *AnalyzeInfo) GetTrainSize() int64 { + if x != nil { + return x.TrainSize + } + return 0 +} + +func (x *AnalyzeInfo) GetMinClusterRatio() float64 { + if x != nil { + return x.MinClusterRatio + } + return 0 +} + +func (x *AnalyzeInfo) GetMaxClusterRatio() float64 { + if x != nil { + return x.MaxClusterRatio + } + return 0 +} + +func (x *AnalyzeInfo) GetMaxClusterSize() int64 { + if x != nil { + return x.MaxClusterSize + } + return 0 +} + +func (x *AnalyzeInfo) GetInsertFiles() map[int64]*InsertFiles { + if x != nil { + return x.InsertFiles + } + return nil +} + +func (x *AnalyzeInfo) GetNumRows() map[int64]int64 { + if x != nil { + return x.NumRows + } + return nil +} + +func (x *AnalyzeInfo) GetFieldSchema() *schemapb.FieldSchema { + if x != nil { + return x.FieldSchema + } + return nil +} + +func (x *AnalyzeInfo) GetStorageConfig() *StorageConfig { + if x != nil { + return x.StorageConfig + } + return nil +} + +type ClusteringCentroidsStats struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Centroids []*schemapb.VectorField `protobuf:"bytes,1,rep,name=centroids,proto3" json:"centroids,omitempty"` +} + +func (x *ClusteringCentroidsStats) Reset() { + *x = ClusteringCentroidsStats{} + if protoimpl.UnsafeEnabled { + mi := &file_clustering_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ClusteringCentroidsStats) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ClusteringCentroidsStats) ProtoMessage() {} + +func (x *ClusteringCentroidsStats) ProtoReflect() protoreflect.Message { + mi := &file_clustering_proto_msgTypes[3] + 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 ClusteringCentroidsStats.ProtoReflect.Descriptor instead. +func (*ClusteringCentroidsStats) Descriptor() ([]byte, []int) { + return file_clustering_proto_rawDescGZIP(), []int{3} +} + +func (x *ClusteringCentroidsStats) GetCentroids() []*schemapb.VectorField { + if x != nil { + return x.Centroids + } + return nil +} + +type ClusteringCentroidIdMappingStats struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CentroidIdMapping []uint32 `protobuf:"varint,1,rep,packed,name=centroid_id_mapping,json=centroidIdMapping,proto3" json:"centroid_id_mapping,omitempty"` + NumInCentroid []int64 `protobuf:"varint,2,rep,packed,name=num_in_centroid,json=numInCentroid,proto3" json:"num_in_centroid,omitempty"` +} + +func (x *ClusteringCentroidIdMappingStats) Reset() { + *x = ClusteringCentroidIdMappingStats{} + if protoimpl.UnsafeEnabled { + mi := &file_clustering_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ClusteringCentroidIdMappingStats) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ClusteringCentroidIdMappingStats) ProtoMessage() {} + +func (x *ClusteringCentroidIdMappingStats) ProtoReflect() protoreflect.Message { + mi := &file_clustering_proto_msgTypes[4] + 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 ClusteringCentroidIdMappingStats.ProtoReflect.Descriptor instead. +func (*ClusteringCentroidIdMappingStats) Descriptor() ([]byte, []int) { + return file_clustering_proto_rawDescGZIP(), []int{4} +} + +func (x *ClusteringCentroidIdMappingStats) GetCentroidIdMapping() []uint32 { + if x != nil { + return x.CentroidIdMapping + } + return nil +} + +func (x *ClusteringCentroidIdMappingStats) GetNumInCentroid() []int64 { + if x != nil { + return x.NumInCentroid + } + return nil +} + +var File_clustering_proto protoreflect.FileDescriptor + +var file_clustering_proto_rawDesc = []byte{ + 0x0a, 0x10, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x12, 0x17, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x1a, 0x0c, 0x73, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x8e, 0x04, 0x0a, 0x0d, 0x53, 0x74, + 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x18, 0x0a, 0x07, 0x61, + 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, + 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, + 0x6b, 0x65, 0x79, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x61, 0x63, 0x63, + 0x65, 0x73, 0x73, 0x4b, 0x65, 0x79, 0x49, 0x44, 0x12, 0x2a, 0x0a, 0x11, 0x73, 0x65, 0x63, 0x72, + 0x65, 0x74, 0x5f, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0f, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, 0x41, 0x63, 0x63, 0x65, 0x73, + 0x73, 0x4b, 0x65, 0x79, 0x12, 0x16, 0x0a, 0x06, 0x75, 0x73, 0x65, 0x53, 0x53, 0x4c, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x75, 0x73, 0x65, 0x53, 0x53, 0x4c, 0x12, 0x1f, 0x0a, 0x0b, + 0x62, 0x75, 0x63, 0x6b, 0x65, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0a, 0x62, 0x75, 0x63, 0x6b, 0x65, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1b, 0x0a, + 0x09, 0x72, 0x6f, 0x6f, 0x74, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x72, 0x6f, 0x6f, 0x74, 0x50, 0x61, 0x74, 0x68, 0x12, 0x16, 0x0a, 0x06, 0x75, 0x73, + 0x65, 0x49, 0x41, 0x4d, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x75, 0x73, 0x65, 0x49, + 0x41, 0x4d, 0x12, 0x20, 0x0a, 0x0b, 0x49, 0x41, 0x4d, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, + 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x49, 0x41, 0x4d, 0x45, 0x6e, 0x64, 0x70, + 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, + 0x74, 0x79, 0x70, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x74, 0x6f, 0x72, + 0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x28, 0x0a, 0x10, 0x75, 0x73, 0x65, 0x5f, 0x76, + 0x69, 0x72, 0x74, 0x75, 0x61, 0x6c, 0x5f, 0x68, 0x6f, 0x73, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x0e, 0x75, 0x73, 0x65, 0x56, 0x69, 0x72, 0x74, 0x75, 0x61, 0x6c, 0x48, 0x6f, 0x73, + 0x74, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x18, 0x0b, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x06, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6c, 0x6f, + 0x75, 0x64, 0x5f, 0x70, 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x18, 0x0c, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0d, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x50, 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, + 0x12, 0x2c, 0x0a, 0x12, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x6f, 0x75, 0x74, 0x5f, 0x6d, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4d, 0x73, 0x12, 0x1c, + 0x0a, 0x09, 0x73, 0x73, 0x6c, 0x43, 0x41, 0x43, 0x65, 0x72, 0x74, 0x18, 0x0e, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x09, 0x73, 0x73, 0x6c, 0x43, 0x41, 0x43, 0x65, 0x72, 0x74, 0x12, 0x2c, 0x0a, 0x11, + 0x47, 0x63, 0x70, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x4a, 0x53, 0x4f, + 0x4e, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x47, 0x63, 0x70, 0x43, 0x72, 0x65, 0x64, + 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x4a, 0x53, 0x4f, 0x4e, 0x22, 0x30, 0x0a, 0x0b, 0x49, 0x6e, + 0x73, 0x65, 0x72, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x69, 0x6e, 0x73, + 0x65, 0x72, 0x74, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x0b, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x22, 0xf7, 0x06, 0x0a, + 0x0b, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1c, 0x0a, 0x09, + 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x62, 0x75, + 0x69, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x62, 0x75, 0x69, + 0x6c, 0x64, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, + 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x64, 0x69, 0x6d, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x03, 0x64, 0x69, 0x6d, 0x12, 0x21, 0x0a, 0x0c, 0x6e, 0x75, 0x6d, 0x5f, 0x63, 0x6c, 0x75, 0x73, + 0x74, 0x65, 0x72, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x6e, 0x75, 0x6d, 0x43, + 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x72, 0x61, 0x69, 0x6e, + 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x72, 0x61, + 0x69, 0x6e, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x2a, 0x0a, 0x11, 0x6d, 0x69, 0x6e, 0x5f, 0x63, 0x6c, + 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x18, 0x0a, 0x20, 0x01, 0x28, + 0x01, 0x52, 0x0f, 0x6d, 0x69, 0x6e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x52, 0x61, 0x74, + 0x69, 0x6f, 0x12, 0x2a, 0x0a, 0x11, 0x6d, 0x61, 0x78, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, + 0x72, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0f, 0x6d, + 0x61, 0x78, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x52, 0x61, 0x74, 0x69, 0x6f, 0x12, 0x28, + 0x0a, 0x10, 0x6d, 0x61, 0x78, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x73, 0x69, + 0x7a, 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x6d, 0x61, 0x78, 0x43, 0x6c, 0x75, + 0x73, 0x74, 0x65, 0x72, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x58, 0x0a, 0x0c, 0x69, 0x6e, 0x73, 0x65, + 0x72, 0x74, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x0d, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x35, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6c, + 0x75, 0x73, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, + 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x73, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0b, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x46, 0x69, 0x6c, + 0x65, 0x73, 0x12, 0x4c, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x0e, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x41, + 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x4e, 0x75, 0x6d, 0x52, 0x6f, + 0x77, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, + 0x12, 0x43, 0x0a, 0x0c, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x46, 0x69, 0x65, + 0x6c, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x0b, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x53, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x4d, 0x0a, 0x0e, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, + 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6c, 0x75, + 0x73, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x0d, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x1a, 0x64, 0x0a, 0x10, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x46, 0x69, + 0x6c, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x3a, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, + 0x69, 0x6e, 0x67, 0x2e, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x3a, 0x0a, 0x0c, 0x4e, 0x75, + 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x5a, 0x0a, 0x18, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, + 0x72, 0x69, 0x6e, 0x67, 0x43, 0x65, 0x6e, 0x74, 0x72, 0x6f, 0x69, 0x64, 0x73, 0x53, 0x74, 0x61, + 0x74, 0x73, 0x12, 0x3e, 0x0a, 0x09, 0x63, 0x65, 0x6e, 0x74, 0x72, 0x6f, 0x69, 0x64, 0x73, 0x18, + 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x56, 0x65, 0x63, 0x74, + 0x6f, 0x72, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x09, 0x63, 0x65, 0x6e, 0x74, 0x72, 0x6f, 0x69, + 0x64, 0x73, 0x22, 0x7a, 0x0a, 0x20, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, + 0x43, 0x65, 0x6e, 0x74, 0x72, 0x6f, 0x69, 0x64, 0x49, 0x64, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, + 0x67, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x2e, 0x0a, 0x13, 0x63, 0x65, 0x6e, 0x74, 0x72, 0x6f, + 0x69, 0x64, 0x5f, 0x69, 0x64, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x0d, 0x52, 0x11, 0x63, 0x65, 0x6e, 0x74, 0x72, 0x6f, 0x69, 0x64, 0x49, 0x64, 0x4d, + 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x12, 0x26, 0x0a, 0x0f, 0x6e, 0x75, 0x6d, 0x5f, 0x69, 0x6e, + 0x5f, 0x63, 0x65, 0x6e, 0x74, 0x72, 0x6f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, + 0x0d, 0x6e, 0x75, 0x6d, 0x49, 0x6e, 0x43, 0x65, 0x6e, 0x74, 0x72, 0x6f, 0x69, 0x64, 0x42, 0x34, + 0x5a, 0x32, 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, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x69, + 0x6e, 0x67, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_clustering_proto_rawDescOnce sync.Once + file_clustering_proto_rawDescData = file_clustering_proto_rawDesc +) + +func file_clustering_proto_rawDescGZIP() []byte { + file_clustering_proto_rawDescOnce.Do(func() { + file_clustering_proto_rawDescData = protoimpl.X.CompressGZIP(file_clustering_proto_rawDescData) + }) + return file_clustering_proto_rawDescData +} + +var file_clustering_proto_msgTypes = make([]protoimpl.MessageInfo, 7) +var file_clustering_proto_goTypes = []interface{}{ + (*StorageConfig)(nil), // 0: milvus.proto.clustering.StorageConfig + (*InsertFiles)(nil), // 1: milvus.proto.clustering.InsertFiles + (*AnalyzeInfo)(nil), // 2: milvus.proto.clustering.AnalyzeInfo + (*ClusteringCentroidsStats)(nil), // 3: milvus.proto.clustering.ClusteringCentroidsStats + (*ClusteringCentroidIdMappingStats)(nil), // 4: milvus.proto.clustering.ClusteringCentroidIdMappingStats + nil, // 5: milvus.proto.clustering.AnalyzeInfo.InsertFilesEntry + nil, // 6: milvus.proto.clustering.AnalyzeInfo.NumRowsEntry + (*schemapb.FieldSchema)(nil), // 7: milvus.proto.schema.FieldSchema + (*schemapb.VectorField)(nil), // 8: milvus.proto.schema.VectorField +} +var file_clustering_proto_depIdxs = []int32{ + 5, // 0: milvus.proto.clustering.AnalyzeInfo.insert_files:type_name -> milvus.proto.clustering.AnalyzeInfo.InsertFilesEntry + 6, // 1: milvus.proto.clustering.AnalyzeInfo.num_rows:type_name -> milvus.proto.clustering.AnalyzeInfo.NumRowsEntry + 7, // 2: milvus.proto.clustering.AnalyzeInfo.field_schema:type_name -> milvus.proto.schema.FieldSchema + 0, // 3: milvus.proto.clustering.AnalyzeInfo.storage_config:type_name -> milvus.proto.clustering.StorageConfig + 8, // 4: milvus.proto.clustering.ClusteringCentroidsStats.centroids:type_name -> milvus.proto.schema.VectorField + 1, // 5: milvus.proto.clustering.AnalyzeInfo.InsertFilesEntry.value:type_name -> milvus.proto.clustering.InsertFiles + 6, // [6:6] is the sub-list for method output_type + 6, // [6:6] is the sub-list for method input_type + 6, // [6:6] is the sub-list for extension type_name + 6, // [6:6] is the sub-list for extension extendee + 0, // [0:6] is the sub-list for field type_name +} + +func init() { file_clustering_proto_init() } +func file_clustering_proto_init() { + if File_clustering_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_clustering_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*StorageConfig); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_clustering_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*InsertFiles); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_clustering_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AnalyzeInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_clustering_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ClusteringCentroidsStats); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_clustering_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ClusteringCentroidIdMappingStats); 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{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_clustering_proto_rawDesc, + NumEnums: 0, + NumMessages: 7, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_clustering_proto_goTypes, + DependencyIndexes: file_clustering_proto_depIdxs, + MessageInfos: file_clustering_proto_msgTypes, + }.Build() + File_clustering_proto = out.File + file_clustering_proto_rawDesc = nil + file_clustering_proto_goTypes = nil + file_clustering_proto_depIdxs = nil +} diff --git a/internal/proto/data_coord.proto b/pkg/proto/data_coord.proto similarity index 99% rename from internal/proto/data_coord.proto rename to pkg/proto/data_coord.proto index 5d78f28cf0ddf..b560cd6f162af 100644 --- a/internal/proto/data_coord.proto +++ b/pkg/proto/data_coord.proto @@ -2,7 +2,7 @@ syntax = "proto3"; package milvus.proto.data; -option go_package = "github.com/milvus-io/milvus/internal/proto/datapb"; +option go_package = "github.com/milvus-io/milvus/pkg/proto/datapb"; import "common.proto"; import "internal.proto"; diff --git a/pkg/proto/datapb/data_coord.pb.go b/pkg/proto/datapb/data_coord.pb.go new file mode 100644 index 0000000000000..baaec64de0ec0 --- /dev/null +++ b/pkg/proto/datapb/data_coord.pb.go @@ -0,0 +1,11904 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.33.0 +// protoc v3.21.4 +// source: data_coord.proto + +package datapb + +import ( + commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + msgpb "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" + schemapb "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" + indexpb "github.com/milvus-io/milvus/pkg/proto/indexpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type SegmentType int32 + +const ( + SegmentType_New SegmentType = 0 + SegmentType_Normal SegmentType = 1 + SegmentType_Flushed SegmentType = 2 + SegmentType_Compacted SegmentType = 3 +) + +// Enum value maps for SegmentType. +var ( + SegmentType_name = map[int32]string{ + 0: "New", + 1: "Normal", + 2: "Flushed", + 3: "Compacted", + } + SegmentType_value = map[string]int32{ + "New": 0, + "Normal": 1, + "Flushed": 2, + "Compacted": 3, + } +) + +func (x SegmentType) Enum() *SegmentType { + p := new(SegmentType) + *p = x + return p +} + +func (x SegmentType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SegmentType) Descriptor() protoreflect.EnumDescriptor { + return file_data_coord_proto_enumTypes[0].Descriptor() +} + +func (SegmentType) Type() protoreflect.EnumType { + return &file_data_coord_proto_enumTypes[0] +} + +func (x SegmentType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use SegmentType.Descriptor instead. +func (SegmentType) EnumDescriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{0} +} + +type SegmentLevel int32 + +const ( + SegmentLevel_Legacy SegmentLevel = 0 // zero value for legacy logic + SegmentLevel_L0 SegmentLevel = 1 // L0 segment, contains delta data for current channel + SegmentLevel_L1 SegmentLevel = 2 // L1 segment, normal segment, with no extra compaction attribute + SegmentLevel_L2 SegmentLevel = 3 // L2 segment, segment with extra data distribution info +) + +// Enum value maps for SegmentLevel. +var ( + SegmentLevel_name = map[int32]string{ + 0: "Legacy", + 1: "L0", + 2: "L1", + 3: "L2", + } + SegmentLevel_value = map[string]int32{ + "Legacy": 0, + "L0": 1, + "L1": 2, + "L2": 3, + } +) + +func (x SegmentLevel) Enum() *SegmentLevel { + p := new(SegmentLevel) + *p = x + return p +} + +func (x SegmentLevel) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SegmentLevel) Descriptor() protoreflect.EnumDescriptor { + return file_data_coord_proto_enumTypes[1].Descriptor() +} + +func (SegmentLevel) Type() protoreflect.EnumType { + return &file_data_coord_proto_enumTypes[1] +} + +func (x SegmentLevel) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use SegmentLevel.Descriptor instead. +func (SegmentLevel) EnumDescriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{1} +} + +type ChannelWatchState int32 + +const ( + ChannelWatchState_Uncomplete ChannelWatchState = 0 // deprecated, keep it for compatibility + ChannelWatchState_Complete ChannelWatchState = 1 // deprecated, keep it for compatibility + ChannelWatchState_ToWatch ChannelWatchState = 2 + ChannelWatchState_WatchSuccess ChannelWatchState = 3 + ChannelWatchState_WatchFailure ChannelWatchState = 4 + ChannelWatchState_ToRelease ChannelWatchState = 5 + ChannelWatchState_ReleaseSuccess ChannelWatchState = 6 + ChannelWatchState_ReleaseFailure ChannelWatchState = 7 +) + +// Enum value maps for ChannelWatchState. +var ( + ChannelWatchState_name = map[int32]string{ + 0: "Uncomplete", + 1: "Complete", + 2: "ToWatch", + 3: "WatchSuccess", + 4: "WatchFailure", + 5: "ToRelease", + 6: "ReleaseSuccess", + 7: "ReleaseFailure", + } + ChannelWatchState_value = map[string]int32{ + "Uncomplete": 0, + "Complete": 1, + "ToWatch": 2, + "WatchSuccess": 3, + "WatchFailure": 4, + "ToRelease": 5, + "ReleaseSuccess": 6, + "ReleaseFailure": 7, + } +) + +func (x ChannelWatchState) Enum() *ChannelWatchState { + p := new(ChannelWatchState) + *p = x + return p +} + +func (x ChannelWatchState) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (ChannelWatchState) Descriptor() protoreflect.EnumDescriptor { + return file_data_coord_proto_enumTypes[2].Descriptor() +} + +func (ChannelWatchState) Type() protoreflect.EnumType { + return &file_data_coord_proto_enumTypes[2] +} + +func (x ChannelWatchState) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ChannelWatchState.Descriptor instead. +func (ChannelWatchState) EnumDescriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{2} +} + +type CompactionType int32 + +const ( + CompactionType_UndefinedCompaction CompactionType = 0 + CompactionType_MergeCompaction CompactionType = 2 + CompactionType_MixCompaction CompactionType = 3 + // compactionV2 + CompactionType_SingleCompaction CompactionType = 4 + CompactionType_MinorCompaction CompactionType = 5 + CompactionType_MajorCompaction CompactionType = 6 + CompactionType_Level0DeleteCompaction CompactionType = 7 + CompactionType_ClusteringCompaction CompactionType = 8 +) + +// Enum value maps for CompactionType. +var ( + CompactionType_name = map[int32]string{ + 0: "UndefinedCompaction", + 2: "MergeCompaction", + 3: "MixCompaction", + 4: "SingleCompaction", + 5: "MinorCompaction", + 6: "MajorCompaction", + 7: "Level0DeleteCompaction", + 8: "ClusteringCompaction", + } + CompactionType_value = map[string]int32{ + "UndefinedCompaction": 0, + "MergeCompaction": 2, + "MixCompaction": 3, + "SingleCompaction": 4, + "MinorCompaction": 5, + "MajorCompaction": 6, + "Level0DeleteCompaction": 7, + "ClusteringCompaction": 8, + } +) + +func (x CompactionType) Enum() *CompactionType { + p := new(CompactionType) + *p = x + return p +} + +func (x CompactionType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (CompactionType) Descriptor() protoreflect.EnumDescriptor { + return file_data_coord_proto_enumTypes[3].Descriptor() +} + +func (CompactionType) Type() protoreflect.EnumType { + return &file_data_coord_proto_enumTypes[3] +} + +func (x CompactionType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use CompactionType.Descriptor instead. +func (CompactionType) EnumDescriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{3} +} + +type ImportTaskStateV2 int32 + +const ( + ImportTaskStateV2_None ImportTaskStateV2 = 0 + ImportTaskStateV2_Pending ImportTaskStateV2 = 1 + ImportTaskStateV2_InProgress ImportTaskStateV2 = 2 + ImportTaskStateV2_Failed ImportTaskStateV2 = 3 + ImportTaskStateV2_Completed ImportTaskStateV2 = 4 +) + +// Enum value maps for ImportTaskStateV2. +var ( + ImportTaskStateV2_name = map[int32]string{ + 0: "None", + 1: "Pending", + 2: "InProgress", + 3: "Failed", + 4: "Completed", + } + ImportTaskStateV2_value = map[string]int32{ + "None": 0, + "Pending": 1, + "InProgress": 2, + "Failed": 3, + "Completed": 4, + } +) + +func (x ImportTaskStateV2) Enum() *ImportTaskStateV2 { + p := new(ImportTaskStateV2) + *p = x + return p +} + +func (x ImportTaskStateV2) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (ImportTaskStateV2) Descriptor() protoreflect.EnumDescriptor { + return file_data_coord_proto_enumTypes[4].Descriptor() +} + +func (ImportTaskStateV2) Type() protoreflect.EnumType { + return &file_data_coord_proto_enumTypes[4] +} + +func (x ImportTaskStateV2) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ImportTaskStateV2.Descriptor instead. +func (ImportTaskStateV2) EnumDescriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{4} +} + +type GcCommand int32 + +const ( + GcCommand__ GcCommand = 0 + GcCommand_Pause GcCommand = 1 + GcCommand_Resume GcCommand = 2 +) + +// Enum value maps for GcCommand. +var ( + GcCommand_name = map[int32]string{ + 0: "_", + 1: "Pause", + 2: "Resume", + } + GcCommand_value = map[string]int32{ + "_": 0, + "Pause": 1, + "Resume": 2, + } +) + +func (x GcCommand) Enum() *GcCommand { + p := new(GcCommand) + *p = x + return p +} + +func (x GcCommand) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (GcCommand) Descriptor() protoreflect.EnumDescriptor { + return file_data_coord_proto_enumTypes[5].Descriptor() +} + +func (GcCommand) Type() protoreflect.EnumType { + return &file_data_coord_proto_enumTypes[5] +} + +func (x GcCommand) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use GcCommand.Descriptor instead. +func (GcCommand) EnumDescriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{5} +} + +type CompactionTaskState int32 + +const ( + CompactionTaskState_unknown CompactionTaskState = 0 + CompactionTaskState_executing CompactionTaskState = 1 + CompactionTaskState_pipelining CompactionTaskState = 2 + CompactionTaskState_completed CompactionTaskState = 3 + CompactionTaskState_failed CompactionTaskState = 4 + CompactionTaskState_timeout CompactionTaskState = 5 + CompactionTaskState_analyzing CompactionTaskState = 6 + CompactionTaskState_indexing CompactionTaskState = 7 + CompactionTaskState_cleaned CompactionTaskState = 8 + CompactionTaskState_meta_saved CompactionTaskState = 9 + CompactionTaskState_statistic CompactionTaskState = 10 +) + +// Enum value maps for CompactionTaskState. +var ( + CompactionTaskState_name = map[int32]string{ + 0: "unknown", + 1: "executing", + 2: "pipelining", + 3: "completed", + 4: "failed", + 5: "timeout", + 6: "analyzing", + 7: "indexing", + 8: "cleaned", + 9: "meta_saved", + 10: "statistic", + } + CompactionTaskState_value = map[string]int32{ + "unknown": 0, + "executing": 1, + "pipelining": 2, + "completed": 3, + "failed": 4, + "timeout": 5, + "analyzing": 6, + "indexing": 7, + "cleaned": 8, + "meta_saved": 9, + "statistic": 10, + } +) + +func (x CompactionTaskState) Enum() *CompactionTaskState { + p := new(CompactionTaskState) + *p = x + return p +} + +func (x CompactionTaskState) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (CompactionTaskState) Descriptor() protoreflect.EnumDescriptor { + return file_data_coord_proto_enumTypes[6].Descriptor() +} + +func (CompactionTaskState) Type() protoreflect.EnumType { + return &file_data_coord_proto_enumTypes[6] +} + +func (x CompactionTaskState) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use CompactionTaskState.Descriptor instead. +func (CompactionTaskState) EnumDescriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{6} +} + +// TODO: import google/protobuf/empty.proto +type Empty struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *Empty) Reset() { + *x = Empty{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Empty) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Empty) ProtoMessage() {} + +func (x *Empty) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[0] + 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 Empty.ProtoReflect.Descriptor instead. +func (*Empty) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{0} +} + +type FlushRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + DbID int64 `protobuf:"varint,2,opt,name=dbID,proto3" json:"dbID,omitempty"` + SegmentIDs []int64 `protobuf:"varint,3,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,omitempty"` + CollectionID int64 `protobuf:"varint,4,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + IsImport bool `protobuf:"varint,5,opt,name=isImport,proto3" json:"isImport,omitempty"` // deprecated +} + +func (x *FlushRequest) Reset() { + *x = FlushRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *FlushRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FlushRequest) ProtoMessage() {} + +func (x *FlushRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[1] + 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 FlushRequest.ProtoReflect.Descriptor instead. +func (*FlushRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{1} +} + +func (x *FlushRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *FlushRequest) GetDbID() int64 { + if x != nil { + return x.DbID + } + return 0 +} + +func (x *FlushRequest) GetSegmentIDs() []int64 { + if x != nil { + return x.SegmentIDs + } + return nil +} + +func (x *FlushRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *FlushRequest) GetIsImport() bool { + if x != nil { + return x.IsImport + } + return false +} + +type FlushResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + DbID int64 `protobuf:"varint,2,opt,name=dbID,proto3" json:"dbID,omitempty"` + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + SegmentIDs []int64 `protobuf:"varint,4,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,omitempty"` // newly sealed segments + FlushSegmentIDs []int64 `protobuf:"varint,5,rep,packed,name=flushSegmentIDs,proto3" json:"flushSegmentIDs,omitempty"` // old flushed segment + TimeOfSeal int64 `protobuf:"varint,6,opt,name=timeOfSeal,proto3" json:"timeOfSeal,omitempty"` + FlushTs uint64 `protobuf:"varint,7,opt,name=flush_ts,json=flushTs,proto3" json:"flush_ts,omitempty"` + ChannelCps map[string]*msgpb.MsgPosition `protobuf:"bytes,8,rep,name=channel_cps,json=channelCps,proto3" json:"channel_cps,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *FlushResponse) Reset() { + *x = FlushResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *FlushResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FlushResponse) ProtoMessage() {} + +func (x *FlushResponse) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[2] + 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 FlushResponse.ProtoReflect.Descriptor instead. +func (*FlushResponse) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{2} +} + +func (x *FlushResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *FlushResponse) GetDbID() int64 { + if x != nil { + return x.DbID + } + return 0 +} + +func (x *FlushResponse) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *FlushResponse) GetSegmentIDs() []int64 { + if x != nil { + return x.SegmentIDs + } + return nil +} + +func (x *FlushResponse) GetFlushSegmentIDs() []int64 { + if x != nil { + return x.FlushSegmentIDs + } + return nil +} + +func (x *FlushResponse) GetTimeOfSeal() int64 { + if x != nil { + return x.TimeOfSeal + } + return 0 +} + +func (x *FlushResponse) GetFlushTs() uint64 { + if x != nil { + return x.FlushTs + } + return 0 +} + +func (x *FlushResponse) GetChannelCps() map[string]*msgpb.MsgPosition { + if x != nil { + return x.ChannelCps + } + return nil +} + +type FlushChannelsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + FlushTs uint64 `protobuf:"varint,2,opt,name=flush_ts,json=flushTs,proto3" json:"flush_ts,omitempty"` + Channels []string `protobuf:"bytes,3,rep,name=channels,proto3" json:"channels,omitempty"` +} + +func (x *FlushChannelsRequest) Reset() { + *x = FlushChannelsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *FlushChannelsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FlushChannelsRequest) ProtoMessage() {} + +func (x *FlushChannelsRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[3] + 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 FlushChannelsRequest.ProtoReflect.Descriptor instead. +func (*FlushChannelsRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{3} +} + +func (x *FlushChannelsRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *FlushChannelsRequest) GetFlushTs() uint64 { + if x != nil { + return x.FlushTs + } + return 0 +} + +func (x *FlushChannelsRequest) GetChannels() []string { + if x != nil { + return x.Channels + } + return nil +} + +type SegmentIDRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Count uint32 `protobuf:"varint,1,opt,name=count,proto3" json:"count,omitempty"` + ChannelName string `protobuf:"bytes,2,opt,name=channel_name,json=channelName,proto3" json:"channel_name,omitempty"` + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionID int64 `protobuf:"varint,4,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + IsImport bool `protobuf:"varint,5,opt,name=isImport,proto3" json:"isImport,omitempty"` // deprecated + ImportTaskID int64 `protobuf:"varint,6,opt,name=importTaskID,proto3" json:"importTaskID,omitempty"` // deprecated + Level SegmentLevel `protobuf:"varint,7,opt,name=level,proto3,enum=milvus.proto.data.SegmentLevel" json:"level,omitempty"` // deprecated +} + +func (x *SegmentIDRequest) Reset() { + *x = SegmentIDRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentIDRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentIDRequest) ProtoMessage() {} + +func (x *SegmentIDRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[4] + 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 SegmentIDRequest.ProtoReflect.Descriptor instead. +func (*SegmentIDRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{4} +} + +func (x *SegmentIDRequest) GetCount() uint32 { + if x != nil { + return x.Count + } + return 0 +} + +func (x *SegmentIDRequest) GetChannelName() string { + if x != nil { + return x.ChannelName + } + return "" +} + +func (x *SegmentIDRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *SegmentIDRequest) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *SegmentIDRequest) GetIsImport() bool { + if x != nil { + return x.IsImport + } + return false +} + +func (x *SegmentIDRequest) GetImportTaskID() int64 { + if x != nil { + return x.ImportTaskID + } + return 0 +} + +func (x *SegmentIDRequest) GetLevel() SegmentLevel { + if x != nil { + return x.Level + } + return SegmentLevel_Legacy +} + +type AllocSegmentRequest 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"` + PartitionId int64 `protobuf:"varint,2,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"` + SegmentId int64 `protobuf:"varint,3,opt,name=segment_id,json=segmentId,proto3" json:"segment_id,omitempty"` // segment id must be allocate from rootcoord idalloc service. + Vchannel string `protobuf:"bytes,4,opt,name=vchannel,proto3" json:"vchannel,omitempty"` +} + +func (x *AllocSegmentRequest) Reset() { + *x = AllocSegmentRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AllocSegmentRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AllocSegmentRequest) ProtoMessage() {} + +func (x *AllocSegmentRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[5] + 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 AllocSegmentRequest.ProtoReflect.Descriptor instead. +func (*AllocSegmentRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{5} +} + +func (x *AllocSegmentRequest) GetCollectionId() int64 { + if x != nil { + return x.CollectionId + } + return 0 +} + +func (x *AllocSegmentRequest) GetPartitionId() int64 { + if x != nil { + return x.PartitionId + } + return 0 +} + +func (x *AllocSegmentRequest) GetSegmentId() int64 { + if x != nil { + return x.SegmentId + } + return 0 +} + +func (x *AllocSegmentRequest) GetVchannel() string { + if x != nil { + return x.Vchannel + } + return "" +} + +type AllocSegmentResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SegmentInfo *SegmentInfo `protobuf:"bytes,1,opt,name=segment_info,json=segmentInfo,proto3" json:"segment_info,omitempty"` + Status *commonpb.Status `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` +} + +func (x *AllocSegmentResponse) Reset() { + *x = AllocSegmentResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AllocSegmentResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AllocSegmentResponse) ProtoMessage() {} + +func (x *AllocSegmentResponse) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[6] + 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 AllocSegmentResponse.ProtoReflect.Descriptor instead. +func (*AllocSegmentResponse) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{6} +} + +func (x *AllocSegmentResponse) GetSegmentInfo() *SegmentInfo { + if x != nil { + return x.SegmentInfo + } + return nil +} + +func (x *AllocSegmentResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +type AssignSegmentIDRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + NodeID int64 `protobuf:"varint,1,opt,name=nodeID,proto3" json:"nodeID,omitempty"` + PeerRole string `protobuf:"bytes,2,opt,name=peer_role,json=peerRole,proto3" json:"peer_role,omitempty"` + SegmentIDRequests []*SegmentIDRequest `protobuf:"bytes,3,rep,name=segmentIDRequests,proto3" json:"segmentIDRequests,omitempty"` +} + +func (x *AssignSegmentIDRequest) Reset() { + *x = AssignSegmentIDRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AssignSegmentIDRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AssignSegmentIDRequest) ProtoMessage() {} + +func (x *AssignSegmentIDRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[7] + 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 AssignSegmentIDRequest.ProtoReflect.Descriptor instead. +func (*AssignSegmentIDRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{7} +} + +func (x *AssignSegmentIDRequest) GetNodeID() int64 { + if x != nil { + return x.NodeID + } + return 0 +} + +func (x *AssignSegmentIDRequest) GetPeerRole() string { + if x != nil { + return x.PeerRole + } + return "" +} + +func (x *AssignSegmentIDRequest) GetSegmentIDRequests() []*SegmentIDRequest { + if x != nil { + return x.SegmentIDRequests + } + return nil +} + +type SegmentIDAssignment struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SegID int64 `protobuf:"varint,1,opt,name=segID,proto3" json:"segID,omitempty"` + ChannelName string `protobuf:"bytes,2,opt,name=channel_name,json=channelName,proto3" json:"channel_name,omitempty"` + Count uint32 `protobuf:"varint,3,opt,name=count,proto3" json:"count,omitempty"` + CollectionID int64 `protobuf:"varint,4,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionID int64 `protobuf:"varint,5,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + ExpireTime uint64 `protobuf:"varint,6,opt,name=expire_time,json=expireTime,proto3" json:"expire_time,omitempty"` + Status *commonpb.Status `protobuf:"bytes,7,opt,name=status,proto3" json:"status,omitempty"` +} + +func (x *SegmentIDAssignment) Reset() { + *x = SegmentIDAssignment{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentIDAssignment) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentIDAssignment) ProtoMessage() {} + +func (x *SegmentIDAssignment) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[8] + 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 SegmentIDAssignment.ProtoReflect.Descriptor instead. +func (*SegmentIDAssignment) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{8} +} + +func (x *SegmentIDAssignment) GetSegID() int64 { + if x != nil { + return x.SegID + } + return 0 +} + +func (x *SegmentIDAssignment) GetChannelName() string { + if x != nil { + return x.ChannelName + } + return "" +} + +func (x *SegmentIDAssignment) GetCount() uint32 { + if x != nil { + return x.Count + } + return 0 +} + +func (x *SegmentIDAssignment) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *SegmentIDAssignment) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *SegmentIDAssignment) GetExpireTime() uint64 { + if x != nil { + return x.ExpireTime + } + return 0 +} + +func (x *SegmentIDAssignment) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +type AssignSegmentIDResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SegIDAssignments []*SegmentIDAssignment `protobuf:"bytes,1,rep,name=segIDAssignments,proto3" json:"segIDAssignments,omitempty"` + Status *commonpb.Status `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` +} + +func (x *AssignSegmentIDResponse) Reset() { + *x = AssignSegmentIDResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AssignSegmentIDResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AssignSegmentIDResponse) ProtoMessage() {} + +func (x *AssignSegmentIDResponse) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[9] + 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 AssignSegmentIDResponse.ProtoReflect.Descriptor instead. +func (*AssignSegmentIDResponse) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{9} +} + +func (x *AssignSegmentIDResponse) GetSegIDAssignments() []*SegmentIDAssignment { + if x != nil { + return x.SegIDAssignments + } + return nil +} + +func (x *AssignSegmentIDResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +type GetSegmentStatesRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + SegmentIDs []int64 `protobuf:"varint,2,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,omitempty"` +} + +func (x *GetSegmentStatesRequest) Reset() { + *x = GetSegmentStatesRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetSegmentStatesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetSegmentStatesRequest) ProtoMessage() {} + +func (x *GetSegmentStatesRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[10] + 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 GetSegmentStatesRequest.ProtoReflect.Descriptor instead. +func (*GetSegmentStatesRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{10} +} + +func (x *GetSegmentStatesRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *GetSegmentStatesRequest) GetSegmentIDs() []int64 { + if x != nil { + return x.SegmentIDs + } + return nil +} + +type SegmentStateInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SegmentID int64 `protobuf:"varint,1,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + State commonpb.SegmentState `protobuf:"varint,2,opt,name=state,proto3,enum=milvus.proto.common.SegmentState" json:"state,omitempty"` + StartPosition *msgpb.MsgPosition `protobuf:"bytes,3,opt,name=start_position,json=startPosition,proto3" json:"start_position,omitempty"` + EndPosition *msgpb.MsgPosition `protobuf:"bytes,4,opt,name=end_position,json=endPosition,proto3" json:"end_position,omitempty"` + Status *commonpb.Status `protobuf:"bytes,5,opt,name=status,proto3" json:"status,omitempty"` +} + +func (x *SegmentStateInfo) Reset() { + *x = SegmentStateInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentStateInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentStateInfo) ProtoMessage() {} + +func (x *SegmentStateInfo) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[11] + 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 SegmentStateInfo.ProtoReflect.Descriptor instead. +func (*SegmentStateInfo) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{11} +} + +func (x *SegmentStateInfo) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *SegmentStateInfo) GetState() commonpb.SegmentState { + if x != nil { + return x.State + } + return commonpb.SegmentState(0) +} + +func (x *SegmentStateInfo) GetStartPosition() *msgpb.MsgPosition { + if x != nil { + return x.StartPosition + } + return nil +} + +func (x *SegmentStateInfo) GetEndPosition() *msgpb.MsgPosition { + if x != nil { + return x.EndPosition + } + return nil +} + +func (x *SegmentStateInfo) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +type GetSegmentStatesResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + States []*SegmentStateInfo `protobuf:"bytes,2,rep,name=states,proto3" json:"states,omitempty"` +} + +func (x *GetSegmentStatesResponse) Reset() { + *x = GetSegmentStatesResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetSegmentStatesResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetSegmentStatesResponse) ProtoMessage() {} + +func (x *GetSegmentStatesResponse) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[12] + 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 GetSegmentStatesResponse.ProtoReflect.Descriptor instead. +func (*GetSegmentStatesResponse) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{12} +} + +func (x *GetSegmentStatesResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *GetSegmentStatesResponse) GetStates() []*SegmentStateInfo { + if x != nil { + return x.States + } + return nil +} + +type GetSegmentInfoRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + SegmentIDs []int64 `protobuf:"varint,2,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,omitempty"` + IncludeUnHealthy bool `protobuf:"varint,3,opt,name=includeUnHealthy,proto3" json:"includeUnHealthy,omitempty"` +} + +func (x *GetSegmentInfoRequest) Reset() { + *x = GetSegmentInfoRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetSegmentInfoRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetSegmentInfoRequest) ProtoMessage() {} + +func (x *GetSegmentInfoRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[13] + 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 GetSegmentInfoRequest.ProtoReflect.Descriptor instead. +func (*GetSegmentInfoRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{13} +} + +func (x *GetSegmentInfoRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *GetSegmentInfoRequest) GetSegmentIDs() []int64 { + if x != nil { + return x.SegmentIDs + } + return nil +} + +func (x *GetSegmentInfoRequest) GetIncludeUnHealthy() bool { + if x != nil { + return x.IncludeUnHealthy + } + return false +} + +type GetSegmentInfoResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + Infos []*SegmentInfo `protobuf:"bytes,2,rep,name=infos,proto3" json:"infos,omitempty"` + ChannelCheckpoint map[string]*msgpb.MsgPosition `protobuf:"bytes,3,rep,name=channel_checkpoint,json=channelCheckpoint,proto3" json:"channel_checkpoint,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *GetSegmentInfoResponse) Reset() { + *x = GetSegmentInfoResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetSegmentInfoResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetSegmentInfoResponse) ProtoMessage() {} + +func (x *GetSegmentInfoResponse) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[14] + 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 GetSegmentInfoResponse.ProtoReflect.Descriptor instead. +func (*GetSegmentInfoResponse) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{14} +} + +func (x *GetSegmentInfoResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *GetSegmentInfoResponse) GetInfos() []*SegmentInfo { + if x != nil { + return x.Infos + } + return nil +} + +func (x *GetSegmentInfoResponse) GetChannelCheckpoint() map[string]*msgpb.MsgPosition { + if x != nil { + return x.ChannelCheckpoint + } + return nil +} + +type GetInsertBinlogPathsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + SegmentID int64 `protobuf:"varint,2,opt,name=segmentID,proto3" json:"segmentID,omitempty"` +} + +func (x *GetInsertBinlogPathsRequest) Reset() { + *x = GetInsertBinlogPathsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetInsertBinlogPathsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetInsertBinlogPathsRequest) ProtoMessage() {} + +func (x *GetInsertBinlogPathsRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[15] + 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 GetInsertBinlogPathsRequest.ProtoReflect.Descriptor instead. +func (*GetInsertBinlogPathsRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{15} +} + +func (x *GetInsertBinlogPathsRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *GetInsertBinlogPathsRequest) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +type GetInsertBinlogPathsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + FieldIDs []int64 `protobuf:"varint,1,rep,packed,name=fieldIDs,proto3" json:"fieldIDs,omitempty"` + Paths []*internalpb.StringList `protobuf:"bytes,2,rep,name=paths,proto3" json:"paths,omitempty"` + Status *commonpb.Status `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"` +} + +func (x *GetInsertBinlogPathsResponse) Reset() { + *x = GetInsertBinlogPathsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetInsertBinlogPathsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetInsertBinlogPathsResponse) ProtoMessage() {} + +func (x *GetInsertBinlogPathsResponse) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[16] + 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 GetInsertBinlogPathsResponse.ProtoReflect.Descriptor instead. +func (*GetInsertBinlogPathsResponse) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{16} +} + +func (x *GetInsertBinlogPathsResponse) GetFieldIDs() []int64 { + if x != nil { + return x.FieldIDs + } + return nil +} + +func (x *GetInsertBinlogPathsResponse) GetPaths() []*internalpb.StringList { + if x != nil { + return x.Paths + } + return nil +} + +func (x *GetInsertBinlogPathsResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +type GetCollectionStatisticsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + DbID int64 `protobuf:"varint,2,opt,name=dbID,proto3" json:"dbID,omitempty"` + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` +} + +func (x *GetCollectionStatisticsRequest) Reset() { + *x = GetCollectionStatisticsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetCollectionStatisticsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetCollectionStatisticsRequest) ProtoMessage() {} + +func (x *GetCollectionStatisticsRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[17] + 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 GetCollectionStatisticsRequest.ProtoReflect.Descriptor instead. +func (*GetCollectionStatisticsRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{17} +} + +func (x *GetCollectionStatisticsRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *GetCollectionStatisticsRequest) GetDbID() int64 { + if x != nil { + return x.DbID + } + return 0 +} + +func (x *GetCollectionStatisticsRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +type GetCollectionStatisticsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Stats []*commonpb.KeyValuePair `protobuf:"bytes,1,rep,name=stats,proto3" json:"stats,omitempty"` + Status *commonpb.Status `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` +} + +func (x *GetCollectionStatisticsResponse) Reset() { + *x = GetCollectionStatisticsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetCollectionStatisticsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetCollectionStatisticsResponse) ProtoMessage() {} + +func (x *GetCollectionStatisticsResponse) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[18] + 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 GetCollectionStatisticsResponse.ProtoReflect.Descriptor instead. +func (*GetCollectionStatisticsResponse) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{18} +} + +func (x *GetCollectionStatisticsResponse) GetStats() []*commonpb.KeyValuePair { + if x != nil { + return x.Stats + } + return nil +} + +func (x *GetCollectionStatisticsResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +type GetPartitionStatisticsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + DbID int64 `protobuf:"varint,2,opt,name=dbID,proto3" json:"dbID,omitempty"` + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionIDs []int64 `protobuf:"varint,4,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` +} + +func (x *GetPartitionStatisticsRequest) Reset() { + *x = GetPartitionStatisticsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[19] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetPartitionStatisticsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetPartitionStatisticsRequest) ProtoMessage() {} + +func (x *GetPartitionStatisticsRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[19] + 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 GetPartitionStatisticsRequest.ProtoReflect.Descriptor instead. +func (*GetPartitionStatisticsRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{19} +} + +func (x *GetPartitionStatisticsRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *GetPartitionStatisticsRequest) GetDbID() int64 { + if x != nil { + return x.DbID + } + return 0 +} + +func (x *GetPartitionStatisticsRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *GetPartitionStatisticsRequest) GetPartitionIDs() []int64 { + if x != nil { + return x.PartitionIDs + } + return nil +} + +type GetPartitionStatisticsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Stats []*commonpb.KeyValuePair `protobuf:"bytes,1,rep,name=stats,proto3" json:"stats,omitempty"` + Status *commonpb.Status `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` +} + +func (x *GetPartitionStatisticsResponse) Reset() { + *x = GetPartitionStatisticsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[20] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetPartitionStatisticsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetPartitionStatisticsResponse) ProtoMessage() {} + +func (x *GetPartitionStatisticsResponse) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[20] + 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 GetPartitionStatisticsResponse.ProtoReflect.Descriptor instead. +func (*GetPartitionStatisticsResponse) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{20} +} + +func (x *GetPartitionStatisticsResponse) GetStats() []*commonpb.KeyValuePair { + if x != nil { + return x.Stats + } + return nil +} + +func (x *GetPartitionStatisticsResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +type GetSegmentInfoChannelRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *GetSegmentInfoChannelRequest) Reset() { + *x = GetSegmentInfoChannelRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[21] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetSegmentInfoChannelRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetSegmentInfoChannelRequest) ProtoMessage() {} + +func (x *GetSegmentInfoChannelRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[21] + 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 GetSegmentInfoChannelRequest.ProtoReflect.Descriptor instead. +func (*GetSegmentInfoChannelRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{21} +} + +type VchannelInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + ChannelName string `protobuf:"bytes,2,opt,name=channelName,proto3" json:"channelName,omitempty"` + SeekPosition *msgpb.MsgPosition `protobuf:"bytes,3,opt,name=seek_position,json=seekPosition,proto3" json:"seek_position,omitempty"` + UnflushedSegments []*SegmentInfo `protobuf:"bytes,4,rep,name=unflushedSegments,proto3" json:"unflushedSegments,omitempty"` // deprecated, keep it for compatibility + FlushedSegments []*SegmentInfo `protobuf:"bytes,5,rep,name=flushedSegments,proto3" json:"flushedSegments,omitempty"` // deprecated, keep it for compatibility + DroppedSegments []*SegmentInfo `protobuf:"bytes,6,rep,name=dropped_segments,json=droppedSegments,proto3" json:"dropped_segments,omitempty"` // deprecated, keep it for compatibility + UnflushedSegmentIds []int64 `protobuf:"varint,7,rep,packed,name=unflushedSegmentIds,proto3" json:"unflushedSegmentIds,omitempty"` + FlushedSegmentIds []int64 `protobuf:"varint,8,rep,packed,name=flushedSegmentIds,proto3" json:"flushedSegmentIds,omitempty"` + DroppedSegmentIds []int64 `protobuf:"varint,9,rep,packed,name=dropped_segmentIds,json=droppedSegmentIds,proto3" json:"dropped_segmentIds,omitempty"` + IndexedSegmentIds []int64 `protobuf:"varint,10,rep,packed,name=indexed_segmentIds,json=indexedSegmentIds,proto3" json:"indexed_segmentIds,omitempty"` // deprecated, keep it for compatibility + IndexedSegments []*SegmentInfo `protobuf:"bytes,11,rep,name=indexed_segments,json=indexedSegments,proto3" json:"indexed_segments,omitempty"` // deprecated, keep it for compatibility + LevelZeroSegmentIds []int64 `protobuf:"varint,12,rep,packed,name=level_zero_segment_ids,json=levelZeroSegmentIds,proto3" json:"level_zero_segment_ids,omitempty"` + PartitionStatsVersions map[int64]int64 `protobuf:"bytes,13,rep,name=partition_stats_versions,json=partitionStatsVersions,proto3" json:"partition_stats_versions,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` +} + +func (x *VchannelInfo) Reset() { + *x = VchannelInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[22] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *VchannelInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*VchannelInfo) ProtoMessage() {} + +func (x *VchannelInfo) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[22] + 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 VchannelInfo.ProtoReflect.Descriptor instead. +func (*VchannelInfo) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{22} +} + +func (x *VchannelInfo) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *VchannelInfo) GetChannelName() string { + if x != nil { + return x.ChannelName + } + return "" +} + +func (x *VchannelInfo) GetSeekPosition() *msgpb.MsgPosition { + if x != nil { + return x.SeekPosition + } + return nil +} + +func (x *VchannelInfo) GetUnflushedSegments() []*SegmentInfo { + if x != nil { + return x.UnflushedSegments + } + return nil +} + +func (x *VchannelInfo) GetFlushedSegments() []*SegmentInfo { + if x != nil { + return x.FlushedSegments + } + return nil +} + +func (x *VchannelInfo) GetDroppedSegments() []*SegmentInfo { + if x != nil { + return x.DroppedSegments + } + return nil +} + +func (x *VchannelInfo) GetUnflushedSegmentIds() []int64 { + if x != nil { + return x.UnflushedSegmentIds + } + return nil +} + +func (x *VchannelInfo) GetFlushedSegmentIds() []int64 { + if x != nil { + return x.FlushedSegmentIds + } + return nil +} + +func (x *VchannelInfo) GetDroppedSegmentIds() []int64 { + if x != nil { + return x.DroppedSegmentIds + } + return nil +} + +func (x *VchannelInfo) GetIndexedSegmentIds() []int64 { + if x != nil { + return x.IndexedSegmentIds + } + return nil +} + +func (x *VchannelInfo) GetIndexedSegments() []*SegmentInfo { + if x != nil { + return x.IndexedSegments + } + return nil +} + +func (x *VchannelInfo) GetLevelZeroSegmentIds() []int64 { + if x != nil { + return x.LevelZeroSegmentIds + } + return nil +} + +func (x *VchannelInfo) GetPartitionStatsVersions() map[int64]int64 { + if x != nil { + return x.PartitionStatsVersions + } + return nil +} + +type WatchDmChannelsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + Vchannels []*VchannelInfo `protobuf:"bytes,2,rep,name=vchannels,proto3" json:"vchannels,omitempty"` +} + +func (x *WatchDmChannelsRequest) Reset() { + *x = WatchDmChannelsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[23] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WatchDmChannelsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WatchDmChannelsRequest) ProtoMessage() {} + +func (x *WatchDmChannelsRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[23] + 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 WatchDmChannelsRequest.ProtoReflect.Descriptor instead. +func (*WatchDmChannelsRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{23} +} + +func (x *WatchDmChannelsRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *WatchDmChannelsRequest) GetVchannels() []*VchannelInfo { + if x != nil { + return x.Vchannels + } + return nil +} + +type FlushSegmentsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + DbID int64 `protobuf:"varint,2,opt,name=dbID,proto3" json:"dbID,omitempty"` + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + SegmentIDs []int64 `protobuf:"varint,4,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,omitempty"` // segments to flush + ChannelName string `protobuf:"bytes,5,opt,name=channelName,proto3" json:"channelName,omitempty"` // vchannel name to flush +} + +func (x *FlushSegmentsRequest) Reset() { + *x = FlushSegmentsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[24] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *FlushSegmentsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FlushSegmentsRequest) ProtoMessage() {} + +func (x *FlushSegmentsRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[24] + 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 FlushSegmentsRequest.ProtoReflect.Descriptor instead. +func (*FlushSegmentsRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{24} +} + +func (x *FlushSegmentsRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *FlushSegmentsRequest) GetDbID() int64 { + if x != nil { + return x.DbID + } + return 0 +} + +func (x *FlushSegmentsRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *FlushSegmentsRequest) GetSegmentIDs() []int64 { + if x != nil { + return x.SegmentIDs + } + return nil +} + +func (x *FlushSegmentsRequest) GetChannelName() string { + if x != nil { + return x.ChannelName + } + return "" +} + +type SegmentMsg struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + Segment *SegmentInfo `protobuf:"bytes,2,opt,name=segment,proto3" json:"segment,omitempty"` +} + +func (x *SegmentMsg) Reset() { + *x = SegmentMsg{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[25] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentMsg) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentMsg) ProtoMessage() {} + +func (x *SegmentMsg) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[25] + 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 SegmentMsg.ProtoReflect.Descriptor instead. +func (*SegmentMsg) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{25} +} + +func (x *SegmentMsg) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *SegmentMsg) GetSegment() *SegmentInfo { + if x != nil { + return x.Segment + } + return nil +} + +type SegmentInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ID int64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"` + CollectionID int64 `protobuf:"varint,2,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionID int64 `protobuf:"varint,3,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + InsertChannel string `protobuf:"bytes,4,opt,name=insert_channel,json=insertChannel,proto3" json:"insert_channel,omitempty"` + NumOfRows int64 `protobuf:"varint,5,opt,name=num_of_rows,json=numOfRows,proto3" json:"num_of_rows,omitempty"` + State commonpb.SegmentState `protobuf:"varint,6,opt,name=state,proto3,enum=milvus.proto.common.SegmentState" json:"state,omitempty"` + MaxRowNum int64 `protobuf:"varint,7,opt,name=max_row_num,json=maxRowNum,proto3" json:"max_row_num,omitempty"` + LastExpireTime uint64 `protobuf:"varint,8,opt,name=last_expire_time,json=lastExpireTime,proto3" json:"last_expire_time,omitempty"` + StartPosition *msgpb.MsgPosition `protobuf:"bytes,9,opt,name=start_position,json=startPosition,proto3" json:"start_position,omitempty"` + DmlPosition *msgpb.MsgPosition `protobuf:"bytes,10,opt,name=dml_position,json=dmlPosition,proto3" json:"dml_position,omitempty"` + // binlogs consist of insert binlogs + Binlogs []*FieldBinlog `protobuf:"bytes,11,rep,name=binlogs,proto3" json:"binlogs,omitempty"` + Statslogs []*FieldBinlog `protobuf:"bytes,12,rep,name=statslogs,proto3" json:"statslogs,omitempty"` + // deltalogs consists of delete binlogs. FieldID is not used yet since delete is always applied on primary key + Deltalogs []*FieldBinlog `protobuf:"bytes,13,rep,name=deltalogs,proto3" json:"deltalogs,omitempty"` + CreatedByCompaction bool `protobuf:"varint,14,opt,name=createdByCompaction,proto3" json:"createdByCompaction,omitempty"` + CompactionFrom []int64 `protobuf:"varint,15,rep,packed,name=compactionFrom,proto3" json:"compactionFrom,omitempty"` + DroppedAt uint64 `protobuf:"varint,16,opt,name=dropped_at,json=droppedAt,proto3" json:"dropped_at,omitempty"` // timestamp when segment marked drop + // A flag indicating if: + // (1) this segment is created by bulk insert, and + // (2) the bulk insert task that creates this segment has not yet reached `ImportCompleted` state. + IsImporting bool `protobuf:"varint,17,opt,name=is_importing,json=isImporting,proto3" json:"is_importing,omitempty"` + IsFake bool `protobuf:"varint,18,opt,name=is_fake,json=isFake,proto3" json:"is_fake,omitempty"` + // denote if this segment is compacted to other segment. + // For compatibility reasons, this flag of an old compacted segment may still be False. + // As for new fields added in the message, they will be populated with their respective field types' default values. + Compacted bool `protobuf:"varint,19,opt,name=compacted,proto3" json:"compacted,omitempty"` + // Segment level, indicating compaction segment level + // Available value: Legacy, L0, L1, L2 + // For legacy level, it represent old segment before segment level introduced + // so segments with Legacy level shall be treated as L1 segment + Level SegmentLevel `protobuf:"varint,20,opt,name=level,proto3,enum=milvus.proto.data.SegmentLevel" json:"level,omitempty"` + StorageVersion int64 `protobuf:"varint,21,opt,name=storage_version,json=storageVersion,proto3" json:"storage_version,omitempty"` + PartitionStatsVersion int64 `protobuf:"varint,22,opt,name=partition_stats_version,json=partitionStatsVersion,proto3" json:"partition_stats_version,omitempty"` + // use in major compaction, if compaction fail, should revert segment level to last value + LastLevel SegmentLevel `protobuf:"varint,23,opt,name=last_level,json=lastLevel,proto3,enum=milvus.proto.data.SegmentLevel" json:"last_level,omitempty"` + // use in major compaction, if compaction fail, should revert partition stats version to last value + LastPartitionStatsVersion int64 `protobuf:"varint,24,opt,name=last_partition_stats_version,json=lastPartitionStatsVersion,proto3" json:"last_partition_stats_version,omitempty"` + // used to indicate whether the segment is sorted by primary key. + IsSorted bool `protobuf:"varint,25,opt,name=is_sorted,json=isSorted,proto3" json:"is_sorted,omitempty"` + // textStatsLogs is used to record tokenization index for fields. + TextStatsLogs map[int64]*TextIndexStats `protobuf:"bytes,26,rep,name=textStatsLogs,proto3" json:"textStatsLogs,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Bm25Statslogs []*FieldBinlog `protobuf:"bytes,27,rep,name=bm25statslogs,proto3" json:"bm25statslogs,omitempty"` + // This field is used to indicate that some intermediate state segments should not be loaded. + // For example, segments that have been clustered but haven't undergone stats yet. + IsInvisible bool `protobuf:"varint,28,opt,name=is_invisible,json=isInvisible,proto3" json:"is_invisible,omitempty"` +} + +func (x *SegmentInfo) Reset() { + *x = SegmentInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[26] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentInfo) ProtoMessage() {} + +func (x *SegmentInfo) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[26] + 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 SegmentInfo.ProtoReflect.Descriptor instead. +func (*SegmentInfo) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{26} +} + +func (x *SegmentInfo) GetID() int64 { + if x != nil { + return x.ID + } + return 0 +} + +func (x *SegmentInfo) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *SegmentInfo) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *SegmentInfo) GetInsertChannel() string { + if x != nil { + return x.InsertChannel + } + return "" +} + +func (x *SegmentInfo) GetNumOfRows() int64 { + if x != nil { + return x.NumOfRows + } + return 0 +} + +func (x *SegmentInfo) GetState() commonpb.SegmentState { + if x != nil { + return x.State + } + return commonpb.SegmentState(0) +} + +func (x *SegmentInfo) GetMaxRowNum() int64 { + if x != nil { + return x.MaxRowNum + } + return 0 +} + +func (x *SegmentInfo) GetLastExpireTime() uint64 { + if x != nil { + return x.LastExpireTime + } + return 0 +} + +func (x *SegmentInfo) GetStartPosition() *msgpb.MsgPosition { + if x != nil { + return x.StartPosition + } + return nil +} + +func (x *SegmentInfo) GetDmlPosition() *msgpb.MsgPosition { + if x != nil { + return x.DmlPosition + } + return nil +} + +func (x *SegmentInfo) GetBinlogs() []*FieldBinlog { + if x != nil { + return x.Binlogs + } + return nil +} + +func (x *SegmentInfo) GetStatslogs() []*FieldBinlog { + if x != nil { + return x.Statslogs + } + return nil +} + +func (x *SegmentInfo) GetDeltalogs() []*FieldBinlog { + if x != nil { + return x.Deltalogs + } + return nil +} + +func (x *SegmentInfo) GetCreatedByCompaction() bool { + if x != nil { + return x.CreatedByCompaction + } + return false +} + +func (x *SegmentInfo) GetCompactionFrom() []int64 { + if x != nil { + return x.CompactionFrom + } + return nil +} + +func (x *SegmentInfo) GetDroppedAt() uint64 { + if x != nil { + return x.DroppedAt + } + return 0 +} + +func (x *SegmentInfo) GetIsImporting() bool { + if x != nil { + return x.IsImporting + } + return false +} + +func (x *SegmentInfo) GetIsFake() bool { + if x != nil { + return x.IsFake + } + return false +} + +func (x *SegmentInfo) GetCompacted() bool { + if x != nil { + return x.Compacted + } + return false +} + +func (x *SegmentInfo) GetLevel() SegmentLevel { + if x != nil { + return x.Level + } + return SegmentLevel_Legacy +} + +func (x *SegmentInfo) GetStorageVersion() int64 { + if x != nil { + return x.StorageVersion + } + return 0 +} + +func (x *SegmentInfo) GetPartitionStatsVersion() int64 { + if x != nil { + return x.PartitionStatsVersion + } + return 0 +} + +func (x *SegmentInfo) GetLastLevel() SegmentLevel { + if x != nil { + return x.LastLevel + } + return SegmentLevel_Legacy +} + +func (x *SegmentInfo) GetLastPartitionStatsVersion() int64 { + if x != nil { + return x.LastPartitionStatsVersion + } + return 0 +} + +func (x *SegmentInfo) GetIsSorted() bool { + if x != nil { + return x.IsSorted + } + return false +} + +func (x *SegmentInfo) GetTextStatsLogs() map[int64]*TextIndexStats { + if x != nil { + return x.TextStatsLogs + } + return nil +} + +func (x *SegmentInfo) GetBm25Statslogs() []*FieldBinlog { + if x != nil { + return x.Bm25Statslogs + } + return nil +} + +func (x *SegmentInfo) GetIsInvisible() bool { + if x != nil { + return x.IsInvisible + } + return false +} + +type SegmentStartPosition struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + StartPosition *msgpb.MsgPosition `protobuf:"bytes,1,opt,name=start_position,json=startPosition,proto3" json:"start_position,omitempty"` + SegmentID int64 `protobuf:"varint,2,opt,name=segmentID,proto3" json:"segmentID,omitempty"` +} + +func (x *SegmentStartPosition) Reset() { + *x = SegmentStartPosition{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[27] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentStartPosition) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentStartPosition) ProtoMessage() {} + +func (x *SegmentStartPosition) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[27] + 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 SegmentStartPosition.ProtoReflect.Descriptor instead. +func (*SegmentStartPosition) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{27} +} + +func (x *SegmentStartPosition) GetStartPosition() *msgpb.MsgPosition { + if x != nil { + return x.StartPosition + } + return nil +} + +func (x *SegmentStartPosition) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +type SaveBinlogPathsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + SegmentID int64 `protobuf:"varint,2,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + Field2BinlogPaths []*FieldBinlog `protobuf:"bytes,4,rep,name=field2BinlogPaths,proto3" json:"field2BinlogPaths,omitempty"` + CheckPoints []*CheckPoint `protobuf:"bytes,5,rep,name=checkPoints,proto3" json:"checkPoints,omitempty"` + StartPositions []*SegmentStartPosition `protobuf:"bytes,6,rep,name=start_positions,json=startPositions,proto3" json:"start_positions,omitempty"` + Flushed bool `protobuf:"varint,7,opt,name=flushed,proto3" json:"flushed,omitempty"` + Field2StatslogPaths []*FieldBinlog `protobuf:"bytes,8,rep,name=field2StatslogPaths,proto3" json:"field2StatslogPaths,omitempty"` + Deltalogs []*FieldBinlog `protobuf:"bytes,9,rep,name=deltalogs,proto3" json:"deltalogs,omitempty"` + Dropped bool `protobuf:"varint,10,opt,name=dropped,proto3" json:"dropped,omitempty"` + Importing bool `protobuf:"varint,11,opt,name=importing,proto3" json:"importing,omitempty"` // deprecated + Channel string `protobuf:"bytes,12,opt,name=channel,proto3" json:"channel,omitempty"` // report channel name for verification + SegLevel SegmentLevel `protobuf:"varint,13,opt,name=seg_level,json=segLevel,proto3,enum=milvus.proto.data.SegmentLevel" json:"seg_level,omitempty"` + PartitionID int64 `protobuf:"varint,14,opt,name=partitionID,proto3" json:"partitionID,omitempty"` // report partitionID for create L0 segment + StorageVersion int64 `protobuf:"varint,15,opt,name=storageVersion,proto3" json:"storageVersion,omitempty"` + Field2Bm25LogPaths []*FieldBinlog `protobuf:"bytes,16,rep,name=field2Bm25logPaths,proto3" json:"field2Bm25logPaths,omitempty"` +} + +func (x *SaveBinlogPathsRequest) Reset() { + *x = SaveBinlogPathsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[28] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SaveBinlogPathsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SaveBinlogPathsRequest) ProtoMessage() {} + +func (x *SaveBinlogPathsRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_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 SaveBinlogPathsRequest.ProtoReflect.Descriptor instead. +func (*SaveBinlogPathsRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{28} +} + +func (x *SaveBinlogPathsRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *SaveBinlogPathsRequest) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *SaveBinlogPathsRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *SaveBinlogPathsRequest) GetField2BinlogPaths() []*FieldBinlog { + if x != nil { + return x.Field2BinlogPaths + } + return nil +} + +func (x *SaveBinlogPathsRequest) GetCheckPoints() []*CheckPoint { + if x != nil { + return x.CheckPoints + } + return nil +} + +func (x *SaveBinlogPathsRequest) GetStartPositions() []*SegmentStartPosition { + if x != nil { + return x.StartPositions + } + return nil +} + +func (x *SaveBinlogPathsRequest) GetFlushed() bool { + if x != nil { + return x.Flushed + } + return false +} + +func (x *SaveBinlogPathsRequest) GetField2StatslogPaths() []*FieldBinlog { + if x != nil { + return x.Field2StatslogPaths + } + return nil +} + +func (x *SaveBinlogPathsRequest) GetDeltalogs() []*FieldBinlog { + if x != nil { + return x.Deltalogs + } + return nil +} + +func (x *SaveBinlogPathsRequest) GetDropped() bool { + if x != nil { + return x.Dropped + } + return false +} + +func (x *SaveBinlogPathsRequest) GetImporting() bool { + if x != nil { + return x.Importing + } + return false +} + +func (x *SaveBinlogPathsRequest) GetChannel() string { + if x != nil { + return x.Channel + } + return "" +} + +func (x *SaveBinlogPathsRequest) GetSegLevel() SegmentLevel { + if x != nil { + return x.SegLevel + } + return SegmentLevel_Legacy +} + +func (x *SaveBinlogPathsRequest) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *SaveBinlogPathsRequest) GetStorageVersion() int64 { + if x != nil { + return x.StorageVersion + } + return 0 +} + +func (x *SaveBinlogPathsRequest) GetField2Bm25LogPaths() []*FieldBinlog { + if x != nil { + return x.Field2Bm25LogPaths + } + return nil +} + +type CheckPoint struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SegmentID int64 `protobuf:"varint,1,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + Position *msgpb.MsgPosition `protobuf:"bytes,2,opt,name=position,proto3" json:"position,omitempty"` + NumOfRows int64 `protobuf:"varint,3,opt,name=num_of_rows,json=numOfRows,proto3" json:"num_of_rows,omitempty"` +} + +func (x *CheckPoint) Reset() { + *x = CheckPoint{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[29] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CheckPoint) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CheckPoint) ProtoMessage() {} + +func (x *CheckPoint) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_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 CheckPoint.ProtoReflect.Descriptor instead. +func (*CheckPoint) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{29} +} + +func (x *CheckPoint) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *CheckPoint) GetPosition() *msgpb.MsgPosition { + if x != nil { + return x.Position + } + return nil +} + +func (x *CheckPoint) GetNumOfRows() int64 { + if x != nil { + return x.NumOfRows + } + return 0 +} + +type DeltaLogInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + RecordEntries uint64 `protobuf:"varint,1,opt,name=record_entries,json=recordEntries,proto3" json:"record_entries,omitempty"` + TimestampFrom uint64 `protobuf:"varint,2,opt,name=timestamp_from,json=timestampFrom,proto3" json:"timestamp_from,omitempty"` + TimestampTo uint64 `protobuf:"varint,3,opt,name=timestamp_to,json=timestampTo,proto3" json:"timestamp_to,omitempty"` + DeltaLogPath string `protobuf:"bytes,4,opt,name=delta_log_path,json=deltaLogPath,proto3" json:"delta_log_path,omitempty"` + DeltaLogSize int64 `protobuf:"varint,5,opt,name=delta_log_size,json=deltaLogSize,proto3" json:"delta_log_size,omitempty"` +} + +func (x *DeltaLogInfo) Reset() { + *x = DeltaLogInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[30] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeltaLogInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeltaLogInfo) ProtoMessage() {} + +func (x *DeltaLogInfo) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[30] + 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 DeltaLogInfo.ProtoReflect.Descriptor instead. +func (*DeltaLogInfo) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{30} +} + +func (x *DeltaLogInfo) GetRecordEntries() uint64 { + if x != nil { + return x.RecordEntries + } + return 0 +} + +func (x *DeltaLogInfo) GetTimestampFrom() uint64 { + if x != nil { + return x.TimestampFrom + } + return 0 +} + +func (x *DeltaLogInfo) GetTimestampTo() uint64 { + if x != nil { + return x.TimestampTo + } + return 0 +} + +func (x *DeltaLogInfo) GetDeltaLogPath() string { + if x != nil { + return x.DeltaLogPath + } + return "" +} + +func (x *DeltaLogInfo) GetDeltaLogSize() int64 { + if x != nil { + return x.DeltaLogSize + } + return 0 +} + +type ChannelStatus struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + State ChannelWatchState `protobuf:"varint,2,opt,name=state,proto3,enum=milvus.proto.data.ChannelWatchState" json:"state,omitempty"` + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` +} + +func (x *ChannelStatus) Reset() { + *x = ChannelStatus{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[31] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ChannelStatus) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ChannelStatus) ProtoMessage() {} + +func (x *ChannelStatus) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[31] + 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 ChannelStatus.ProtoReflect.Descriptor instead. +func (*ChannelStatus) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{31} +} + +func (x *ChannelStatus) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *ChannelStatus) GetState() ChannelWatchState { + if x != nil { + return x.State + } + return ChannelWatchState_Uncomplete +} + +func (x *ChannelStatus) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +type DataNodeInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Address string `protobuf:"bytes,1,opt,name=address,proto3" json:"address,omitempty"` + Version int64 `protobuf:"varint,2,opt,name=version,proto3" json:"version,omitempty"` + Channels []*ChannelStatus `protobuf:"bytes,3,rep,name=channels,proto3" json:"channels,omitempty"` +} + +func (x *DataNodeInfo) Reset() { + *x = DataNodeInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[32] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DataNodeInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DataNodeInfo) ProtoMessage() {} + +func (x *DataNodeInfo) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[32] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DataNodeInfo.ProtoReflect.Descriptor instead. +func (*DataNodeInfo) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{32} +} + +func (x *DataNodeInfo) GetAddress() string { + if x != nil { + return x.Address + } + return "" +} + +func (x *DataNodeInfo) GetVersion() int64 { + if x != nil { + return x.Version + } + return 0 +} + +func (x *DataNodeInfo) GetChannels() []*ChannelStatus { + if x != nil { + return x.Channels + } + return nil +} + +type SegmentBinlogs struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SegmentID int64 `protobuf:"varint,1,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + FieldBinlogs []*FieldBinlog `protobuf:"bytes,2,rep,name=fieldBinlogs,proto3" json:"fieldBinlogs,omitempty"` + NumOfRows int64 `protobuf:"varint,3,opt,name=num_of_rows,json=numOfRows,proto3" json:"num_of_rows,omitempty"` + Statslogs []*FieldBinlog `protobuf:"bytes,4,rep,name=statslogs,proto3" json:"statslogs,omitempty"` + Deltalogs []*FieldBinlog `protobuf:"bytes,5,rep,name=deltalogs,proto3" json:"deltalogs,omitempty"` + InsertChannel string `protobuf:"bytes,6,opt,name=insert_channel,json=insertChannel,proto3" json:"insert_channel,omitempty"` + TextStatsLogs map[int64]*TextIndexStats `protobuf:"bytes,7,rep,name=textStatsLogs,proto3" json:"textStatsLogs,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *SegmentBinlogs) Reset() { + *x = SegmentBinlogs{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[33] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentBinlogs) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentBinlogs) ProtoMessage() {} + +func (x *SegmentBinlogs) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[33] + 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 SegmentBinlogs.ProtoReflect.Descriptor instead. +func (*SegmentBinlogs) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{33} +} + +func (x *SegmentBinlogs) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *SegmentBinlogs) GetFieldBinlogs() []*FieldBinlog { + if x != nil { + return x.FieldBinlogs + } + return nil +} + +func (x *SegmentBinlogs) GetNumOfRows() int64 { + if x != nil { + return x.NumOfRows + } + return 0 +} + +func (x *SegmentBinlogs) GetStatslogs() []*FieldBinlog { + if x != nil { + return x.Statslogs + } + return nil +} + +func (x *SegmentBinlogs) GetDeltalogs() []*FieldBinlog { + if x != nil { + return x.Deltalogs + } + return nil +} + +func (x *SegmentBinlogs) GetInsertChannel() string { + if x != nil { + return x.InsertChannel + } + return "" +} + +func (x *SegmentBinlogs) GetTextStatsLogs() map[int64]*TextIndexStats { + if x != nil { + return x.TextStatsLogs + } + return nil +} + +type FieldBinlog struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + FieldID int64 `protobuf:"varint,1,opt,name=fieldID,proto3" json:"fieldID,omitempty"` + Binlogs []*Binlog `protobuf:"bytes,2,rep,name=binlogs,proto3" json:"binlogs,omitempty"` +} + +func (x *FieldBinlog) Reset() { + *x = FieldBinlog{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[34] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *FieldBinlog) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FieldBinlog) ProtoMessage() {} + +func (x *FieldBinlog) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[34] + 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 FieldBinlog.ProtoReflect.Descriptor instead. +func (*FieldBinlog) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{34} +} + +func (x *FieldBinlog) GetFieldID() int64 { + if x != nil { + return x.FieldID + } + return 0 +} + +func (x *FieldBinlog) GetBinlogs() []*Binlog { + if x != nil { + return x.Binlogs + } + return nil +} + +type TextIndexStats struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + FieldID int64 `protobuf:"varint,1,opt,name=fieldID,proto3" json:"fieldID,omitempty"` + Version int64 `protobuf:"varint,2,opt,name=version,proto3" json:"version,omitempty"` + Files []string `protobuf:"bytes,3,rep,name=files,proto3" json:"files,omitempty"` + LogSize int64 `protobuf:"varint,4,opt,name=log_size,json=logSize,proto3" json:"log_size,omitempty"` + MemorySize int64 `protobuf:"varint,5,opt,name=memory_size,json=memorySize,proto3" json:"memory_size,omitempty"` + BuildID int64 `protobuf:"varint,6,opt,name=buildID,proto3" json:"buildID,omitempty"` +} + +func (x *TextIndexStats) Reset() { + *x = TextIndexStats{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[35] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TextIndexStats) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TextIndexStats) ProtoMessage() {} + +func (x *TextIndexStats) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[35] + 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 TextIndexStats.ProtoReflect.Descriptor instead. +func (*TextIndexStats) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{35} +} + +func (x *TextIndexStats) GetFieldID() int64 { + if x != nil { + return x.FieldID + } + return 0 +} + +func (x *TextIndexStats) GetVersion() int64 { + if x != nil { + return x.Version + } + return 0 +} + +func (x *TextIndexStats) GetFiles() []string { + if x != nil { + return x.Files + } + return nil +} + +func (x *TextIndexStats) GetLogSize() int64 { + if x != nil { + return x.LogSize + } + return 0 +} + +func (x *TextIndexStats) GetMemorySize() int64 { + if x != nil { + return x.MemorySize + } + return 0 +} + +func (x *TextIndexStats) GetBuildID() int64 { + if x != nil { + return x.BuildID + } + return 0 +} + +type Binlog struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + EntriesNum int64 `protobuf:"varint,1,opt,name=entries_num,json=entriesNum,proto3" json:"entries_num,omitempty"` + TimestampFrom uint64 `protobuf:"varint,2,opt,name=timestamp_from,json=timestampFrom,proto3" json:"timestamp_from,omitempty"` + TimestampTo uint64 `protobuf:"varint,3,opt,name=timestamp_to,json=timestampTo,proto3" json:"timestamp_to,omitempty"` + // deprecated + LogPath string `protobuf:"bytes,4,opt,name=log_path,json=logPath,proto3" json:"log_path,omitempty"` + LogSize int64 `protobuf:"varint,5,opt,name=log_size,json=logSize,proto3" json:"log_size,omitempty"` + LogID int64 `protobuf:"varint,6,opt,name=logID,proto3" json:"logID,omitempty"` + // memory_size represents the size occupied by loading data into memory. + // log_size represents the size after data serialized. + // for stats_log, the memory_size always equal log_size. + MemorySize int64 `protobuf:"varint,7,opt,name=memory_size,json=memorySize,proto3" json:"memory_size,omitempty"` +} + +func (x *Binlog) Reset() { + *x = Binlog{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[36] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Binlog) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Binlog) ProtoMessage() {} + +func (x *Binlog) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[36] + 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 Binlog.ProtoReflect.Descriptor instead. +func (*Binlog) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{36} +} + +func (x *Binlog) GetEntriesNum() int64 { + if x != nil { + return x.EntriesNum + } + return 0 +} + +func (x *Binlog) GetTimestampFrom() uint64 { + if x != nil { + return x.TimestampFrom + } + return 0 +} + +func (x *Binlog) GetTimestampTo() uint64 { + if x != nil { + return x.TimestampTo + } + return 0 +} + +func (x *Binlog) GetLogPath() string { + if x != nil { + return x.LogPath + } + return "" +} + +func (x *Binlog) GetLogSize() int64 { + if x != nil { + return x.LogSize + } + return 0 +} + +func (x *Binlog) GetLogID() int64 { + if x != nil { + return x.LogID + } + return 0 +} + +func (x *Binlog) GetMemorySize() int64 { + if x != nil { + return x.MemorySize + } + return 0 +} + +type GetRecoveryInfoResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + Channels []*VchannelInfo `protobuf:"bytes,2,rep,name=channels,proto3" json:"channels,omitempty"` + Binlogs []*SegmentBinlogs `protobuf:"bytes,3,rep,name=binlogs,proto3" json:"binlogs,omitempty"` +} + +func (x *GetRecoveryInfoResponse) Reset() { + *x = GetRecoveryInfoResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[37] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetRecoveryInfoResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetRecoveryInfoResponse) ProtoMessage() {} + +func (x *GetRecoveryInfoResponse) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[37] + 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 GetRecoveryInfoResponse.ProtoReflect.Descriptor instead. +func (*GetRecoveryInfoResponse) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{37} +} + +func (x *GetRecoveryInfoResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *GetRecoveryInfoResponse) GetChannels() []*VchannelInfo { + if x != nil { + return x.Channels + } + return nil +} + +func (x *GetRecoveryInfoResponse) GetBinlogs() []*SegmentBinlogs { + if x != nil { + return x.Binlogs + } + return nil +} + +type GetRecoveryInfoRequest 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"` + PartitionID int64 `protobuf:"varint,3,opt,name=partitionID,proto3" json:"partitionID,omitempty"` +} + +func (x *GetRecoveryInfoRequest) Reset() { + *x = GetRecoveryInfoRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[38] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetRecoveryInfoRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetRecoveryInfoRequest) ProtoMessage() {} + +func (x *GetRecoveryInfoRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[38] + 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 GetRecoveryInfoRequest.ProtoReflect.Descriptor instead. +func (*GetRecoveryInfoRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{38} +} + +func (x *GetRecoveryInfoRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *GetRecoveryInfoRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *GetRecoveryInfoRequest) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +type GetRecoveryInfoResponseV2 struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + Channels []*VchannelInfo `protobuf:"bytes,2,rep,name=channels,proto3" json:"channels,omitempty"` + Segments []*SegmentInfo `protobuf:"bytes,3,rep,name=segments,proto3" json:"segments,omitempty"` +} + +func (x *GetRecoveryInfoResponseV2) Reset() { + *x = GetRecoveryInfoResponseV2{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[39] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetRecoveryInfoResponseV2) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetRecoveryInfoResponseV2) ProtoMessage() {} + +func (x *GetRecoveryInfoResponseV2) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[39] + 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 GetRecoveryInfoResponseV2.ProtoReflect.Descriptor instead. +func (*GetRecoveryInfoResponseV2) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{39} +} + +func (x *GetRecoveryInfoResponseV2) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *GetRecoveryInfoResponseV2) GetChannels() []*VchannelInfo { + if x != nil { + return x.Channels + } + return nil +} + +func (x *GetRecoveryInfoResponseV2) GetSegments() []*SegmentInfo { + if x != nil { + return x.Segments + } + return nil +} + +type GetRecoveryInfoRequestV2 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"` + PartitionIDs []int64 `protobuf:"varint,3,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` +} + +func (x *GetRecoveryInfoRequestV2) Reset() { + *x = GetRecoveryInfoRequestV2{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[40] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetRecoveryInfoRequestV2) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetRecoveryInfoRequestV2) ProtoMessage() {} + +func (x *GetRecoveryInfoRequestV2) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[40] + 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 GetRecoveryInfoRequestV2.ProtoReflect.Descriptor instead. +func (*GetRecoveryInfoRequestV2) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{40} +} + +func (x *GetRecoveryInfoRequestV2) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *GetRecoveryInfoRequestV2) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *GetRecoveryInfoRequestV2) GetPartitionIDs() []int64 { + if x != nil { + return x.PartitionIDs + } + return nil +} + +type GetChannelRecoveryInfoRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + Vchannel string `protobuf:"bytes,2,opt,name=vchannel,proto3" json:"vchannel,omitempty"` +} + +func (x *GetChannelRecoveryInfoRequest) Reset() { + *x = GetChannelRecoveryInfoRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[41] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetChannelRecoveryInfoRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetChannelRecoveryInfoRequest) ProtoMessage() {} + +func (x *GetChannelRecoveryInfoRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[41] + 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 GetChannelRecoveryInfoRequest.ProtoReflect.Descriptor instead. +func (*GetChannelRecoveryInfoRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{41} +} + +func (x *GetChannelRecoveryInfoRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *GetChannelRecoveryInfoRequest) GetVchannel() string { + if x != nil { + return x.Vchannel + } + return "" +} + +type GetChannelRecoveryInfoResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + Info *VchannelInfo `protobuf:"bytes,2,opt,name=info,proto3" json:"info,omitempty"` + Schema *schemapb.CollectionSchema `protobuf:"bytes,3,opt,name=schema,proto3" json:"schema,omitempty"` +} + +func (x *GetChannelRecoveryInfoResponse) Reset() { + *x = GetChannelRecoveryInfoResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[42] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetChannelRecoveryInfoResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetChannelRecoveryInfoResponse) ProtoMessage() {} + +func (x *GetChannelRecoveryInfoResponse) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[42] + 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 GetChannelRecoveryInfoResponse.ProtoReflect.Descriptor instead. +func (*GetChannelRecoveryInfoResponse) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{42} +} + +func (x *GetChannelRecoveryInfoResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *GetChannelRecoveryInfoResponse) GetInfo() *VchannelInfo { + if x != nil { + return x.Info + } + return nil +} + +func (x *GetChannelRecoveryInfoResponse) GetSchema() *schemapb.CollectionSchema { + if x != nil { + return x.Schema + } + return nil +} + +type GetSegmentsByStatesRequest 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"` + PartitionID int64 `protobuf:"varint,3,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + States []commonpb.SegmentState `protobuf:"varint,4,rep,packed,name=states,proto3,enum=milvus.proto.common.SegmentState" json:"states,omitempty"` +} + +func (x *GetSegmentsByStatesRequest) Reset() { + *x = GetSegmentsByStatesRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[43] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetSegmentsByStatesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetSegmentsByStatesRequest) ProtoMessage() {} + +func (x *GetSegmentsByStatesRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[43] + 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 GetSegmentsByStatesRequest.ProtoReflect.Descriptor instead. +func (*GetSegmentsByStatesRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{43} +} + +func (x *GetSegmentsByStatesRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *GetSegmentsByStatesRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *GetSegmentsByStatesRequest) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *GetSegmentsByStatesRequest) GetStates() []commonpb.SegmentState { + if x != nil { + return x.States + } + return nil +} + +type GetSegmentsByStatesResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + Segments []int64 `protobuf:"varint,2,rep,packed,name=segments,proto3" json:"segments,omitempty"` +} + +func (x *GetSegmentsByStatesResponse) Reset() { + *x = GetSegmentsByStatesResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[44] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetSegmentsByStatesResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetSegmentsByStatesResponse) ProtoMessage() {} + +func (x *GetSegmentsByStatesResponse) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[44] + 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 GetSegmentsByStatesResponse.ProtoReflect.Descriptor instead. +func (*GetSegmentsByStatesResponse) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{44} +} + +func (x *GetSegmentsByStatesResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *GetSegmentsByStatesResponse) GetSegments() []int64 { + if x != nil { + return x.Segments + } + return nil +} + +type GetFlushedSegmentsRequest 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"` + PartitionID int64 `protobuf:"varint,3,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + IncludeUnhealthy bool `protobuf:"varint,4,opt,name=includeUnhealthy,proto3" json:"includeUnhealthy,omitempty"` +} + +func (x *GetFlushedSegmentsRequest) Reset() { + *x = GetFlushedSegmentsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[45] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetFlushedSegmentsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetFlushedSegmentsRequest) ProtoMessage() {} + +func (x *GetFlushedSegmentsRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[45] + 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 GetFlushedSegmentsRequest.ProtoReflect.Descriptor instead. +func (*GetFlushedSegmentsRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{45} +} + +func (x *GetFlushedSegmentsRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *GetFlushedSegmentsRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *GetFlushedSegmentsRequest) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *GetFlushedSegmentsRequest) GetIncludeUnhealthy() bool { + if x != nil { + return x.IncludeUnhealthy + } + return false +} + +type GetFlushedSegmentsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + Segments []int64 `protobuf:"varint,2,rep,packed,name=segments,proto3" json:"segments,omitempty"` +} + +func (x *GetFlushedSegmentsResponse) Reset() { + *x = GetFlushedSegmentsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[46] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetFlushedSegmentsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetFlushedSegmentsResponse) ProtoMessage() {} + +func (x *GetFlushedSegmentsResponse) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[46] + 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 GetFlushedSegmentsResponse.ProtoReflect.Descriptor instead. +func (*GetFlushedSegmentsResponse) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{46} +} + +func (x *GetFlushedSegmentsResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *GetFlushedSegmentsResponse) GetSegments() []int64 { + if x != nil { + return x.Segments + } + return nil +} + +type SegmentFlushCompletedMsg struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + Segment *SegmentInfo `protobuf:"bytes,2,opt,name=segment,proto3" json:"segment,omitempty"` +} + +func (x *SegmentFlushCompletedMsg) Reset() { + *x = SegmentFlushCompletedMsg{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[47] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentFlushCompletedMsg) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentFlushCompletedMsg) ProtoMessage() {} + +func (x *SegmentFlushCompletedMsg) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[47] + 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 SegmentFlushCompletedMsg.ProtoReflect.Descriptor instead. +func (*SegmentFlushCompletedMsg) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{47} +} + +func (x *SegmentFlushCompletedMsg) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *SegmentFlushCompletedMsg) GetSegment() *SegmentInfo { + if x != nil { + return x.Segment + } + return nil +} + +type ChannelWatchInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Vchan *VchannelInfo `protobuf:"bytes,1,opt,name=vchan,proto3" json:"vchan,omitempty"` + StartTs int64 `protobuf:"varint,2,opt,name=startTs,proto3" json:"startTs,omitempty"` + State ChannelWatchState `protobuf:"varint,3,opt,name=state,proto3,enum=milvus.proto.data.ChannelWatchState" json:"state,omitempty"` + // the timeout ts, datanode shall do nothing after it + // NOT USED. + TimeoutTs int64 `protobuf:"varint,4,opt,name=timeoutTs,proto3" json:"timeoutTs,omitempty"` + // the schema of the collection to watch, to avoid get schema rpc issues. + Schema *schemapb.CollectionSchema `protobuf:"bytes,5,opt,name=schema,proto3" json:"schema,omitempty"` + // watch progress, deprecated + Progress int32 `protobuf:"varint,6,opt,name=progress,proto3" json:"progress,omitempty"` + OpID int64 `protobuf:"varint,7,opt,name=opID,proto3" json:"opID,omitempty"` + DbProperties []*commonpb.KeyValuePair `protobuf:"bytes,8,rep,name=dbProperties,proto3" json:"dbProperties,omitempty"` +} + +func (x *ChannelWatchInfo) Reset() { + *x = ChannelWatchInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[48] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ChannelWatchInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ChannelWatchInfo) ProtoMessage() {} + +func (x *ChannelWatchInfo) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[48] + 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 ChannelWatchInfo.ProtoReflect.Descriptor instead. +func (*ChannelWatchInfo) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{48} +} + +func (x *ChannelWatchInfo) GetVchan() *VchannelInfo { + if x != nil { + return x.Vchan + } + return nil +} + +func (x *ChannelWatchInfo) GetStartTs() int64 { + if x != nil { + return x.StartTs + } + return 0 +} + +func (x *ChannelWatchInfo) GetState() ChannelWatchState { + if x != nil { + return x.State + } + return ChannelWatchState_Uncomplete +} + +func (x *ChannelWatchInfo) GetTimeoutTs() int64 { + if x != nil { + return x.TimeoutTs + } + return 0 +} + +func (x *ChannelWatchInfo) GetSchema() *schemapb.CollectionSchema { + if x != nil { + return x.Schema + } + return nil +} + +func (x *ChannelWatchInfo) GetProgress() int32 { + if x != nil { + return x.Progress + } + return 0 +} + +func (x *ChannelWatchInfo) GetOpID() int64 { + if x != nil { + return x.OpID + } + return 0 +} + +func (x *ChannelWatchInfo) GetDbProperties() []*commonpb.KeyValuePair { + if x != nil { + return x.DbProperties + } + return nil +} + +type CompactionStateRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + PlanID int64 `protobuf:"varint,2,opt,name=planID,proto3" json:"planID,omitempty"` +} + +func (x *CompactionStateRequest) Reset() { + *x = CompactionStateRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[49] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CompactionStateRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CompactionStateRequest) ProtoMessage() {} + +func (x *CompactionStateRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[49] + 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 CompactionStateRequest.ProtoReflect.Descriptor instead. +func (*CompactionStateRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{49} +} + +func (x *CompactionStateRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *CompactionStateRequest) GetPlanID() int64 { + if x != nil { + return x.PlanID + } + return 0 +} + +type SyncSegmentInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SegmentId int64 `protobuf:"varint,1,opt,name=segment_id,json=segmentId,proto3" json:"segment_id,omitempty"` + PkStatsLog *FieldBinlog `protobuf:"bytes,2,opt,name=pk_stats_log,json=pkStatsLog,proto3" json:"pk_stats_log,omitempty"` + State commonpb.SegmentState `protobuf:"varint,3,opt,name=state,proto3,enum=milvus.proto.common.SegmentState" json:"state,omitempty"` + Level SegmentLevel `protobuf:"varint,4,opt,name=level,proto3,enum=milvus.proto.data.SegmentLevel" json:"level,omitempty"` + NumOfRows int64 `protobuf:"varint,5,opt,name=num_of_rows,json=numOfRows,proto3" json:"num_of_rows,omitempty"` +} + +func (x *SyncSegmentInfo) Reset() { + *x = SyncSegmentInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[50] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SyncSegmentInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SyncSegmentInfo) ProtoMessage() {} + +func (x *SyncSegmentInfo) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[50] + 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 SyncSegmentInfo.ProtoReflect.Descriptor instead. +func (*SyncSegmentInfo) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{50} +} + +func (x *SyncSegmentInfo) GetSegmentId() int64 { + if x != nil { + return x.SegmentId + } + return 0 +} + +func (x *SyncSegmentInfo) GetPkStatsLog() *FieldBinlog { + if x != nil { + return x.PkStatsLog + } + return nil +} + +func (x *SyncSegmentInfo) GetState() commonpb.SegmentState { + if x != nil { + return x.State + } + return commonpb.SegmentState(0) +} + +func (x *SyncSegmentInfo) GetLevel() SegmentLevel { + if x != nil { + return x.Level + } + return SegmentLevel_Legacy +} + +func (x *SyncSegmentInfo) GetNumOfRows() int64 { + if x != nil { + return x.NumOfRows + } + return 0 +} + +type SyncSegmentsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Deprecated, after v2.4.3 + PlanID int64 `protobuf:"varint,1,opt,name=planID,proto3" json:"planID,omitempty"` + // Deprecated, after v2.4.3 + CompactedTo int64 `protobuf:"varint,2,opt,name=compacted_to,json=compactedTo,proto3" json:"compacted_to,omitempty"` + // Deprecated, after v2.4.3 + NumOfRows int64 `protobuf:"varint,3,opt,name=num_of_rows,json=numOfRows,proto3" json:"num_of_rows,omitempty"` + // Deprecated, after v2.4.3 + CompactedFrom []int64 `protobuf:"varint,4,rep,packed,name=compacted_from,json=compactedFrom,proto3" json:"compacted_from,omitempty"` + // Deprecated, after v2.4.3 + StatsLogs []*FieldBinlog `protobuf:"bytes,5,rep,name=stats_logs,json=statsLogs,proto3" json:"stats_logs,omitempty"` + ChannelName string `protobuf:"bytes,6,opt,name=channel_name,json=channelName,proto3" json:"channel_name,omitempty"` + PartitionId int64 `protobuf:"varint,7,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"` + CollectionId int64 `protobuf:"varint,8,opt,name=collection_id,json=collectionId,proto3" json:"collection_id,omitempty"` + SegmentInfos map[int64]*SyncSegmentInfo `protobuf:"bytes,9,rep,name=segment_infos,json=segmentInfos,proto3" json:"segment_infos,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *SyncSegmentsRequest) Reset() { + *x = SyncSegmentsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[51] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SyncSegmentsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SyncSegmentsRequest) ProtoMessage() {} + +func (x *SyncSegmentsRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[51] + 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 SyncSegmentsRequest.ProtoReflect.Descriptor instead. +func (*SyncSegmentsRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{51} +} + +func (x *SyncSegmentsRequest) GetPlanID() int64 { + if x != nil { + return x.PlanID + } + return 0 +} + +func (x *SyncSegmentsRequest) GetCompactedTo() int64 { + if x != nil { + return x.CompactedTo + } + return 0 +} + +func (x *SyncSegmentsRequest) GetNumOfRows() int64 { + if x != nil { + return x.NumOfRows + } + return 0 +} + +func (x *SyncSegmentsRequest) GetCompactedFrom() []int64 { + if x != nil { + return x.CompactedFrom + } + return nil +} + +func (x *SyncSegmentsRequest) GetStatsLogs() []*FieldBinlog { + if x != nil { + return x.StatsLogs + } + return nil +} + +func (x *SyncSegmentsRequest) GetChannelName() string { + if x != nil { + return x.ChannelName + } + return "" +} + +func (x *SyncSegmentsRequest) GetPartitionId() int64 { + if x != nil { + return x.PartitionId + } + return 0 +} + +func (x *SyncSegmentsRequest) GetCollectionId() int64 { + if x != nil { + return x.CollectionId + } + return 0 +} + +func (x *SyncSegmentsRequest) GetSegmentInfos() map[int64]*SyncSegmentInfo { + if x != nil { + return x.SegmentInfos + } + return nil +} + +type CompactionSegmentBinlogs struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SegmentID int64 `protobuf:"varint,1,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + FieldBinlogs []*FieldBinlog `protobuf:"bytes,2,rep,name=fieldBinlogs,proto3" json:"fieldBinlogs,omitempty"` + Field2StatslogPaths []*FieldBinlog `protobuf:"bytes,3,rep,name=field2StatslogPaths,proto3" json:"field2StatslogPaths,omitempty"` + Deltalogs []*FieldBinlog `protobuf:"bytes,4,rep,name=deltalogs,proto3" json:"deltalogs,omitempty"` + InsertChannel string `protobuf:"bytes,5,opt,name=insert_channel,json=insertChannel,proto3" json:"insert_channel,omitempty"` + Level SegmentLevel `protobuf:"varint,6,opt,name=level,proto3,enum=milvus.proto.data.SegmentLevel" json:"level,omitempty"` + CollectionID int64 `protobuf:"varint,7,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionID int64 `protobuf:"varint,8,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + IsSorted bool `protobuf:"varint,9,opt,name=is_sorted,json=isSorted,proto3" json:"is_sorted,omitempty"` +} + +func (x *CompactionSegmentBinlogs) Reset() { + *x = CompactionSegmentBinlogs{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[52] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CompactionSegmentBinlogs) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CompactionSegmentBinlogs) ProtoMessage() {} + +func (x *CompactionSegmentBinlogs) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[52] + 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 CompactionSegmentBinlogs.ProtoReflect.Descriptor instead. +func (*CompactionSegmentBinlogs) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{52} +} + +func (x *CompactionSegmentBinlogs) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *CompactionSegmentBinlogs) GetFieldBinlogs() []*FieldBinlog { + if x != nil { + return x.FieldBinlogs + } + return nil +} + +func (x *CompactionSegmentBinlogs) GetField2StatslogPaths() []*FieldBinlog { + if x != nil { + return x.Field2StatslogPaths + } + return nil +} + +func (x *CompactionSegmentBinlogs) GetDeltalogs() []*FieldBinlog { + if x != nil { + return x.Deltalogs + } + return nil +} + +func (x *CompactionSegmentBinlogs) GetInsertChannel() string { + if x != nil { + return x.InsertChannel + } + return "" +} + +func (x *CompactionSegmentBinlogs) GetLevel() SegmentLevel { + if x != nil { + return x.Level + } + return SegmentLevel_Legacy +} + +func (x *CompactionSegmentBinlogs) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *CompactionSegmentBinlogs) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *CompactionSegmentBinlogs) GetIsSorted() bool { + if x != nil { + return x.IsSorted + } + return false +} + +type CompactionPlan struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + PlanID int64 `protobuf:"varint,1,opt,name=planID,proto3" json:"planID,omitempty"` + SegmentBinlogs []*CompactionSegmentBinlogs `protobuf:"bytes,2,rep,name=segmentBinlogs,proto3" json:"segmentBinlogs,omitempty"` + StartTime int64 `protobuf:"varint,3,opt,name=start_time,json=startTime,proto3" json:"start_time,omitempty"` + TimeoutInSeconds int32 `protobuf:"varint,4,opt,name=timeout_in_seconds,json=timeoutInSeconds,proto3" json:"timeout_in_seconds,omitempty"` + Type CompactionType `protobuf:"varint,5,opt,name=type,proto3,enum=milvus.proto.data.CompactionType" json:"type,omitempty"` + Timetravel uint64 `protobuf:"varint,6,opt,name=timetravel,proto3" json:"timetravel,omitempty"` + Channel string `protobuf:"bytes,7,opt,name=channel,proto3" json:"channel,omitempty"` + CollectionTtl int64 `protobuf:"varint,8,opt,name=collection_ttl,json=collectionTtl,proto3" json:"collection_ttl,omitempty"` // nanoseconds + TotalRows int64 `protobuf:"varint,9,opt,name=total_rows,json=totalRows,proto3" json:"total_rows,omitempty"` + Schema *schemapb.CollectionSchema `protobuf:"bytes,10,opt,name=schema,proto3" json:"schema,omitempty"` + ClusteringKeyField int64 `protobuf:"varint,11,opt,name=clustering_key_field,json=clusteringKeyField,proto3" json:"clustering_key_field,omitempty"` + MaxSegmentRows int64 `protobuf:"varint,12,opt,name=max_segment_rows,json=maxSegmentRows,proto3" json:"max_segment_rows,omitempty"` + PreferSegmentRows int64 `protobuf:"varint,13,opt,name=prefer_segment_rows,json=preferSegmentRows,proto3" json:"prefer_segment_rows,omitempty"` + AnalyzeResultPath string `protobuf:"bytes,14,opt,name=analyze_result_path,json=analyzeResultPath,proto3" json:"analyze_result_path,omitempty"` + AnalyzeSegmentIds []int64 `protobuf:"varint,15,rep,packed,name=analyze_segment_ids,json=analyzeSegmentIds,proto3" json:"analyze_segment_ids,omitempty"` + State int32 `protobuf:"varint,16,opt,name=state,proto3" json:"state,omitempty"` + BeginLogID int64 `protobuf:"varint,17,opt,name=begin_logID,json=beginLogID,proto3" json:"begin_logID,omitempty"` + PreAllocatedSegmentIDs *IDRange `protobuf:"bytes,18,opt,name=pre_allocated_segmentIDs,json=preAllocatedSegmentIDs,proto3" json:"pre_allocated_segmentIDs,omitempty"` + SlotUsage int64 `protobuf:"varint,19,opt,name=slot_usage,json=slotUsage,proto3" json:"slot_usage,omitempty"` + MaxSize int64 `protobuf:"varint,20,opt,name=max_size,json=maxSize,proto3" json:"max_size,omitempty"` +} + +func (x *CompactionPlan) Reset() { + *x = CompactionPlan{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[53] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CompactionPlan) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CompactionPlan) ProtoMessage() {} + +func (x *CompactionPlan) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[53] + 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 CompactionPlan.ProtoReflect.Descriptor instead. +func (*CompactionPlan) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{53} +} + +func (x *CompactionPlan) GetPlanID() int64 { + if x != nil { + return x.PlanID + } + return 0 +} + +func (x *CompactionPlan) GetSegmentBinlogs() []*CompactionSegmentBinlogs { + if x != nil { + return x.SegmentBinlogs + } + return nil +} + +func (x *CompactionPlan) GetStartTime() int64 { + if x != nil { + return x.StartTime + } + return 0 +} + +func (x *CompactionPlan) GetTimeoutInSeconds() int32 { + if x != nil { + return x.TimeoutInSeconds + } + return 0 +} + +func (x *CompactionPlan) GetType() CompactionType { + if x != nil { + return x.Type + } + return CompactionType_UndefinedCompaction +} + +func (x *CompactionPlan) GetTimetravel() uint64 { + if x != nil { + return x.Timetravel + } + return 0 +} + +func (x *CompactionPlan) GetChannel() string { + if x != nil { + return x.Channel + } + return "" +} + +func (x *CompactionPlan) GetCollectionTtl() int64 { + if x != nil { + return x.CollectionTtl + } + return 0 +} + +func (x *CompactionPlan) GetTotalRows() int64 { + if x != nil { + return x.TotalRows + } + return 0 +} + +func (x *CompactionPlan) GetSchema() *schemapb.CollectionSchema { + if x != nil { + return x.Schema + } + return nil +} + +func (x *CompactionPlan) GetClusteringKeyField() int64 { + if x != nil { + return x.ClusteringKeyField + } + return 0 +} + +func (x *CompactionPlan) GetMaxSegmentRows() int64 { + if x != nil { + return x.MaxSegmentRows + } + return 0 +} + +func (x *CompactionPlan) GetPreferSegmentRows() int64 { + if x != nil { + return x.PreferSegmentRows + } + return 0 +} + +func (x *CompactionPlan) GetAnalyzeResultPath() string { + if x != nil { + return x.AnalyzeResultPath + } + return "" +} + +func (x *CompactionPlan) GetAnalyzeSegmentIds() []int64 { + if x != nil { + return x.AnalyzeSegmentIds + } + return nil +} + +func (x *CompactionPlan) GetState() int32 { + if x != nil { + return x.State + } + return 0 +} + +func (x *CompactionPlan) GetBeginLogID() int64 { + if x != nil { + return x.BeginLogID + } + return 0 +} + +func (x *CompactionPlan) GetPreAllocatedSegmentIDs() *IDRange { + if x != nil { + return x.PreAllocatedSegmentIDs + } + return nil +} + +func (x *CompactionPlan) GetSlotUsage() int64 { + if x != nil { + return x.SlotUsage + } + return 0 +} + +func (x *CompactionPlan) GetMaxSize() int64 { + if x != nil { + return x.MaxSize + } + return 0 +} + +type CompactionSegment struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + PlanID int64 `protobuf:"varint,1,opt,name=planID,proto3" json:"planID,omitempty"` // deprecated after 2.3.4 + SegmentID int64 `protobuf:"varint,2,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + NumOfRows int64 `protobuf:"varint,3,opt,name=num_of_rows,json=numOfRows,proto3" json:"num_of_rows,omitempty"` + InsertLogs []*FieldBinlog `protobuf:"bytes,4,rep,name=insert_logs,json=insertLogs,proto3" json:"insert_logs,omitempty"` + Field2StatslogPaths []*FieldBinlog `protobuf:"bytes,5,rep,name=field2StatslogPaths,proto3" json:"field2StatslogPaths,omitempty"` + Deltalogs []*FieldBinlog `protobuf:"bytes,6,rep,name=deltalogs,proto3" json:"deltalogs,omitempty"` + Channel string `protobuf:"bytes,7,opt,name=channel,proto3" json:"channel,omitempty"` + IsSorted bool `protobuf:"varint,8,opt,name=is_sorted,json=isSorted,proto3" json:"is_sorted,omitempty"` + Bm25Logs []*FieldBinlog `protobuf:"bytes,9,rep,name=bm25logs,proto3" json:"bm25logs,omitempty"` +} + +func (x *CompactionSegment) Reset() { + *x = CompactionSegment{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[54] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CompactionSegment) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CompactionSegment) ProtoMessage() {} + +func (x *CompactionSegment) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[54] + 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 CompactionSegment.ProtoReflect.Descriptor instead. +func (*CompactionSegment) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{54} +} + +func (x *CompactionSegment) GetPlanID() int64 { + if x != nil { + return x.PlanID + } + return 0 +} + +func (x *CompactionSegment) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *CompactionSegment) GetNumOfRows() int64 { + if x != nil { + return x.NumOfRows + } + return 0 +} + +func (x *CompactionSegment) GetInsertLogs() []*FieldBinlog { + if x != nil { + return x.InsertLogs + } + return nil +} + +func (x *CompactionSegment) GetField2StatslogPaths() []*FieldBinlog { + if x != nil { + return x.Field2StatslogPaths + } + return nil +} + +func (x *CompactionSegment) GetDeltalogs() []*FieldBinlog { + if x != nil { + return x.Deltalogs + } + return nil +} + +func (x *CompactionSegment) GetChannel() string { + if x != nil { + return x.Channel + } + return "" +} + +func (x *CompactionSegment) GetIsSorted() bool { + if x != nil { + return x.IsSorted + } + return false +} + +func (x *CompactionSegment) GetBm25Logs() []*FieldBinlog { + if x != nil { + return x.Bm25Logs + } + return nil +} + +type CompactionPlanResult struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + PlanID int64 `protobuf:"varint,1,opt,name=planID,proto3" json:"planID,omitempty"` + State CompactionTaskState `protobuf:"varint,2,opt,name=state,proto3,enum=milvus.proto.data.CompactionTaskState" json:"state,omitempty"` + Segments []*CompactionSegment `protobuf:"bytes,3,rep,name=segments,proto3" json:"segments,omitempty"` + Channel string `protobuf:"bytes,4,opt,name=channel,proto3" json:"channel,omitempty"` + Type CompactionType `protobuf:"varint,5,opt,name=type,proto3,enum=milvus.proto.data.CompactionType" json:"type,omitempty"` +} + +func (x *CompactionPlanResult) Reset() { + *x = CompactionPlanResult{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[55] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CompactionPlanResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CompactionPlanResult) ProtoMessage() {} + +func (x *CompactionPlanResult) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[55] + 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 CompactionPlanResult.ProtoReflect.Descriptor instead. +func (*CompactionPlanResult) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{55} +} + +func (x *CompactionPlanResult) GetPlanID() int64 { + if x != nil { + return x.PlanID + } + return 0 +} + +func (x *CompactionPlanResult) GetState() CompactionTaskState { + if x != nil { + return x.State + } + return CompactionTaskState_unknown +} + +func (x *CompactionPlanResult) GetSegments() []*CompactionSegment { + if x != nil { + return x.Segments + } + return nil +} + +func (x *CompactionPlanResult) GetChannel() string { + if x != nil { + return x.Channel + } + return "" +} + +func (x *CompactionPlanResult) GetType() CompactionType { + if x != nil { + return x.Type + } + return CompactionType_UndefinedCompaction +} + +type CompactionStateResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + Results []*CompactionPlanResult `protobuf:"bytes,2,rep,name=results,proto3" json:"results,omitempty"` +} + +func (x *CompactionStateResponse) Reset() { + *x = CompactionStateResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[56] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CompactionStateResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CompactionStateResponse) ProtoMessage() {} + +func (x *CompactionStateResponse) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[56] + 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 CompactionStateResponse.ProtoReflect.Descriptor instead. +func (*CompactionStateResponse) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{56} +} + +func (x *CompactionStateResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *CompactionStateResponse) GetResults() []*CompactionPlanResult { + if x != nil { + return x.Results + } + return nil +} + +// Deprecated +type SegmentFieldBinlogMeta struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + FieldID int64 `protobuf:"varint,1,opt,name=fieldID,proto3" json:"fieldID,omitempty"` + BinlogPath string `protobuf:"bytes,2,opt,name=binlog_path,json=binlogPath,proto3" json:"binlog_path,omitempty"` +} + +func (x *SegmentFieldBinlogMeta) Reset() { + *x = SegmentFieldBinlogMeta{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[57] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentFieldBinlogMeta) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentFieldBinlogMeta) ProtoMessage() {} + +func (x *SegmentFieldBinlogMeta) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[57] + 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 SegmentFieldBinlogMeta.ProtoReflect.Descriptor instead. +func (*SegmentFieldBinlogMeta) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{57} +} + +func (x *SegmentFieldBinlogMeta) GetFieldID() int64 { + if x != nil { + return x.FieldID + } + return 0 +} + +func (x *SegmentFieldBinlogMeta) GetBinlogPath() string { + if x != nil { + return x.BinlogPath + } + return "" +} + +type WatchChannelsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + ChannelNames []string `protobuf:"bytes,2,rep,name=channelNames,proto3" json:"channelNames,omitempty"` + StartPositions []*commonpb.KeyDataPair `protobuf:"bytes,3,rep,name=start_positions,json=startPositions,proto3" json:"start_positions,omitempty"` + Schema *schemapb.CollectionSchema `protobuf:"bytes,4,opt,name=schema,proto3" json:"schema,omitempty"` + CreateTimestamp uint64 `protobuf:"varint,5,opt,name=create_timestamp,json=createTimestamp,proto3" json:"create_timestamp,omitempty"` + DbProperties []*commonpb.KeyValuePair `protobuf:"bytes,6,rep,name=db_properties,json=dbProperties,proto3" json:"db_properties,omitempty"` +} + +func (x *WatchChannelsRequest) Reset() { + *x = WatchChannelsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[58] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WatchChannelsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WatchChannelsRequest) ProtoMessage() {} + +func (x *WatchChannelsRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[58] + 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 WatchChannelsRequest.ProtoReflect.Descriptor instead. +func (*WatchChannelsRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{58} +} + +func (x *WatchChannelsRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *WatchChannelsRequest) GetChannelNames() []string { + if x != nil { + return x.ChannelNames + } + return nil +} + +func (x *WatchChannelsRequest) GetStartPositions() []*commonpb.KeyDataPair { + if x != nil { + return x.StartPositions + } + return nil +} + +func (x *WatchChannelsRequest) GetSchema() *schemapb.CollectionSchema { + if x != nil { + return x.Schema + } + return nil +} + +func (x *WatchChannelsRequest) GetCreateTimestamp() uint64 { + if x != nil { + return x.CreateTimestamp + } + return 0 +} + +func (x *WatchChannelsRequest) GetDbProperties() []*commonpb.KeyValuePair { + if x != nil { + return x.DbProperties + } + return nil +} + +type WatchChannelsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` +} + +func (x *WatchChannelsResponse) Reset() { + *x = WatchChannelsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[59] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WatchChannelsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WatchChannelsResponse) ProtoMessage() {} + +func (x *WatchChannelsResponse) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[59] + 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 WatchChannelsResponse.ProtoReflect.Descriptor instead. +func (*WatchChannelsResponse) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{59} +} + +func (x *WatchChannelsResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +type SetSegmentStateRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + SegmentId int64 `protobuf:"varint,2,opt,name=segment_id,json=segmentId,proto3" json:"segment_id,omitempty"` + NewState commonpb.SegmentState `protobuf:"varint,3,opt,name=new_state,json=newState,proto3,enum=milvus.proto.common.SegmentState" json:"new_state,omitempty"` +} + +func (x *SetSegmentStateRequest) Reset() { + *x = SetSegmentStateRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[60] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SetSegmentStateRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SetSegmentStateRequest) ProtoMessage() {} + +func (x *SetSegmentStateRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[60] + 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 SetSegmentStateRequest.ProtoReflect.Descriptor instead. +func (*SetSegmentStateRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{60} +} + +func (x *SetSegmentStateRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *SetSegmentStateRequest) GetSegmentId() int64 { + if x != nil { + return x.SegmentId + } + return 0 +} + +func (x *SetSegmentStateRequest) GetNewState() commonpb.SegmentState { + if x != nil { + return x.NewState + } + return commonpb.SegmentState(0) +} + +type SetSegmentStateResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` +} + +func (x *SetSegmentStateResponse) Reset() { + *x = SetSegmentStateResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[61] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SetSegmentStateResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SetSegmentStateResponse) ProtoMessage() {} + +func (x *SetSegmentStateResponse) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[61] + 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 SetSegmentStateResponse.ProtoReflect.Descriptor instead. +func (*SetSegmentStateResponse) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{61} +} + +func (x *SetSegmentStateResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +type DropVirtualChannelRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + ChannelName string `protobuf:"bytes,2,opt,name=channel_name,json=channelName,proto3" json:"channel_name,omitempty"` + Segments []*DropVirtualChannelSegment `protobuf:"bytes,3,rep,name=segments,proto3" json:"segments,omitempty"` +} + +func (x *DropVirtualChannelRequest) Reset() { + *x = DropVirtualChannelRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[62] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DropVirtualChannelRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DropVirtualChannelRequest) ProtoMessage() {} + +func (x *DropVirtualChannelRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[62] + 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 DropVirtualChannelRequest.ProtoReflect.Descriptor instead. +func (*DropVirtualChannelRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{62} +} + +func (x *DropVirtualChannelRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *DropVirtualChannelRequest) GetChannelName() string { + if x != nil { + return x.ChannelName + } + return "" +} + +func (x *DropVirtualChannelRequest) GetSegments() []*DropVirtualChannelSegment { + if x != nil { + return x.Segments + } + return nil +} + +type DropVirtualChannelSegment struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SegmentID int64 `protobuf:"varint,1,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + CollectionID int64 `protobuf:"varint,2,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + Field2BinlogPaths []*FieldBinlog `protobuf:"bytes,3,rep,name=field2BinlogPaths,proto3" json:"field2BinlogPaths,omitempty"` + Field2StatslogPaths []*FieldBinlog `protobuf:"bytes,4,rep,name=field2StatslogPaths,proto3" json:"field2StatslogPaths,omitempty"` + Deltalogs []*FieldBinlog `protobuf:"bytes,5,rep,name=deltalogs,proto3" json:"deltalogs,omitempty"` + StartPosition *msgpb.MsgPosition `protobuf:"bytes,6,opt,name=startPosition,proto3" json:"startPosition,omitempty"` + CheckPoint *msgpb.MsgPosition `protobuf:"bytes,7,opt,name=checkPoint,proto3" json:"checkPoint,omitempty"` + NumOfRows int64 `protobuf:"varint,8,opt,name=numOfRows,proto3" json:"numOfRows,omitempty"` +} + +func (x *DropVirtualChannelSegment) Reset() { + *x = DropVirtualChannelSegment{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[63] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DropVirtualChannelSegment) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DropVirtualChannelSegment) ProtoMessage() {} + +func (x *DropVirtualChannelSegment) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[63] + 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 DropVirtualChannelSegment.ProtoReflect.Descriptor instead. +func (*DropVirtualChannelSegment) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{63} +} + +func (x *DropVirtualChannelSegment) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *DropVirtualChannelSegment) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *DropVirtualChannelSegment) GetField2BinlogPaths() []*FieldBinlog { + if x != nil { + return x.Field2BinlogPaths + } + return nil +} + +func (x *DropVirtualChannelSegment) GetField2StatslogPaths() []*FieldBinlog { + if x != nil { + return x.Field2StatslogPaths + } + return nil +} + +func (x *DropVirtualChannelSegment) GetDeltalogs() []*FieldBinlog { + if x != nil { + return x.Deltalogs + } + return nil +} + +func (x *DropVirtualChannelSegment) GetStartPosition() *msgpb.MsgPosition { + if x != nil { + return x.StartPosition + } + return nil +} + +func (x *DropVirtualChannelSegment) GetCheckPoint() *msgpb.MsgPosition { + if x != nil { + return x.CheckPoint + } + return nil +} + +func (x *DropVirtualChannelSegment) GetNumOfRows() int64 { + if x != nil { + return x.NumOfRows + } + return 0 +} + +type DropVirtualChannelResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` +} + +func (x *DropVirtualChannelResponse) Reset() { + *x = DropVirtualChannelResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[64] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DropVirtualChannelResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DropVirtualChannelResponse) ProtoMessage() {} + +func (x *DropVirtualChannelResponse) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[64] + 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 DropVirtualChannelResponse.ProtoReflect.Descriptor instead. +func (*DropVirtualChannelResponse) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{64} +} + +func (x *DropVirtualChannelResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +type UpdateSegmentStatisticsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + Stats []*commonpb.SegmentStats `protobuf:"bytes,2,rep,name=stats,proto3" json:"stats,omitempty"` +} + +func (x *UpdateSegmentStatisticsRequest) Reset() { + *x = UpdateSegmentStatisticsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[65] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateSegmentStatisticsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateSegmentStatisticsRequest) ProtoMessage() {} + +func (x *UpdateSegmentStatisticsRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[65] + 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 UpdateSegmentStatisticsRequest.ProtoReflect.Descriptor instead. +func (*UpdateSegmentStatisticsRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{65} +} + +func (x *UpdateSegmentStatisticsRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *UpdateSegmentStatisticsRequest) GetStats() []*commonpb.SegmentStats { + if x != nil { + return x.Stats + } + return nil +} + +type UpdateChannelCheckpointRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + VChannel string `protobuf:"bytes,2,opt,name=vChannel,proto3" json:"vChannel,omitempty"` // deprecated, keep it for compatibility + Position *msgpb.MsgPosition `protobuf:"bytes,3,opt,name=position,proto3" json:"position,omitempty"` // deprecated, keep it for compatibility + ChannelCheckpoints []*msgpb.MsgPosition `protobuf:"bytes,4,rep,name=channel_checkpoints,json=channelCheckpoints,proto3" json:"channel_checkpoints,omitempty"` +} + +func (x *UpdateChannelCheckpointRequest) Reset() { + *x = UpdateChannelCheckpointRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[66] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateChannelCheckpointRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateChannelCheckpointRequest) ProtoMessage() {} + +func (x *UpdateChannelCheckpointRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[66] + 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 UpdateChannelCheckpointRequest.ProtoReflect.Descriptor instead. +func (*UpdateChannelCheckpointRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{66} +} + +func (x *UpdateChannelCheckpointRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *UpdateChannelCheckpointRequest) GetVChannel() string { + if x != nil { + return x.VChannel + } + return "" +} + +func (x *UpdateChannelCheckpointRequest) GetPosition() *msgpb.MsgPosition { + if x != nil { + return x.Position + } + return nil +} + +func (x *UpdateChannelCheckpointRequest) GetChannelCheckpoints() []*msgpb.MsgPosition { + if x != nil { + return x.ChannelCheckpoints + } + return nil +} + +type ResendSegmentStatsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` +} + +func (x *ResendSegmentStatsRequest) Reset() { + *x = ResendSegmentStatsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[67] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ResendSegmentStatsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResendSegmentStatsRequest) ProtoMessage() {} + +func (x *ResendSegmentStatsRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[67] + 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 ResendSegmentStatsRequest.ProtoReflect.Descriptor instead. +func (*ResendSegmentStatsRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{67} +} + +func (x *ResendSegmentStatsRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +type ResendSegmentStatsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + SegResent []int64 `protobuf:"varint,2,rep,packed,name=seg_resent,json=segResent,proto3" json:"seg_resent,omitempty"` +} + +func (x *ResendSegmentStatsResponse) Reset() { + *x = ResendSegmentStatsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[68] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ResendSegmentStatsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResendSegmentStatsResponse) ProtoMessage() {} + +func (x *ResendSegmentStatsResponse) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[68] + 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 ResendSegmentStatsResponse.ProtoReflect.Descriptor instead. +func (*ResendSegmentStatsResponse) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{68} +} + +func (x *ResendSegmentStatsResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *ResendSegmentStatsResponse) GetSegResent() []int64 { + if x != nil { + return x.SegResent + } + return nil +} + +type MarkSegmentsDroppedRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + SegmentIds []int64 `protobuf:"varint,2,rep,packed,name=segment_ids,json=segmentIds,proto3" json:"segment_ids,omitempty"` // IDs of segments that needs to be marked as `dropped`. +} + +func (x *MarkSegmentsDroppedRequest) Reset() { + *x = MarkSegmentsDroppedRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[69] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *MarkSegmentsDroppedRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MarkSegmentsDroppedRequest) ProtoMessage() {} + +func (x *MarkSegmentsDroppedRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[69] + 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 MarkSegmentsDroppedRequest.ProtoReflect.Descriptor instead. +func (*MarkSegmentsDroppedRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{69} +} + +func (x *MarkSegmentsDroppedRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *MarkSegmentsDroppedRequest) GetSegmentIds() []int64 { + if x != nil { + return x.SegmentIds + } + return nil +} + +type SegmentReferenceLock struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + TaskID int64 `protobuf:"varint,1,opt,name=taskID,proto3" json:"taskID,omitempty"` + NodeID int64 `protobuf:"varint,2,opt,name=nodeID,proto3" json:"nodeID,omitempty"` + SegmentIDs []int64 `protobuf:"varint,3,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,omitempty"` +} + +func (x *SegmentReferenceLock) Reset() { + *x = SegmentReferenceLock{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[70] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentReferenceLock) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentReferenceLock) ProtoMessage() {} + +func (x *SegmentReferenceLock) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[70] + 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 SegmentReferenceLock.ProtoReflect.Descriptor instead. +func (*SegmentReferenceLock) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{70} +} + +func (x *SegmentReferenceLock) GetTaskID() int64 { + if x != nil { + return x.TaskID + } + return 0 +} + +func (x *SegmentReferenceLock) GetNodeID() int64 { + if x != nil { + return x.NodeID + } + return 0 +} + +func (x *SegmentReferenceLock) GetSegmentIDs() []int64 { + if x != nil { + return x.SegmentIDs + } + return nil +} + +type AlterCollectionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + Schema *schemapb.CollectionSchema `protobuf:"bytes,2,opt,name=schema,proto3" json:"schema,omitempty"` + PartitionIDs []int64 `protobuf:"varint,3,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` + StartPositions []*commonpb.KeyDataPair `protobuf:"bytes,4,rep,name=start_positions,json=startPositions,proto3" json:"start_positions,omitempty"` + Properties []*commonpb.KeyValuePair `protobuf:"bytes,5,rep,name=properties,proto3" json:"properties,omitempty"` + DbID int64 `protobuf:"varint,6,opt,name=dbID,proto3" json:"dbID,omitempty"` + VChannels []string `protobuf:"bytes,7,rep,name=vChannels,proto3" json:"vChannels,omitempty"` +} + +func (x *AlterCollectionRequest) Reset() { + *x = AlterCollectionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[71] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AlterCollectionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AlterCollectionRequest) ProtoMessage() {} + +func (x *AlterCollectionRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[71] + 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 AlterCollectionRequest.ProtoReflect.Descriptor instead. +func (*AlterCollectionRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{71} +} + +func (x *AlterCollectionRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *AlterCollectionRequest) GetSchema() *schemapb.CollectionSchema { + if x != nil { + return x.Schema + } + return nil +} + +func (x *AlterCollectionRequest) GetPartitionIDs() []int64 { + if x != nil { + return x.PartitionIDs + } + return nil +} + +func (x *AlterCollectionRequest) GetStartPositions() []*commonpb.KeyDataPair { + if x != nil { + return x.StartPositions + } + return nil +} + +func (x *AlterCollectionRequest) GetProperties() []*commonpb.KeyValuePair { + if x != nil { + return x.Properties + } + return nil +} + +func (x *AlterCollectionRequest) GetDbID() int64 { + if x != nil { + return x.DbID + } + return 0 +} + +func (x *AlterCollectionRequest) GetVChannels() []string { + if x != nil { + return x.VChannels + } + return nil +} + +type GcConfirmRequest 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"` + PartitionId int64 `protobuf:"varint,2,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"` // -1 means whole collection. +} + +func (x *GcConfirmRequest) Reset() { + *x = GcConfirmRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[72] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GcConfirmRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GcConfirmRequest) ProtoMessage() {} + +func (x *GcConfirmRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[72] + 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 GcConfirmRequest.ProtoReflect.Descriptor instead. +func (*GcConfirmRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{72} +} + +func (x *GcConfirmRequest) GetCollectionId() int64 { + if x != nil { + return x.CollectionId + } + return 0 +} + +func (x *GcConfirmRequest) GetPartitionId() int64 { + if x != nil { + return x.PartitionId + } + return 0 +} + +type GcConfirmResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + GcFinished bool `protobuf:"varint,2,opt,name=gc_finished,json=gcFinished,proto3" json:"gc_finished,omitempty"` +} + +func (x *GcConfirmResponse) Reset() { + *x = GcConfirmResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[73] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GcConfirmResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GcConfirmResponse) ProtoMessage() {} + +func (x *GcConfirmResponse) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[73] + 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 GcConfirmResponse.ProtoReflect.Descriptor instead. +func (*GcConfirmResponse) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{73} +} + +func (x *GcConfirmResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *GcConfirmResponse) GetGcFinished() bool { + if x != nil { + return x.GcFinished + } + return false +} + +type ReportDataNodeTtMsgsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + Msgs []*msgpb.DataNodeTtMsg `protobuf:"bytes,2,rep,name=msgs,proto3" json:"msgs,omitempty"` // -1 means whole collection. +} + +func (x *ReportDataNodeTtMsgsRequest) Reset() { + *x = ReportDataNodeTtMsgsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[74] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ReportDataNodeTtMsgsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ReportDataNodeTtMsgsRequest) ProtoMessage() {} + +func (x *ReportDataNodeTtMsgsRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[74] + 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 ReportDataNodeTtMsgsRequest.ProtoReflect.Descriptor instead. +func (*ReportDataNodeTtMsgsRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{74} +} + +func (x *ReportDataNodeTtMsgsRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *ReportDataNodeTtMsgsRequest) GetMsgs() []*msgpb.DataNodeTtMsg { + if x != nil { + return x.Msgs + } + return nil +} + +type GetFlushStateRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SegmentIDs []int64 `protobuf:"varint,1,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,omitempty"` + FlushTs uint64 `protobuf:"varint,2,opt,name=flush_ts,json=flushTs,proto3" json:"flush_ts,omitempty"` + DbName string `protobuf:"bytes,3,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` + CollectionName string `protobuf:"bytes,4,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` + CollectionID int64 `protobuf:"varint,5,opt,name=collectionID,proto3" json:"collectionID,omitempty"` +} + +func (x *GetFlushStateRequest) Reset() { + *x = GetFlushStateRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[75] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetFlushStateRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetFlushStateRequest) ProtoMessage() {} + +func (x *GetFlushStateRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[75] + 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 GetFlushStateRequest.ProtoReflect.Descriptor instead. +func (*GetFlushStateRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{75} +} + +func (x *GetFlushStateRequest) GetSegmentIDs() []int64 { + if x != nil { + return x.SegmentIDs + } + return nil +} + +func (x *GetFlushStateRequest) GetFlushTs() uint64 { + if x != nil { + return x.FlushTs + } + return 0 +} + +func (x *GetFlushStateRequest) GetDbName() string { + if x != nil { + return x.DbName + } + return "" +} + +func (x *GetFlushStateRequest) GetCollectionName() string { + if x != nil { + return x.CollectionName + } + return "" +} + +func (x *GetFlushStateRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +type ChannelOperationsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Infos []*ChannelWatchInfo `protobuf:"bytes,1,rep,name=infos,proto3" json:"infos,omitempty"` +} + +func (x *ChannelOperationsRequest) Reset() { + *x = ChannelOperationsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[76] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ChannelOperationsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ChannelOperationsRequest) ProtoMessage() {} + +func (x *ChannelOperationsRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[76] + 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 ChannelOperationsRequest.ProtoReflect.Descriptor instead. +func (*ChannelOperationsRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{76} +} + +func (x *ChannelOperationsRequest) GetInfos() []*ChannelWatchInfo { + if x != nil { + return x.Infos + } + return nil +} + +type ChannelOperationProgressResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + OpID int64 `protobuf:"varint,2,opt,name=opID,proto3" json:"opID,omitempty"` + State ChannelWatchState `protobuf:"varint,3,opt,name=state,proto3,enum=milvus.proto.data.ChannelWatchState" json:"state,omitempty"` + Progress int32 `protobuf:"varint,4,opt,name=progress,proto3" json:"progress,omitempty"` +} + +func (x *ChannelOperationProgressResponse) Reset() { + *x = ChannelOperationProgressResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[77] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ChannelOperationProgressResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ChannelOperationProgressResponse) ProtoMessage() {} + +func (x *ChannelOperationProgressResponse) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[77] + 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 ChannelOperationProgressResponse.ProtoReflect.Descriptor instead. +func (*ChannelOperationProgressResponse) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{77} +} + +func (x *ChannelOperationProgressResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *ChannelOperationProgressResponse) GetOpID() int64 { + if x != nil { + return x.OpID + } + return 0 +} + +func (x *ChannelOperationProgressResponse) GetState() ChannelWatchState { + if x != nil { + return x.State + } + return ChannelWatchState_Uncomplete +} + +func (x *ChannelOperationProgressResponse) GetProgress() int32 { + if x != nil { + return x.Progress + } + return 0 +} + +type PreImportRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ClusterID string `protobuf:"bytes,1,opt,name=clusterID,proto3" json:"clusterID,omitempty"` + JobID int64 `protobuf:"varint,2,opt,name=jobID,proto3" json:"jobID,omitempty"` + TaskID int64 `protobuf:"varint,3,opt,name=taskID,proto3" json:"taskID,omitempty"` + CollectionID int64 `protobuf:"varint,4,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionIDs []int64 `protobuf:"varint,5,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` + Vchannels []string `protobuf:"bytes,6,rep,name=vchannels,proto3" json:"vchannels,omitempty"` + Schema *schemapb.CollectionSchema `protobuf:"bytes,7,opt,name=schema,proto3" json:"schema,omitempty"` + ImportFiles []*internalpb.ImportFile `protobuf:"bytes,8,rep,name=import_files,json=importFiles,proto3" json:"import_files,omitempty"` + Options []*commonpb.KeyValuePair `protobuf:"bytes,9,rep,name=options,proto3" json:"options,omitempty"` +} + +func (x *PreImportRequest) Reset() { + *x = PreImportRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[78] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PreImportRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PreImportRequest) ProtoMessage() {} + +func (x *PreImportRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[78] + 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 PreImportRequest.ProtoReflect.Descriptor instead. +func (*PreImportRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{78} +} + +func (x *PreImportRequest) GetClusterID() string { + if x != nil { + return x.ClusterID + } + return "" +} + +func (x *PreImportRequest) GetJobID() int64 { + if x != nil { + return x.JobID + } + return 0 +} + +func (x *PreImportRequest) GetTaskID() int64 { + if x != nil { + return x.TaskID + } + return 0 +} + +func (x *PreImportRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *PreImportRequest) GetPartitionIDs() []int64 { + if x != nil { + return x.PartitionIDs + } + return nil +} + +func (x *PreImportRequest) GetVchannels() []string { + if x != nil { + return x.Vchannels + } + return nil +} + +func (x *PreImportRequest) GetSchema() *schemapb.CollectionSchema { + if x != nil { + return x.Schema + } + return nil +} + +func (x *PreImportRequest) GetImportFiles() []*internalpb.ImportFile { + if x != nil { + return x.ImportFiles + } + return nil +} + +func (x *PreImportRequest) GetOptions() []*commonpb.KeyValuePair { + if x != nil { + return x.Options + } + return nil +} + +type IDRange struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Begin int64 `protobuf:"varint,1,opt,name=begin,proto3" json:"begin,omitempty"` + End int64 `protobuf:"varint,2,opt,name=end,proto3" json:"end,omitempty"` +} + +func (x *IDRange) Reset() { + *x = IDRange{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[79] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *IDRange) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*IDRange) ProtoMessage() {} + +func (x *IDRange) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[79] + 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 IDRange.ProtoReflect.Descriptor instead. +func (*IDRange) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{79} +} + +func (x *IDRange) GetBegin() int64 { + if x != nil { + return x.Begin + } + return 0 +} + +func (x *IDRange) GetEnd() int64 { + if x != nil { + return x.End + } + return 0 +} + +type ImportRequestSegment struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SegmentID int64 `protobuf:"varint,1,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + PartitionID int64 `protobuf:"varint,2,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + Vchannel string `protobuf:"bytes,3,opt,name=vchannel,proto3" json:"vchannel,omitempty"` +} + +func (x *ImportRequestSegment) Reset() { + *x = ImportRequestSegment{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[80] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ImportRequestSegment) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ImportRequestSegment) ProtoMessage() {} + +func (x *ImportRequestSegment) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[80] + 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 ImportRequestSegment.ProtoReflect.Descriptor instead. +func (*ImportRequestSegment) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{80} +} + +func (x *ImportRequestSegment) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *ImportRequestSegment) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *ImportRequestSegment) GetVchannel() string { + if x != nil { + return x.Vchannel + } + return "" +} + +type ImportRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ClusterID string `protobuf:"bytes,1,opt,name=clusterID,proto3" json:"clusterID,omitempty"` + JobID int64 `protobuf:"varint,2,opt,name=jobID,proto3" json:"jobID,omitempty"` + TaskID int64 `protobuf:"varint,3,opt,name=taskID,proto3" json:"taskID,omitempty"` + CollectionID int64 `protobuf:"varint,4,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionIDs []int64 `protobuf:"varint,5,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` + Vchannels []string `protobuf:"bytes,6,rep,name=vchannels,proto3" json:"vchannels,omitempty"` + Schema *schemapb.CollectionSchema `protobuf:"bytes,7,opt,name=schema,proto3" json:"schema,omitempty"` + Files []*internalpb.ImportFile `protobuf:"bytes,8,rep,name=files,proto3" json:"files,omitempty"` + Options []*commonpb.KeyValuePair `protobuf:"bytes,9,rep,name=options,proto3" json:"options,omitempty"` + Ts uint64 `protobuf:"varint,10,opt,name=ts,proto3" json:"ts,omitempty"` + IDRange *IDRange `protobuf:"bytes,11,opt,name=ID_range,json=IDRange,proto3" json:"ID_range,omitempty"` + RequestSegments []*ImportRequestSegment `protobuf:"bytes,12,rep,name=request_segments,json=requestSegments,proto3" json:"request_segments,omitempty"` +} + +func (x *ImportRequest) Reset() { + *x = ImportRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[81] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ImportRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ImportRequest) ProtoMessage() {} + +func (x *ImportRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[81] + 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 ImportRequest.ProtoReflect.Descriptor instead. +func (*ImportRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{81} +} + +func (x *ImportRequest) GetClusterID() string { + if x != nil { + return x.ClusterID + } + return "" +} + +func (x *ImportRequest) GetJobID() int64 { + if x != nil { + return x.JobID + } + return 0 +} + +func (x *ImportRequest) GetTaskID() int64 { + if x != nil { + return x.TaskID + } + return 0 +} + +func (x *ImportRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *ImportRequest) GetPartitionIDs() []int64 { + if x != nil { + return x.PartitionIDs + } + return nil +} + +func (x *ImportRequest) GetVchannels() []string { + if x != nil { + return x.Vchannels + } + return nil +} + +func (x *ImportRequest) GetSchema() *schemapb.CollectionSchema { + if x != nil { + return x.Schema + } + return nil +} + +func (x *ImportRequest) GetFiles() []*internalpb.ImportFile { + if x != nil { + return x.Files + } + return nil +} + +func (x *ImportRequest) GetOptions() []*commonpb.KeyValuePair { + if x != nil { + return x.Options + } + return nil +} + +func (x *ImportRequest) GetTs() uint64 { + if x != nil { + return x.Ts + } + return 0 +} + +func (x *ImportRequest) GetIDRange() *IDRange { + if x != nil { + return x.IDRange + } + return nil +} + +func (x *ImportRequest) GetRequestSegments() []*ImportRequestSegment { + if x != nil { + return x.RequestSegments + } + return nil +} + +type QueryPreImportRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ClusterID string `protobuf:"bytes,1,opt,name=clusterID,proto3" json:"clusterID,omitempty"` + JobID int64 `protobuf:"varint,2,opt,name=jobID,proto3" json:"jobID,omitempty"` + TaskID int64 `protobuf:"varint,3,opt,name=taskID,proto3" json:"taskID,omitempty"` +} + +func (x *QueryPreImportRequest) Reset() { + *x = QueryPreImportRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[82] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *QueryPreImportRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryPreImportRequest) ProtoMessage() {} + +func (x *QueryPreImportRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[82] + 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 QueryPreImportRequest.ProtoReflect.Descriptor instead. +func (*QueryPreImportRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{82} +} + +func (x *QueryPreImportRequest) GetClusterID() string { + if x != nil { + return x.ClusterID + } + return "" +} + +func (x *QueryPreImportRequest) GetJobID() int64 { + if x != nil { + return x.JobID + } + return 0 +} + +func (x *QueryPreImportRequest) GetTaskID() int64 { + if x != nil { + return x.TaskID + } + return 0 +} + +type PartitionImportStats struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + PartitionRows map[int64]int64 `protobuf:"bytes,1,rep,name=partition_rows,json=partitionRows,proto3" json:"partition_rows,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` // partitionID -> numRows + PartitionDataSize map[int64]int64 `protobuf:"bytes,2,rep,name=partition_data_size,json=partitionDataSize,proto3" json:"partition_data_size,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` // partitionID -> dataSize +} + +func (x *PartitionImportStats) Reset() { + *x = PartitionImportStats{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[83] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PartitionImportStats) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PartitionImportStats) ProtoMessage() {} + +func (x *PartitionImportStats) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[83] + 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 PartitionImportStats.ProtoReflect.Descriptor instead. +func (*PartitionImportStats) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{83} +} + +func (x *PartitionImportStats) GetPartitionRows() map[int64]int64 { + if x != nil { + return x.PartitionRows + } + return nil +} + +func (x *PartitionImportStats) GetPartitionDataSize() map[int64]int64 { + if x != nil { + return x.PartitionDataSize + } + return nil +} + +type ImportFileStats struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ImportFile *internalpb.ImportFile `protobuf:"bytes,1,opt,name=import_file,json=importFile,proto3" json:"import_file,omitempty"` + FileSize int64 `protobuf:"varint,2,opt,name=file_size,json=fileSize,proto3" json:"file_size,omitempty"` + TotalRows int64 `protobuf:"varint,3,opt,name=total_rows,json=totalRows,proto3" json:"total_rows,omitempty"` + TotalMemorySize int64 `protobuf:"varint,4,opt,name=total_memory_size,json=totalMemorySize,proto3" json:"total_memory_size,omitempty"` + HashedStats map[string]*PartitionImportStats `protobuf:"bytes,5,rep,name=hashed_stats,json=hashedStats,proto3" json:"hashed_stats,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // channel -> PartitionImportStats +} + +func (x *ImportFileStats) Reset() { + *x = ImportFileStats{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[84] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ImportFileStats) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ImportFileStats) ProtoMessage() {} + +func (x *ImportFileStats) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[84] + 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 ImportFileStats.ProtoReflect.Descriptor instead. +func (*ImportFileStats) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{84} +} + +func (x *ImportFileStats) GetImportFile() *internalpb.ImportFile { + if x != nil { + return x.ImportFile + } + return nil +} + +func (x *ImportFileStats) GetFileSize() int64 { + if x != nil { + return x.FileSize + } + return 0 +} + +func (x *ImportFileStats) GetTotalRows() int64 { + if x != nil { + return x.TotalRows + } + return 0 +} + +func (x *ImportFileStats) GetTotalMemorySize() int64 { + if x != nil { + return x.TotalMemorySize + } + return 0 +} + +func (x *ImportFileStats) GetHashedStats() map[string]*PartitionImportStats { + if x != nil { + return x.HashedStats + } + return nil +} + +type QueryPreImportResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + TaskID int64 `protobuf:"varint,2,opt,name=taskID,proto3" json:"taskID,omitempty"` + State ImportTaskStateV2 `protobuf:"varint,3,opt,name=state,proto3,enum=milvus.proto.data.ImportTaskStateV2" json:"state,omitempty"` + Reason string `protobuf:"bytes,4,opt,name=reason,proto3" json:"reason,omitempty"` + Slots int64 `protobuf:"varint,5,opt,name=slots,proto3" json:"slots,omitempty"` + FileStats []*ImportFileStats `protobuf:"bytes,6,rep,name=file_stats,json=fileStats,proto3" json:"file_stats,omitempty"` +} + +func (x *QueryPreImportResponse) Reset() { + *x = QueryPreImportResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[85] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *QueryPreImportResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryPreImportResponse) ProtoMessage() {} + +func (x *QueryPreImportResponse) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[85] + 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 QueryPreImportResponse.ProtoReflect.Descriptor instead. +func (*QueryPreImportResponse) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{85} +} + +func (x *QueryPreImportResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *QueryPreImportResponse) GetTaskID() int64 { + if x != nil { + return x.TaskID + } + return 0 +} + +func (x *QueryPreImportResponse) GetState() ImportTaskStateV2 { + if x != nil { + return x.State + } + return ImportTaskStateV2_None +} + +func (x *QueryPreImportResponse) GetReason() string { + if x != nil { + return x.Reason + } + return "" +} + +func (x *QueryPreImportResponse) GetSlots() int64 { + if x != nil { + return x.Slots + } + return 0 +} + +func (x *QueryPreImportResponse) GetFileStats() []*ImportFileStats { + if x != nil { + return x.FileStats + } + return nil +} + +type QueryImportRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ClusterID string `protobuf:"bytes,1,opt,name=clusterID,proto3" json:"clusterID,omitempty"` + JobID int64 `protobuf:"varint,2,opt,name=jobID,proto3" json:"jobID,omitempty"` + TaskID int64 `protobuf:"varint,3,opt,name=taskID,proto3" json:"taskID,omitempty"` + QuerySlot bool `protobuf:"varint,4,opt,name=querySlot,proto3" json:"querySlot,omitempty"` +} + +func (x *QueryImportRequest) Reset() { + *x = QueryImportRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[86] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *QueryImportRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryImportRequest) ProtoMessage() {} + +func (x *QueryImportRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_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 QueryImportRequest.ProtoReflect.Descriptor instead. +func (*QueryImportRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{86} +} + +func (x *QueryImportRequest) GetClusterID() string { + if x != nil { + return x.ClusterID + } + return "" +} + +func (x *QueryImportRequest) GetJobID() int64 { + if x != nil { + return x.JobID + } + return 0 +} + +func (x *QueryImportRequest) GetTaskID() int64 { + if x != nil { + return x.TaskID + } + return 0 +} + +func (x *QueryImportRequest) GetQuerySlot() bool { + if x != nil { + return x.QuerySlot + } + return false +} + +type ImportSegmentInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SegmentID int64 `protobuf:"varint,1,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + ImportedRows int64 `protobuf:"varint,2,opt,name=imported_rows,json=importedRows,proto3" json:"imported_rows,omitempty"` + Binlogs []*FieldBinlog `protobuf:"bytes,3,rep,name=binlogs,proto3" json:"binlogs,omitempty"` + Statslogs []*FieldBinlog `protobuf:"bytes,4,rep,name=statslogs,proto3" json:"statslogs,omitempty"` + Deltalogs []*FieldBinlog `protobuf:"bytes,5,rep,name=deltalogs,proto3" json:"deltalogs,omitempty"` +} + +func (x *ImportSegmentInfo) Reset() { + *x = ImportSegmentInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[87] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ImportSegmentInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ImportSegmentInfo) ProtoMessage() {} + +func (x *ImportSegmentInfo) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[87] + 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 ImportSegmentInfo.ProtoReflect.Descriptor instead. +func (*ImportSegmentInfo) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{87} +} + +func (x *ImportSegmentInfo) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *ImportSegmentInfo) GetImportedRows() int64 { + if x != nil { + return x.ImportedRows + } + return 0 +} + +func (x *ImportSegmentInfo) GetBinlogs() []*FieldBinlog { + if x != nil { + return x.Binlogs + } + return nil +} + +func (x *ImportSegmentInfo) GetStatslogs() []*FieldBinlog { + if x != nil { + return x.Statslogs + } + return nil +} + +func (x *ImportSegmentInfo) GetDeltalogs() []*FieldBinlog { + if x != nil { + return x.Deltalogs + } + return nil +} + +type QueryImportResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + TaskID int64 `protobuf:"varint,2,opt,name=taskID,proto3" json:"taskID,omitempty"` + State ImportTaskStateV2 `protobuf:"varint,3,opt,name=state,proto3,enum=milvus.proto.data.ImportTaskStateV2" json:"state,omitempty"` + Reason string `protobuf:"bytes,4,opt,name=reason,proto3" json:"reason,omitempty"` + Slots int64 `protobuf:"varint,5,opt,name=slots,proto3" json:"slots,omitempty"` + ImportSegmentsInfo []*ImportSegmentInfo `protobuf:"bytes,6,rep,name=import_segments_info,json=importSegmentsInfo,proto3" json:"import_segments_info,omitempty"` +} + +func (x *QueryImportResponse) Reset() { + *x = QueryImportResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[88] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *QueryImportResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryImportResponse) ProtoMessage() {} + +func (x *QueryImportResponse) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[88] + 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 QueryImportResponse.ProtoReflect.Descriptor instead. +func (*QueryImportResponse) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{88} +} + +func (x *QueryImportResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *QueryImportResponse) GetTaskID() int64 { + if x != nil { + return x.TaskID + } + return 0 +} + +func (x *QueryImportResponse) GetState() ImportTaskStateV2 { + if x != nil { + return x.State + } + return ImportTaskStateV2_None +} + +func (x *QueryImportResponse) GetReason() string { + if x != nil { + return x.Reason + } + return "" +} + +func (x *QueryImportResponse) GetSlots() int64 { + if x != nil { + return x.Slots + } + return 0 +} + +func (x *QueryImportResponse) GetImportSegmentsInfo() []*ImportSegmentInfo { + if x != nil { + return x.ImportSegmentsInfo + } + return nil +} + +type DropImportRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ClusterID string `protobuf:"bytes,1,opt,name=clusterID,proto3" json:"clusterID,omitempty"` + JobID int64 `protobuf:"varint,2,opt,name=jobID,proto3" json:"jobID,omitempty"` + TaskID int64 `protobuf:"varint,3,opt,name=taskID,proto3" json:"taskID,omitempty"` +} + +func (x *DropImportRequest) Reset() { + *x = DropImportRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[89] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DropImportRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DropImportRequest) ProtoMessage() {} + +func (x *DropImportRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[89] + 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 DropImportRequest.ProtoReflect.Descriptor instead. +func (*DropImportRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{89} +} + +func (x *DropImportRequest) GetClusterID() string { + if x != nil { + return x.ClusterID + } + return "" +} + +func (x *DropImportRequest) GetJobID() int64 { + if x != nil { + return x.JobID + } + return 0 +} + +func (x *DropImportRequest) GetTaskID() int64 { + if x != nil { + return x.TaskID + } + return 0 +} + +type ImportJob struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + JobID int64 `protobuf:"varint,1,opt,name=jobID,proto3" json:"jobID,omitempty"` + DbID int64 `protobuf:"varint,2,opt,name=dbID,proto3" json:"dbID,omitempty"` + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + CollectionName string `protobuf:"bytes,4,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` + PartitionIDs []int64 `protobuf:"varint,5,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` + Vchannels []string `protobuf:"bytes,6,rep,name=vchannels,proto3" json:"vchannels,omitempty"` + Schema *schemapb.CollectionSchema `protobuf:"bytes,7,opt,name=schema,proto3" json:"schema,omitempty"` + TimeoutTs uint64 `protobuf:"varint,8,opt,name=timeout_ts,json=timeoutTs,proto3" json:"timeout_ts,omitempty"` + CleanupTs uint64 `protobuf:"varint,9,opt,name=cleanup_ts,json=cleanupTs,proto3" json:"cleanup_ts,omitempty"` + RequestedDiskSize int64 `protobuf:"varint,10,opt,name=requestedDiskSize,proto3" json:"requestedDiskSize,omitempty"` + State internalpb.ImportJobState `protobuf:"varint,11,opt,name=state,proto3,enum=milvus.proto.internal.ImportJobState" json:"state,omitempty"` + Reason string `protobuf:"bytes,12,opt,name=reason,proto3" json:"reason,omitempty"` + CompleteTime string `protobuf:"bytes,13,opt,name=complete_time,json=completeTime,proto3" json:"complete_time,omitempty"` + Files []*internalpb.ImportFile `protobuf:"bytes,14,rep,name=files,proto3" json:"files,omitempty"` + Options []*commonpb.KeyValuePair `protobuf:"bytes,15,rep,name=options,proto3" json:"options,omitempty"` + StartTime string `protobuf:"bytes,16,opt,name=start_time,json=startTime,proto3" json:"start_time,omitempty"` +} + +func (x *ImportJob) Reset() { + *x = ImportJob{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[90] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ImportJob) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ImportJob) ProtoMessage() {} + +func (x *ImportJob) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[90] + 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 ImportJob.ProtoReflect.Descriptor instead. +func (*ImportJob) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{90} +} + +func (x *ImportJob) GetJobID() int64 { + if x != nil { + return x.JobID + } + return 0 +} + +func (x *ImportJob) GetDbID() int64 { + if x != nil { + return x.DbID + } + return 0 +} + +func (x *ImportJob) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *ImportJob) GetCollectionName() string { + if x != nil { + return x.CollectionName + } + return "" +} + +func (x *ImportJob) GetPartitionIDs() []int64 { + if x != nil { + return x.PartitionIDs + } + return nil +} + +func (x *ImportJob) GetVchannels() []string { + if x != nil { + return x.Vchannels + } + return nil +} + +func (x *ImportJob) GetSchema() *schemapb.CollectionSchema { + if x != nil { + return x.Schema + } + return nil +} + +func (x *ImportJob) GetTimeoutTs() uint64 { + if x != nil { + return x.TimeoutTs + } + return 0 +} + +func (x *ImportJob) GetCleanupTs() uint64 { + if x != nil { + return x.CleanupTs + } + return 0 +} + +func (x *ImportJob) GetRequestedDiskSize() int64 { + if x != nil { + return x.RequestedDiskSize + } + return 0 +} + +func (x *ImportJob) GetState() internalpb.ImportJobState { + if x != nil { + return x.State + } + return internalpb.ImportJobState(0) +} + +func (x *ImportJob) GetReason() string { + if x != nil { + return x.Reason + } + return "" +} + +func (x *ImportJob) GetCompleteTime() string { + if x != nil { + return x.CompleteTime + } + return "" +} + +func (x *ImportJob) GetFiles() []*internalpb.ImportFile { + if x != nil { + return x.Files + } + return nil +} + +func (x *ImportJob) GetOptions() []*commonpb.KeyValuePair { + if x != nil { + return x.Options + } + return nil +} + +func (x *ImportJob) GetStartTime() string { + if x != nil { + return x.StartTime + } + return "" +} + +type PreImportTask struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + JobID int64 `protobuf:"varint,1,opt,name=jobID,proto3" json:"jobID,omitempty"` + TaskID int64 `protobuf:"varint,2,opt,name=taskID,proto3" json:"taskID,omitempty"` + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + NodeID int64 `protobuf:"varint,6,opt,name=nodeID,proto3" json:"nodeID,omitempty"` + State ImportTaskStateV2 `protobuf:"varint,7,opt,name=state,proto3,enum=milvus.proto.data.ImportTaskStateV2" json:"state,omitempty"` + Reason string `protobuf:"bytes,8,opt,name=reason,proto3" json:"reason,omitempty"` + FileStats []*ImportFileStats `protobuf:"bytes,10,rep,name=file_stats,json=fileStats,proto3" json:"file_stats,omitempty"` + CreatedTime string `protobuf:"bytes,11,opt,name=created_time,json=createdTime,proto3" json:"created_time,omitempty"` + CompleteTime string `protobuf:"bytes,12,opt,name=complete_time,json=completeTime,proto3" json:"complete_time,omitempty"` +} + +func (x *PreImportTask) Reset() { + *x = PreImportTask{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[91] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PreImportTask) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PreImportTask) ProtoMessage() {} + +func (x *PreImportTask) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[91] + 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 PreImportTask.ProtoReflect.Descriptor instead. +func (*PreImportTask) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{91} +} + +func (x *PreImportTask) GetJobID() int64 { + if x != nil { + return x.JobID + } + return 0 +} + +func (x *PreImportTask) GetTaskID() int64 { + if x != nil { + return x.TaskID + } + return 0 +} + +func (x *PreImportTask) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *PreImportTask) GetNodeID() int64 { + if x != nil { + return x.NodeID + } + return 0 +} + +func (x *PreImportTask) GetState() ImportTaskStateV2 { + if x != nil { + return x.State + } + return ImportTaskStateV2_None +} + +func (x *PreImportTask) GetReason() string { + if x != nil { + return x.Reason + } + return "" +} + +func (x *PreImportTask) GetFileStats() []*ImportFileStats { + if x != nil { + return x.FileStats + } + return nil +} + +func (x *PreImportTask) GetCreatedTime() string { + if x != nil { + return x.CreatedTime + } + return "" +} + +func (x *PreImportTask) GetCompleteTime() string { + if x != nil { + return x.CompleteTime + } + return "" +} + +type ImportTaskV2 struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + JobID int64 `protobuf:"varint,1,opt,name=jobID,proto3" json:"jobID,omitempty"` + TaskID int64 `protobuf:"varint,2,opt,name=taskID,proto3" json:"taskID,omitempty"` + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + SegmentIDs []int64 `protobuf:"varint,4,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,omitempty"` + NodeID int64 `protobuf:"varint,5,opt,name=nodeID,proto3" json:"nodeID,omitempty"` + State ImportTaskStateV2 `protobuf:"varint,6,opt,name=state,proto3,enum=milvus.proto.data.ImportTaskStateV2" json:"state,omitempty"` + Reason string `protobuf:"bytes,7,opt,name=reason,proto3" json:"reason,omitempty"` + CompleteTime string `protobuf:"bytes,8,opt,name=complete_time,json=completeTime,proto3" json:"complete_time,omitempty"` + FileStats []*ImportFileStats `protobuf:"bytes,9,rep,name=file_stats,json=fileStats,proto3" json:"file_stats,omitempty"` + StatsSegmentIDs []int64 `protobuf:"varint,10,rep,packed,name=stats_segmentIDs,json=statsSegmentIDs,proto3" json:"stats_segmentIDs,omitempty"` + CreatedTime string `protobuf:"bytes,11,opt,name=created_time,json=createdTime,proto3" json:"created_time,omitempty"` +} + +func (x *ImportTaskV2) Reset() { + *x = ImportTaskV2{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[92] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ImportTaskV2) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ImportTaskV2) ProtoMessage() {} + +func (x *ImportTaskV2) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[92] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ImportTaskV2.ProtoReflect.Descriptor instead. +func (*ImportTaskV2) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{92} +} + +func (x *ImportTaskV2) GetJobID() int64 { + if x != nil { + return x.JobID + } + return 0 +} + +func (x *ImportTaskV2) GetTaskID() int64 { + if x != nil { + return x.TaskID + } + return 0 +} + +func (x *ImportTaskV2) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *ImportTaskV2) GetSegmentIDs() []int64 { + if x != nil { + return x.SegmentIDs + } + return nil +} + +func (x *ImportTaskV2) GetNodeID() int64 { + if x != nil { + return x.NodeID + } + return 0 +} + +func (x *ImportTaskV2) GetState() ImportTaskStateV2 { + if x != nil { + return x.State + } + return ImportTaskStateV2_None +} + +func (x *ImportTaskV2) GetReason() string { + if x != nil { + return x.Reason + } + return "" +} + +func (x *ImportTaskV2) GetCompleteTime() string { + if x != nil { + return x.CompleteTime + } + return "" +} + +func (x *ImportTaskV2) GetFileStats() []*ImportFileStats { + if x != nil { + return x.FileStats + } + return nil +} + +func (x *ImportTaskV2) GetStatsSegmentIDs() []int64 { + if x != nil { + return x.StatsSegmentIDs + } + return nil +} + +func (x *ImportTaskV2) GetCreatedTime() string { + if x != nil { + return x.CreatedTime + } + return "" +} + +type GcControlRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + Command GcCommand `protobuf:"varint,2,opt,name=command,proto3,enum=milvus.proto.data.GcCommand" json:"command,omitempty"` + Params []*commonpb.KeyValuePair `protobuf:"bytes,3,rep,name=params,proto3" json:"params,omitempty"` +} + +func (x *GcControlRequest) Reset() { + *x = GcControlRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[93] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GcControlRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GcControlRequest) ProtoMessage() {} + +func (x *GcControlRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[93] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GcControlRequest.ProtoReflect.Descriptor instead. +func (*GcControlRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{93} +} + +func (x *GcControlRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *GcControlRequest) GetCommand() GcCommand { + if x != nil { + return x.Command + } + return GcCommand__ +} + +func (x *GcControlRequest) GetParams() []*commonpb.KeyValuePair { + if x != nil { + return x.Params + } + return nil +} + +type QuerySlotRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *QuerySlotRequest) Reset() { + *x = QuerySlotRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[94] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *QuerySlotRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QuerySlotRequest) ProtoMessage() {} + +func (x *QuerySlotRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[94] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QuerySlotRequest.ProtoReflect.Descriptor instead. +func (*QuerySlotRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{94} +} + +type QuerySlotResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + NumSlots int64 `protobuf:"varint,2,opt,name=num_slots,json=numSlots,proto3" json:"num_slots,omitempty"` +} + +func (x *QuerySlotResponse) Reset() { + *x = QuerySlotResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[95] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *QuerySlotResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QuerySlotResponse) ProtoMessage() {} + +func (x *QuerySlotResponse) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[95] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QuerySlotResponse.ProtoReflect.Descriptor instead. +func (*QuerySlotResponse) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{95} +} + +func (x *QuerySlotResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *QuerySlotResponse) GetNumSlots() int64 { + if x != nil { + return x.NumSlots + } + return 0 +} + +type CompactionTask struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + PlanID int64 `protobuf:"varint,1,opt,name=planID,proto3" json:"planID,omitempty"` + TriggerID int64 `protobuf:"varint,2,opt,name=triggerID,proto3" json:"triggerID,omitempty"` + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionID int64 `protobuf:"varint,4,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + Channel string `protobuf:"bytes,5,opt,name=channel,proto3" json:"channel,omitempty"` + Type CompactionType `protobuf:"varint,6,opt,name=type,proto3,enum=milvus.proto.data.CompactionType" json:"type,omitempty"` + State CompactionTaskState `protobuf:"varint,7,opt,name=state,proto3,enum=milvus.proto.data.CompactionTaskState" json:"state,omitempty"` + FailReason string `protobuf:"bytes,8,opt,name=fail_reason,json=failReason,proto3" json:"fail_reason,omitempty"` + StartTime int64 `protobuf:"varint,9,opt,name=start_time,json=startTime,proto3" json:"start_time,omitempty"` + EndTime int64 `protobuf:"varint,10,opt,name=end_time,json=endTime,proto3" json:"end_time,omitempty"` + TimeoutInSeconds int32 `protobuf:"varint,11,opt,name=timeout_in_seconds,json=timeoutInSeconds,proto3" json:"timeout_in_seconds,omitempty"` + RetryTimes int32 `protobuf:"varint,12,opt,name=retry_times,json=retryTimes,proto3" json:"retry_times,omitempty"` + CollectionTtl int64 `protobuf:"varint,13,opt,name=collection_ttl,json=collectionTtl,proto3" json:"collection_ttl,omitempty"` + TotalRows int64 `protobuf:"varint,14,opt,name=total_rows,json=totalRows,proto3" json:"total_rows,omitempty"` + InputSegments []int64 `protobuf:"varint,15,rep,packed,name=inputSegments,proto3" json:"inputSegments,omitempty"` + ResultSegments []int64 `protobuf:"varint,16,rep,packed,name=resultSegments,proto3" json:"resultSegments,omitempty"` + Pos *msgpb.MsgPosition `protobuf:"bytes,17,opt,name=pos,proto3" json:"pos,omitempty"` + NodeID int64 `protobuf:"varint,18,opt,name=nodeID,proto3" json:"nodeID,omitempty"` + Schema *schemapb.CollectionSchema `protobuf:"bytes,19,opt,name=schema,proto3" json:"schema,omitempty"` + ClusteringKeyField *schemapb.FieldSchema `protobuf:"bytes,20,opt,name=clustering_key_field,json=clusteringKeyField,proto3" json:"clustering_key_field,omitempty"` + MaxSegmentRows int64 `protobuf:"varint,21,opt,name=max_segment_rows,json=maxSegmentRows,proto3" json:"max_segment_rows,omitempty"` + PreferSegmentRows int64 `protobuf:"varint,22,opt,name=prefer_segment_rows,json=preferSegmentRows,proto3" json:"prefer_segment_rows,omitempty"` + AnalyzeTaskID int64 `protobuf:"varint,23,opt,name=analyzeTaskID,proto3" json:"analyzeTaskID,omitempty"` + AnalyzeVersion int64 `protobuf:"varint,24,opt,name=analyzeVersion,proto3" json:"analyzeVersion,omitempty"` + LastStateStartTime int64 `protobuf:"varint,25,opt,name=lastStateStartTime,proto3" json:"lastStateStartTime,omitempty"` + MaxSize int64 `protobuf:"varint,26,opt,name=max_size,json=maxSize,proto3" json:"max_size,omitempty"` + TmpSegments []int64 `protobuf:"varint,27,rep,packed,name=tmpSegments,proto3" json:"tmpSegments,omitempty"` + PreAllocatedSegmentIDs *IDRange `protobuf:"bytes,28,opt,name=pre_allocated_segmentIDs,json=preAllocatedSegmentIDs,proto3" json:"pre_allocated_segmentIDs,omitempty"` +} + +func (x *CompactionTask) Reset() { + *x = CompactionTask{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[96] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CompactionTask) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CompactionTask) ProtoMessage() {} + +func (x *CompactionTask) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[96] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CompactionTask.ProtoReflect.Descriptor instead. +func (*CompactionTask) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{96} +} + +func (x *CompactionTask) GetPlanID() int64 { + if x != nil { + return x.PlanID + } + return 0 +} + +func (x *CompactionTask) GetTriggerID() int64 { + if x != nil { + return x.TriggerID + } + return 0 +} + +func (x *CompactionTask) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *CompactionTask) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *CompactionTask) GetChannel() string { + if x != nil { + return x.Channel + } + return "" +} + +func (x *CompactionTask) GetType() CompactionType { + if x != nil { + return x.Type + } + return CompactionType_UndefinedCompaction +} + +func (x *CompactionTask) GetState() CompactionTaskState { + if x != nil { + return x.State + } + return CompactionTaskState_unknown +} + +func (x *CompactionTask) GetFailReason() string { + if x != nil { + return x.FailReason + } + return "" +} + +func (x *CompactionTask) GetStartTime() int64 { + if x != nil { + return x.StartTime + } + return 0 +} + +func (x *CompactionTask) GetEndTime() int64 { + if x != nil { + return x.EndTime + } + return 0 +} + +func (x *CompactionTask) GetTimeoutInSeconds() int32 { + if x != nil { + return x.TimeoutInSeconds + } + return 0 +} + +func (x *CompactionTask) GetRetryTimes() int32 { + if x != nil { + return x.RetryTimes + } + return 0 +} + +func (x *CompactionTask) GetCollectionTtl() int64 { + if x != nil { + return x.CollectionTtl + } + return 0 +} + +func (x *CompactionTask) GetTotalRows() int64 { + if x != nil { + return x.TotalRows + } + return 0 +} + +func (x *CompactionTask) GetInputSegments() []int64 { + if x != nil { + return x.InputSegments + } + return nil +} + +func (x *CompactionTask) GetResultSegments() []int64 { + if x != nil { + return x.ResultSegments + } + return nil +} + +func (x *CompactionTask) GetPos() *msgpb.MsgPosition { + if x != nil { + return x.Pos + } + return nil +} + +func (x *CompactionTask) GetNodeID() int64 { + if x != nil { + return x.NodeID + } + return 0 +} + +func (x *CompactionTask) GetSchema() *schemapb.CollectionSchema { + if x != nil { + return x.Schema + } + return nil +} + +func (x *CompactionTask) GetClusteringKeyField() *schemapb.FieldSchema { + if x != nil { + return x.ClusteringKeyField + } + return nil +} + +func (x *CompactionTask) GetMaxSegmentRows() int64 { + if x != nil { + return x.MaxSegmentRows + } + return 0 +} + +func (x *CompactionTask) GetPreferSegmentRows() int64 { + if x != nil { + return x.PreferSegmentRows + } + return 0 +} + +func (x *CompactionTask) GetAnalyzeTaskID() int64 { + if x != nil { + return x.AnalyzeTaskID + } + return 0 +} + +func (x *CompactionTask) GetAnalyzeVersion() int64 { + if x != nil { + return x.AnalyzeVersion + } + return 0 +} + +func (x *CompactionTask) GetLastStateStartTime() int64 { + if x != nil { + return x.LastStateStartTime + } + return 0 +} + +func (x *CompactionTask) GetMaxSize() int64 { + if x != nil { + return x.MaxSize + } + return 0 +} + +func (x *CompactionTask) GetTmpSegments() []int64 { + if x != nil { + return x.TmpSegments + } + return nil +} + +func (x *CompactionTask) GetPreAllocatedSegmentIDs() *IDRange { + if x != nil { + return x.PreAllocatedSegmentIDs + } + return nil +} + +type PartitionStatsInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionID int64 `protobuf:"varint,2,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + VChannel string `protobuf:"bytes,3,opt,name=vChannel,proto3" json:"vChannel,omitempty"` + Version int64 `protobuf:"varint,4,opt,name=version,proto3" json:"version,omitempty"` + SegmentIDs []int64 `protobuf:"varint,5,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,omitempty"` + AnalyzeTaskID int64 `protobuf:"varint,6,opt,name=analyzeTaskID,proto3" json:"analyzeTaskID,omitempty"` + CommitTime int64 `protobuf:"varint,7,opt,name=commitTime,proto3" json:"commitTime,omitempty"` +} + +func (x *PartitionStatsInfo) Reset() { + *x = PartitionStatsInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[97] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PartitionStatsInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PartitionStatsInfo) ProtoMessage() {} + +func (x *PartitionStatsInfo) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[97] + 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 PartitionStatsInfo.ProtoReflect.Descriptor instead. +func (*PartitionStatsInfo) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{97} +} + +func (x *PartitionStatsInfo) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *PartitionStatsInfo) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *PartitionStatsInfo) GetVChannel() string { + if x != nil { + return x.VChannel + } + return "" +} + +func (x *PartitionStatsInfo) GetVersion() int64 { + if x != nil { + return x.Version + } + return 0 +} + +func (x *PartitionStatsInfo) GetSegmentIDs() []int64 { + if x != nil { + return x.SegmentIDs + } + return nil +} + +func (x *PartitionStatsInfo) GetAnalyzeTaskID() int64 { + if x != nil { + return x.AnalyzeTaskID + } + return 0 +} + +func (x *PartitionStatsInfo) GetCommitTime() int64 { + if x != nil { + return x.CommitTime + } + return 0 +} + +type DropCompactionPlanRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + PlanID int64 `protobuf:"varint,1,opt,name=planID,proto3" json:"planID,omitempty"` +} + +func (x *DropCompactionPlanRequest) Reset() { + *x = DropCompactionPlanRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[98] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DropCompactionPlanRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DropCompactionPlanRequest) ProtoMessage() {} + +func (x *DropCompactionPlanRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[98] + 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 DropCompactionPlanRequest.ProtoReflect.Descriptor instead. +func (*DropCompactionPlanRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{98} +} + +func (x *DropCompactionPlanRequest) GetPlanID() int64 { + if x != nil { + return x.PlanID + } + return 0 +} + +var File_data_coord_proto protoreflect.FileDescriptor + +var file_data_coord_proto_rawDesc = []byte{ + 0x0a, 0x10, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x12, 0x11, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x64, 0x61, 0x74, 0x61, 0x1a, 0x0c, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x1a, 0x0e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x1a, 0x0c, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x1a, 0x0c, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, + 0x09, 0x6d, 0x73, 0x67, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x11, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x5f, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x07, 0x0a, + 0x05, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0xb4, 0x01, 0x0a, 0x0c, 0x46, 0x6c, 0x75, 0x73, 0x68, + 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, 0x12, 0x0a, 0x04, 0x64, 0x62, 0x49, + 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x44, 0x12, 0x1e, 0x0a, + 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, + 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, 0x22, 0x0a, + 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x44, 0x12, 0x1a, 0x0a, 0x08, 0x69, 0x73, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x22, 0xb2, 0x03, + 0x0a, 0x0d, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x62, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, + 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1e, 0x0a, 0x0a, + 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x03, + 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, 0x28, 0x0a, 0x0f, + 0x66, 0x6c, 0x75, 0x73, 0x68, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, + 0x05, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0f, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x74, 0x69, 0x6d, 0x65, 0x4f, 0x66, + 0x53, 0x65, 0x61, 0x6c, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x74, 0x69, 0x6d, 0x65, + 0x4f, 0x66, 0x53, 0x65, 0x61, 0x6c, 0x12, 0x19, 0x0a, 0x08, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x5f, + 0x74, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x54, + 0x73, 0x12, 0x51, 0x0a, 0x0b, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x63, 0x70, 0x73, + 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, + 0x43, 0x70, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 0x43, 0x70, 0x73, 0x1a, 0x5c, 0x0a, 0x0f, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, + 0x70, 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, 0x33, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x4d, 0x73, 0x67, 0x50, + 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, + 0x38, 0x01, 0x22, 0x7f, 0x0a, 0x14, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x43, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, + 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, + 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 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, 0x12, 0x1a, 0x0a, 0x08, 0x63, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, 0x63, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x73, 0x22, 0x88, 0x02, 0x0a, 0x10, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, + 0x44, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, + 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x21, + 0x0a, 0x0c, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, + 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1a, 0x0a, 0x08, 0x69, 0x73, 0x49, 0x6d, 0x70, + 0x6f, 0x72, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x49, 0x6d, 0x70, + 0x6f, 0x72, 0x74, 0x12, 0x22, 0x0a, 0x0c, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, + 0x6b, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x69, 0x6d, 0x70, 0x6f, 0x72, + 0x74, 0x54, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x35, 0x0a, 0x05, 0x6c, 0x65, 0x76, 0x65, 0x6c, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x05, 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x22, 0x98, + 0x01, 0x0a, 0x13, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 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, 0x12, 0x1d, + 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x1a, 0x0a, + 0x08, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x08, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x22, 0x8e, 0x01, 0x0a, 0x14, 0x41, 0x6c, + 0x6c, 0x6f, 0x63, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x41, 0x0a, 0x0c, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x6e, + 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0b, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0xa0, 0x01, 0x0a, 0x16, 0x41, + 0x73, 0x73, 0x69, 0x67, 0x6e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x1b, 0x0a, + 0x09, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x72, 0x6f, 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x70, 0x65, 0x65, 0x72, 0x52, 0x6f, 0x6c, 0x65, 0x12, 0x51, 0x0a, 0x11, 0x73, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x18, + 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x44, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x11, 0x73, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x49, 0x44, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x22, 0x80, 0x02, + 0x0a, 0x13, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x41, 0x73, 0x73, 0x69, 0x67, + 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x65, 0x67, 0x49, 0x44, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x73, 0x65, 0x67, 0x49, 0x44, 0x12, 0x21, 0x0a, 0x0c, 0x63, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0b, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x14, + 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x63, + 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1f, 0x0a, 0x0b, 0x65, 0x78, + 0x70, 0x69, 0x72, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, + 0x0a, 0x65, 0x78, 0x70, 0x69, 0x72, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x22, 0xa2, 0x01, 0x0a, 0x17, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x44, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x52, 0x0a, 0x10, + 0x73, 0x65, 0x67, 0x49, 0x44, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, + 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x44, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x10, + 0x73, 0x65, 0x67, 0x49, 0x44, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, + 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x6b, 0x0a, 0x17, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, + 0x73, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, + 0x44, 0x73, 0x22, 0xa6, 0x02, 0x0a, 0x10, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x37, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x44, + 0x0a, 0x0e, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x4d, 0x73, 0x67, 0x50, 0x6f, 0x73, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0d, 0x73, 0x74, 0x61, 0x72, 0x74, 0x50, 0x6f, 0x73, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x40, 0x0a, 0x0c, 0x65, 0x6e, 0x64, 0x5f, 0x70, 0x6f, 0x73, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x4d, 0x73, + 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0b, 0x65, 0x6e, 0x64, 0x50, 0x6f, + 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x8c, 0x01, 0x0a, 0x18, + 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x3b, 0x0a, + 0x06, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x49, 0x6e, + 0x66, 0x6f, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x22, 0x95, 0x01, 0x0a, 0x15, 0x47, + 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 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, 0x1e, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, 0x2a, 0x0a, 0x10, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, + 0x65, 0x55, 0x6e, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x10, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x55, 0x6e, 0x48, 0x65, 0x61, 0x6c, 0x74, + 0x68, 0x79, 0x22, 0xd9, 0x02, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, + 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x12, 0x34, 0x0a, 0x05, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, + 0x6f, 0x52, 0x05, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x6f, 0x0a, 0x12, 0x63, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x03, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x40, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, + 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, + 0x74, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x11, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, + 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x1a, 0x63, 0x0a, 0x16, 0x43, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x33, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x4d, 0x73, 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x6d, + 0x0a, 0x1b, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x42, 0x69, 0x6e, 0x6c, 0x6f, + 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, + 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, + 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x22, 0xa8, 0x01, + 0x0a, 0x1c, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x42, 0x69, 0x6e, 0x6c, 0x6f, + 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1a, + 0x0a, 0x08, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x03, + 0x52, 0x08, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x73, 0x12, 0x37, 0x0a, 0x05, 0x70, 0x61, + 0x74, 0x68, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, + 0x6c, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x05, 0x70, 0x61, + 0x74, 0x68, 0x73, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x8a, 0x01, 0x0a, 0x1e, 0x47, 0x65, 0x74, + 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, + 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 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, 0x12, 0x0a, + 0x04, 0x64, 0x62, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, + 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x22, 0x8f, 0x01, 0x0a, 0x1f, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, + 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x37, 0x0a, 0x05, 0x73, 0x74, 0x61, + 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, + 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x05, 0x73, 0x74, 0x61, + 0x74, 0x73, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, + 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0xad, 0x01, 0x0a, 0x1d, 0x47, 0x65, 0x74, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, + 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 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, 0x12, 0x0a, 0x04, 0x64, + 0x62, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x44, 0x12, + 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x44, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x22, 0x8e, 0x01, 0x0a, 0x1e, 0x47, 0x65, 0x74, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, + 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x37, 0x0a, 0x05, 0x73, 0x74, + 0x61, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x05, 0x73, 0x74, + 0x61, 0x74, 0x73, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x1e, 0x0a, 0x1c, 0x47, 0x65, 0x74, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0xfb, 0x06, 0x0a, 0x0c, 0x56, 0x63, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, + 0x0b, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0b, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, + 0x42, 0x0a, 0x0d, 0x73, 0x65, 0x65, 0x6b, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x4d, 0x73, 0x67, 0x50, 0x6f, 0x73, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0c, 0x73, 0x65, 0x65, 0x6b, 0x50, 0x6f, 0x73, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x4c, 0x0a, 0x11, 0x75, 0x6e, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x11, + 0x75, 0x6e, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x73, 0x12, 0x48, 0x0a, 0x0f, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0f, 0x66, 0x6c, 0x75, 0x73, + 0x68, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x49, 0x0a, 0x10, 0x64, + 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, + 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0f, 0x64, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x30, 0x0a, 0x13, 0x75, 0x6e, 0x66, 0x6c, 0x75, 0x73, + 0x68, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x73, 0x18, 0x07, 0x20, + 0x03, 0x28, 0x03, 0x52, 0x13, 0x75, 0x6e, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x73, 0x12, 0x2c, 0x0a, 0x11, 0x66, 0x6c, 0x75, 0x73, + 0x68, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x73, 0x18, 0x08, 0x20, + 0x03, 0x28, 0x03, 0x52, 0x11, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x49, 0x64, 0x73, 0x12, 0x2d, 0x0a, 0x12, 0x64, 0x72, 0x6f, 0x70, 0x70, 0x65, + 0x64, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x73, 0x18, 0x09, 0x20, 0x03, + 0x28, 0x03, 0x52, 0x11, 0x64, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x64, 0x73, 0x12, 0x2d, 0x0a, 0x12, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x64, + 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, + 0x03, 0x52, 0x11, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x64, 0x73, 0x12, 0x49, 0x0a, 0x10, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x64, 0x5f, + 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x0b, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0f, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, + 0x33, 0x0a, 0x16, 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x5f, 0x7a, 0x65, 0x72, 0x6f, 0x5f, 0x73, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x0c, 0x20, 0x03, 0x28, 0x03, 0x52, + 0x13, 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x5a, 0x65, 0x72, 0x6f, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x64, 0x73, 0x12, 0x75, 0x0a, 0x18, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, + 0x18, 0x0d, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x63, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x16, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, + 0x61, 0x74, 0x73, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x49, 0x0a, 0x1b, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x56, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x89, 0x01, 0x0a, 0x16, 0x57, 0x61, 0x74, 0x63, 0x68, + 0x44, 0x6d, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, + 0x61, 0x73, 0x65, 0x12, 0x3d, 0x0a, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x63, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 0x73, 0x22, 0xc2, 0x01, 0x0a, 0x14, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, + 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x12, 0x0a, + 0x04, 0x64, 0x62, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, + 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x49, 0x44, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, 0x20, 0x0a, 0x0b, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, + 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x78, 0x0a, 0x0a, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x4d, 0x73, 0x67, 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, 0x38, 0x0a, 0x07, 0x73, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x07, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x22, 0x9b, 0x0b, 0x0a, 0x0b, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, + 0x6f, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x49, + 0x44, 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, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x73, 0x65, 0x72, + 0x74, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0d, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x1e, + 0x0a, 0x0b, 0x6e, 0x75, 0x6d, 0x5f, 0x6f, 0x66, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x09, 0x6e, 0x75, 0x6d, 0x4f, 0x66, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x37, + 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1e, 0x0a, 0x0b, 0x6d, 0x61, 0x78, 0x5f, 0x72, + 0x6f, 0x77, 0x5f, 0x6e, 0x75, 0x6d, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x6d, 0x61, + 0x78, 0x52, 0x6f, 0x77, 0x4e, 0x75, 0x6d, 0x12, 0x28, 0x0a, 0x10, 0x6c, 0x61, 0x73, 0x74, 0x5f, + 0x65, 0x78, 0x70, 0x69, 0x72, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, + 0x04, 0x52, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x45, 0x78, 0x70, 0x69, 0x72, 0x65, 0x54, 0x69, 0x6d, + 0x65, 0x12, 0x44, 0x0a, 0x0e, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x4d, 0x73, 0x67, + 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0d, 0x73, 0x74, 0x61, 0x72, 0x74, 0x50, + 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x40, 0x0a, 0x0c, 0x64, 0x6d, 0x6c, 0x5f, 0x70, + 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, + 0x2e, 0x4d, 0x73, 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0b, 0x64, 0x6d, + 0x6c, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x38, 0x0a, 0x07, 0x62, 0x69, 0x6e, + 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x0b, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, + 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x07, 0x62, 0x69, 0x6e, 0x6c, + 0x6f, 0x67, 0x73, 0x12, 0x3c, 0x0a, 0x09, 0x73, 0x74, 0x61, 0x74, 0x73, 0x6c, 0x6f, 0x67, 0x73, + 0x18, 0x0c, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, + 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x73, 0x6c, 0x6f, 0x67, + 0x73, 0x12, 0x3c, 0x0a, 0x09, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x0d, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, + 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x09, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x73, 0x12, + 0x30, 0x0a, 0x13, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x42, 0x79, 0x43, 0x6f, 0x6d, 0x70, + 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x08, 0x52, 0x13, 0x63, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x64, 0x42, 0x79, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x12, 0x26, 0x0a, 0x0e, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x46, + 0x72, 0x6f, 0x6d, 0x18, 0x0f, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0e, 0x63, 0x6f, 0x6d, 0x70, 0x61, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x72, 0x6f, 0x6d, 0x12, 0x1d, 0x0a, 0x0a, 0x64, 0x72, 0x6f, + 0x70, 0x70, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x10, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x64, + 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x41, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x69, 0x73, 0x5f, 0x69, + 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x18, 0x11, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, + 0x69, 0x73, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x12, 0x17, 0x0a, 0x07, 0x69, + 0x73, 0x5f, 0x66, 0x61, 0x6b, 0x65, 0x18, 0x12, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x69, 0x73, + 0x46, 0x61, 0x6b, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x65, + 0x64, 0x18, 0x13, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, + 0x65, 0x64, 0x12, 0x35, 0x0a, 0x05, 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x18, 0x14, 0x20, 0x01, 0x28, + 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4c, 0x65, 0x76, + 0x65, 0x6c, 0x52, 0x05, 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x74, 0x6f, + 0x72, 0x61, 0x67, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x15, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x0e, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x12, 0x36, 0x0a, 0x17, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x16, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x15, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, + 0x61, 0x74, 0x73, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x3e, 0x0a, 0x0a, 0x6c, 0x61, + 0x73, 0x74, 0x5f, 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x18, 0x17, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, + 0x09, 0x6c, 0x61, 0x73, 0x74, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, 0x3f, 0x0a, 0x1c, 0x6c, 0x61, + 0x73, 0x74, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, + 0x74, 0x73, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x18, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x19, 0x6c, 0x61, 0x73, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, + 0x74, 0x61, 0x74, 0x73, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1b, 0x0a, 0x09, 0x69, + 0x73, 0x5f, 0x73, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x18, 0x19, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, + 0x69, 0x73, 0x53, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x12, 0x57, 0x0a, 0x0d, 0x74, 0x65, 0x78, 0x74, + 0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x18, 0x1a, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x2e, + 0x54, 0x65, 0x78, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x0d, 0x74, 0x65, 0x78, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, + 0x73, 0x12, 0x44, 0x0a, 0x0d, 0x62, 0x6d, 0x32, 0x35, 0x73, 0x74, 0x61, 0x74, 0x73, 0x6c, 0x6f, + 0x67, 0x73, 0x18, 0x1b, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, + 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x0d, 0x62, 0x6d, 0x32, 0x35, 0x73, 0x74, + 0x61, 0x74, 0x73, 0x6c, 0x6f, 0x67, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x69, 0x73, 0x5f, 0x69, 0x6e, + 0x76, 0x69, 0x73, 0x69, 0x62, 0x6c, 0x65, 0x18, 0x1c, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x69, + 0x73, 0x49, 0x6e, 0x76, 0x69, 0x73, 0x69, 0x62, 0x6c, 0x65, 0x1a, 0x63, 0x0a, 0x12, 0x54, 0x65, + 0x78, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, + 0x65, 0x79, 0x12, 0x37, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x65, 0x78, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, + 0x74, 0x61, 0x74, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, + 0x7a, 0x0a, 0x14, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x72, 0x74, 0x50, + 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x44, 0x0a, 0x0e, 0x73, 0x74, 0x61, 0x72, 0x74, + 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, + 0x73, 0x67, 0x2e, 0x4d, 0x73, 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0d, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1c, 0x0a, + 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x22, 0xc1, 0x06, 0x0a, 0x16, + 0x53, 0x61, 0x76, 0x65, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, + 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x4c, 0x0a, 0x11, 0x66, 0x69, + 0x65, 0x6c, 0x64, 0x32, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, 0x18, + 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, + 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x11, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x32, 0x42, 0x69, 0x6e, + 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, 0x12, 0x3f, 0x0a, 0x0b, 0x63, 0x68, 0x65, 0x63, + 0x6b, 0x50, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x50, 0x6f, 0x69, 0x6e, 0x74, 0x52, 0x0b, 0x63, 0x68, + 0x65, 0x63, 0x6b, 0x50, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x12, 0x50, 0x0a, 0x0f, 0x73, 0x74, 0x61, + 0x72, 0x74, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x06, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, + 0x61, 0x72, 0x74, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x73, 0x74, 0x61, + 0x72, 0x74, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x66, + 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x66, 0x6c, + 0x75, 0x73, 0x68, 0x65, 0x64, 0x12, 0x50, 0x0a, 0x13, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x32, 0x53, + 0x74, 0x61, 0x74, 0x73, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, 0x18, 0x08, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, + 0x6f, 0x67, 0x52, 0x13, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x32, 0x53, 0x74, 0x61, 0x74, 0x73, 0x6c, + 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, 0x12, 0x3c, 0x0a, 0x09, 0x64, 0x65, 0x6c, 0x74, 0x61, + 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, + 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x09, 0x64, 0x65, 0x6c, 0x74, + 0x61, 0x6c, 0x6f, 0x67, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x64, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, + 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x64, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x12, + 0x1c, 0x0a, 0x09, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x18, 0x0b, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x09, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x12, 0x18, 0x0a, + 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, + 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x3c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x5f, 0x6c, + 0x65, 0x76, 0x65, 0x6c, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x08, 0x73, 0x65, 0x67, + 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x26, 0x0a, 0x0e, 0x73, 0x74, 0x6f, 0x72, 0x61, + 0x67, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0e, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, + 0x4e, 0x0a, 0x12, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x32, 0x42, 0x6d, 0x32, 0x35, 0x6c, 0x6f, 0x67, + 0x50, 0x61, 0x74, 0x68, 0x73, 0x18, 0x10, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x12, 0x66, 0x69, 0x65, + 0x6c, 0x64, 0x32, 0x42, 0x6d, 0x32, 0x35, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, 0x22, + 0x85, 0x01, 0x0a, 0x0a, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x50, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x1c, + 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x39, 0x0a, 0x08, + 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, + 0x67, 0x2e, 0x4d, 0x73, 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x70, + 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0b, 0x6e, 0x75, 0x6d, 0x5f, 0x6f, + 0x66, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x6e, 0x75, + 0x6d, 0x4f, 0x66, 0x52, 0x6f, 0x77, 0x73, 0x22, 0xcb, 0x01, 0x0a, 0x0c, 0x44, 0x65, 0x6c, 0x74, + 0x61, 0x4c, 0x6f, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x25, 0x0a, 0x0e, 0x72, 0x65, 0x63, 0x6f, + 0x72, 0x64, 0x5f, 0x65, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, + 0x52, 0x0d, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x12, + 0x25, 0x0a, 0x0e, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x66, 0x72, 0x6f, + 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, + 0x6d, 0x70, 0x46, 0x72, 0x6f, 0x6d, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, + 0x61, 0x6d, 0x70, 0x5f, 0x74, 0x6f, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x74, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x54, 0x6f, 0x12, 0x24, 0x0a, 0x0e, 0x64, 0x65, 0x6c, + 0x74, 0x61, 0x5f, 0x6c, 0x6f, 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0c, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x4c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x12, + 0x24, 0x0a, 0x0e, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x5f, 0x6c, 0x6f, 0x67, 0x5f, 0x73, 0x69, 0x7a, + 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x4c, 0x6f, + 0x67, 0x53, 0x69, 0x7a, 0x65, 0x22, 0x83, 0x01, 0x0a, 0x0d, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x3a, 0x0a, 0x05, 0x73, + 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x57, 0x61, 0x74, 0x63, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x22, 0x80, 0x01, 0x0a, 0x0c, + 0x44, 0x61, 0x74, 0x61, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x18, 0x0a, 0x07, + 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, + 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x12, 0x3c, 0x0a, 0x08, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x53, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x52, 0x08, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x22, 0xf6, + 0x03, 0x0a, 0x0e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, + 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, + 0x42, 0x0a, 0x0c, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x73, 0x18, + 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, + 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x0c, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, + 0x6f, 0x67, 0x73, 0x12, 0x1e, 0x0a, 0x0b, 0x6e, 0x75, 0x6d, 0x5f, 0x6f, 0x66, 0x5f, 0x72, 0x6f, + 0x77, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x6e, 0x75, 0x6d, 0x4f, 0x66, 0x52, + 0x6f, 0x77, 0x73, 0x12, 0x3c, 0x0a, 0x09, 0x73, 0x74, 0x61, 0x74, 0x73, 0x6c, 0x6f, 0x67, 0x73, + 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, + 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x73, 0x6c, 0x6f, 0x67, + 0x73, 0x12, 0x3c, 0x0a, 0x09, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x05, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, + 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x09, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x73, 0x12, + 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x43, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x5a, 0x0a, 0x0d, 0x74, 0x65, 0x78, 0x74, 0x53, 0x74, + 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x34, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x73, + 0x2e, 0x54, 0x65, 0x78, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x0d, 0x74, 0x65, 0x78, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, + 0x67, 0x73, 0x1a, 0x63, 0x0a, 0x12, 0x54, 0x65, 0x78, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, + 0x6f, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x37, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x65, + 0x78, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x5c, 0x0a, 0x0b, 0x46, 0x69, 0x65, 0x6c, 0x64, + 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, + 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, + 0x12, 0x33, 0x0a, 0x07, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x19, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x07, 0x62, 0x69, + 0x6e, 0x6c, 0x6f, 0x67, 0x73, 0x22, 0xb0, 0x01, 0x0a, 0x0e, 0x54, 0x65, 0x78, 0x74, 0x49, 0x6e, + 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x65, 0x6c, + 0x64, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, + 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x14, 0x0a, 0x05, + 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x66, 0x69, 0x6c, + 0x65, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x6c, 0x6f, 0x67, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6c, 0x6f, 0x67, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x1f, 0x0a, + 0x0b, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x0a, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x18, + 0x0a, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x22, 0xe0, 0x01, 0x0a, 0x06, 0x42, 0x69, 0x6e, + 0x6c, 0x6f, 0x67, 0x12, 0x1f, 0x0a, 0x0b, 0x65, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x5f, 0x6e, + 0x75, 0x6d, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x65, 0x6e, 0x74, 0x72, 0x69, 0x65, + 0x73, 0x4e, 0x75, 0x6d, 0x12, 0x25, 0x0a, 0x0e, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x5f, 0x66, 0x72, 0x6f, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x74, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x46, 0x72, 0x6f, 0x6d, 0x12, 0x21, 0x0a, 0x0c, 0x74, + 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x74, 0x6f, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x04, 0x52, 0x0b, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x54, 0x6f, 0x12, 0x19, + 0x0a, 0x08, 0x6c, 0x6f, 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x07, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x12, 0x19, 0x0a, 0x08, 0x6c, 0x6f, 0x67, + 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6c, 0x6f, 0x67, + 0x53, 0x69, 0x7a, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x6c, 0x6f, 0x67, 0x49, 0x44, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x05, 0x6c, 0x6f, 0x67, 0x49, 0x44, 0x12, 0x1f, 0x0a, 0x0b, 0x6d, 0x65, + 0x6d, 0x6f, 0x72, 0x79, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0a, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65, 0x22, 0xc8, 0x01, 0x0a, 0x17, + 0x47, 0x65, 0x74, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x3b, 0x0a, 0x08, + 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x56, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, + 0x08, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x3b, 0x0a, 0x07, 0x62, 0x69, 0x6e, + 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x73, 0x52, 0x07, 0x62, + 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x73, 0x22, 0x90, 0x01, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x52, 0x65, + 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 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, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x22, 0xc9, 0x01, 0x0a, 0x19, 0x47, 0x65, + 0x74, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x56, 0x32, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x3b, 0x0a, 0x08, + 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x56, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, + 0x08, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x3a, 0x0a, 0x08, 0x73, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x73, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x94, 0x01, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x52, 0x65, 0x63, + 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x56, 0x32, 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, 0x22, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0c, + 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x22, 0x6d, 0x0a, 0x1d, + 0x47, 0x65, 0x74, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, + 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 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, + 0x1a, 0x0a, 0x08, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x08, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x22, 0xc9, 0x01, 0x0a, 0x1e, + 0x47, 0x65, 0x74, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, + 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, + 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x12, 0x33, 0x0a, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, + 0x66, 0x6f, 0x52, 0x04, 0x69, 0x6e, 0x66, 0x6f, 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, 0x22, 0xcf, 0x01, 0x0a, 0x1a, 0x47, 0x65, 0x74, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x42, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 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, 0x20, 0x0a, 0x0b, + 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x39, + 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0e, 0x32, 0x21, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x22, 0x6e, 0x0a, 0x1b, 0x47, 0x65, 0x74, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x42, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x1a, 0x0a, + 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, + 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0xbf, 0x01, 0x0a, 0x19, 0x47, 0x65, + 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 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, 0x20, 0x0a, + 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, + 0x2a, 0x0a, 0x10, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x55, 0x6e, 0x68, 0x65, 0x61, 0x6c, + 0x74, 0x68, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x69, 0x6e, 0x63, 0x6c, 0x75, + 0x64, 0x65, 0x55, 0x6e, 0x68, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x79, 0x22, 0x6d, 0x0a, 0x1a, 0x47, + 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x1a, + 0x0a, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, + 0x52, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x86, 0x01, 0x0a, 0x18, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x43, 0x6f, 0x6d, 0x70, 0x6c, + 0x65, 0x74, 0x65, 0x64, 0x4d, 0x73, 0x67, 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, 0x38, 0x0a, 0x07, 0x73, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x07, 0x73, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x22, 0xf3, 0x02, 0x0a, 0x10, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x57, + 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x35, 0x0a, 0x05, 0x76, 0x63, 0x68, 0x61, + 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x63, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x05, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x12, + 0x18, 0x0a, 0x07, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x07, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x73, 0x12, 0x3a, 0x0a, 0x05, 0x73, 0x74, 0x61, + 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x57, 0x61, 0x74, 0x63, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, + 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, + 0x54, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, + 0x74, 0x54, 0x73, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x05, 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, 0x1a, 0x0a, 0x08, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18, 0x06, + 0x20, 0x01, 0x28, 0x05, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x12, + 0x0a, 0x04, 0x6f, 0x70, 0x49, 0x44, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x6f, 0x70, + 0x49, 0x44, 0x12, 0x45, 0x0a, 0x0c, 0x64, 0x62, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, + 0x65, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, + 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0c, 0x64, 0x62, 0x50, + 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x22, 0x62, 0x0a, 0x16, 0x43, 0x6f, 0x6d, + 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 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, 0x16, 0x0a, 0x06, 0x70, 0x6c, 0x61, 0x6e, 0x49, 0x44, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x70, 0x6c, 0x61, 0x6e, 0x49, 0x44, 0x22, 0x82, 0x02, + 0x0a, 0x0f, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, + 0x6f, 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, + 0x12, 0x40, 0x0a, 0x0c, 0x70, 0x6b, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x6c, 0x6f, 0x67, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, + 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x0a, 0x70, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, + 0x6f, 0x67, 0x12, 0x37, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x0e, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x35, 0x0a, 0x05, 0x6c, + 0x65, 0x76, 0x65, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x05, 0x6c, 0x65, 0x76, + 0x65, 0x6c, 0x12, 0x1e, 0x0a, 0x0b, 0x6e, 0x75, 0x6d, 0x5f, 0x6f, 0x66, 0x5f, 0x72, 0x6f, 0x77, + 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x6e, 0x75, 0x6d, 0x4f, 0x66, 0x52, 0x6f, + 0x77, 0x73, 0x22, 0x85, 0x04, 0x0a, 0x13, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x70, 0x6c, + 0x61, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x70, 0x6c, 0x61, 0x6e, + 0x49, 0x44, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x65, 0x64, 0x5f, + 0x74, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x63, + 0x74, 0x65, 0x64, 0x54, 0x6f, 0x12, 0x1e, 0x0a, 0x0b, 0x6e, 0x75, 0x6d, 0x5f, 0x6f, 0x66, 0x5f, + 0x72, 0x6f, 0x77, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x6e, 0x75, 0x6d, 0x4f, + 0x66, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, + 0x65, 0x64, 0x5f, 0x66, 0x72, 0x6f, 0x6d, 0x18, 0x04, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0d, 0x63, + 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x65, 0x64, 0x46, 0x72, 0x6f, 0x6d, 0x12, 0x3d, 0x0a, 0x0a, + 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, + 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x63, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0b, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x21, + 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x07, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x69, 0x64, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x5d, 0x0a, 0x0d, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x38, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, + 0x6f, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x1a, 0x63, 0x0a, 0x11, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x38, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x53, 0x79, 0x6e, 0x63, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xcd, 0x03, 0x0a, 0x18, 0x43, + 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x42, 0x0a, 0x0c, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, + 0x6e, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x0c, 0x66, 0x69, 0x65, + 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x73, 0x12, 0x50, 0x0a, 0x13, 0x66, 0x69, 0x65, + 0x6c, 0x64, 0x32, 0x53, 0x74, 0x61, 0x74, 0x73, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, + 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, + 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x13, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x32, 0x53, 0x74, + 0x61, 0x74, 0x73, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, 0x12, 0x3c, 0x0a, 0x09, 0x64, + 0x65, 0x6c, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x09, + 0x64, 0x65, 0x6c, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x73, + 0x65, 0x72, 0x74, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0d, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, + 0x12, 0x35, 0x0a, 0x05, 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4c, 0x65, 0x76, 0x65, 0x6c, + 0x52, 0x05, 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1b, 0x0a, + 0x09, 0x69, 0x73, 0x5f, 0x73, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x08, 0x69, 0x73, 0x53, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x22, 0xf3, 0x06, 0x0a, 0x0e, 0x43, + 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, 0x12, 0x16, 0x0a, + 0x06, 0x70, 0x6c, 0x61, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x70, + 0x6c, 0x61, 0x6e, 0x49, 0x44, 0x12, 0x53, 0x0a, 0x0e, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2b, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x73, 0x52, 0x0e, 0x73, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x74, + 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x74, 0x69, 0x6d, + 0x65, 0x6f, 0x75, 0x74, 0x5f, 0x69, 0x6e, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x10, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x49, 0x6e, + 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x35, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x1e, + 0x0a, 0x0a, 0x74, 0x69, 0x6d, 0x65, 0x74, 0x72, 0x61, 0x76, 0x65, 0x6c, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x04, 0x52, 0x0a, 0x74, 0x69, 0x6d, 0x65, 0x74, 0x72, 0x61, 0x76, 0x65, 0x6c, 0x12, 0x18, + 0x0a, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x74, 0x6c, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x74, 0x6c, 0x12, + 0x1d, 0x0a, 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x09, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x3d, + 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x0a, 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, 0x30, 0x0a, + 0x14, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x6b, 0x65, 0x79, 0x5f, + 0x66, 0x69, 0x65, 0x6c, 0x64, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x03, 0x52, 0x12, 0x63, 0x6c, 0x75, + 0x73, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4b, 0x65, 0x79, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x12, + 0x28, 0x0a, 0x10, 0x6d, 0x61, 0x78, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x72, + 0x6f, 0x77, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x6d, 0x61, 0x78, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x2e, 0x0a, 0x13, 0x70, 0x72, 0x65, + 0x66, 0x65, 0x72, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x72, 0x6f, 0x77, 0x73, + 0x18, 0x0d, 0x20, 0x01, 0x28, 0x03, 0x52, 0x11, 0x70, 0x72, 0x65, 0x66, 0x65, 0x72, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x2e, 0x0a, 0x13, 0x61, 0x6e, 0x61, + 0x6c, 0x79, 0x7a, 0x65, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x70, 0x61, 0x74, 0x68, + 0x18, 0x0e, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, + 0x65, 0x73, 0x75, 0x6c, 0x74, 0x50, 0x61, 0x74, 0x68, 0x12, 0x2e, 0x0a, 0x13, 0x61, 0x6e, 0x61, + 0x6c, 0x79, 0x7a, 0x65, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x73, + 0x18, 0x0f, 0x20, 0x03, 0x28, 0x03, 0x52, 0x11, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, + 0x74, 0x65, 0x18, 0x10, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, + 0x1f, 0x0a, 0x0b, 0x62, 0x65, 0x67, 0x69, 0x6e, 0x5f, 0x6c, 0x6f, 0x67, 0x49, 0x44, 0x18, 0x11, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x62, 0x65, 0x67, 0x69, 0x6e, 0x4c, 0x6f, 0x67, 0x49, 0x44, + 0x12, 0x54, 0x0a, 0x18, 0x70, 0x72, 0x65, 0x5f, 0x61, 0x6c, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x65, + 0x64, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x12, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x44, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x16, + 0x70, 0x72, 0x65, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x6c, 0x6f, 0x74, 0x5f, 0x75, + 0x73, 0x61, 0x67, 0x65, 0x18, 0x13, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x6c, 0x6f, 0x74, + 0x55, 0x73, 0x61, 0x67, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x6d, 0x61, 0x78, 0x5f, 0x73, 0x69, 0x7a, + 0x65, 0x18, 0x14, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6d, 0x61, 0x78, 0x53, 0x69, 0x7a, 0x65, + 0x22, 0xad, 0x03, 0x0a, 0x11, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x70, 0x6c, 0x61, 0x6e, 0x49, 0x44, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x70, 0x6c, 0x61, 0x6e, 0x49, 0x44, 0x12, 0x1c, + 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x1e, 0x0a, 0x0b, + 0x6e, 0x75, 0x6d, 0x5f, 0x6f, 0x66, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x09, 0x6e, 0x75, 0x6d, 0x4f, 0x66, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x3f, 0x0a, 0x0b, + 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, + 0x67, 0x52, 0x0a, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x50, 0x0a, + 0x13, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x32, 0x53, 0x74, 0x61, 0x74, 0x73, 0x6c, 0x6f, 0x67, 0x50, + 0x61, 0x74, 0x68, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, + 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x13, 0x66, 0x69, 0x65, 0x6c, + 0x64, 0x32, 0x53, 0x74, 0x61, 0x74, 0x73, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, 0x12, + 0x3c, 0x0a, 0x09, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x06, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, + 0x6f, 0x67, 0x52, 0x09, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x73, 0x12, 0x18, 0x0a, + 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, + 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x73, 0x6f, + 0x72, 0x74, 0x65, 0x64, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x53, 0x6f, + 0x72, 0x74, 0x65, 0x64, 0x12, 0x3a, 0x0a, 0x08, 0x62, 0x6d, 0x32, 0x35, 0x6c, 0x6f, 0x67, 0x73, + 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, + 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x08, 0x62, 0x6d, 0x32, 0x35, 0x6c, 0x6f, 0x67, 0x73, + 0x22, 0xff, 0x01, 0x0a, 0x14, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, + 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x70, 0x6c, 0x61, + 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x70, 0x6c, 0x61, 0x6e, 0x49, + 0x44, 0x12, 0x3c, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, + 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, + 0x40, 0x0a, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x73, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x35, 0x0a, 0x04, 0x74, + 0x79, 0x70, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x6f, + 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, + 0x70, 0x65, 0x22, 0x91, 0x01, 0x0a, 0x17, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, + 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x12, 0x41, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x02, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x07, 0x72, + 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x53, 0x0a, 0x16, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x4d, 0x65, 0x74, 0x61, + 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x1f, 0x0a, 0x0b, 0x62, 0x69, + 0x6e, 0x6c, 0x6f, 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0a, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x22, 0xdb, 0x02, 0x0a, 0x14, + 0x57, 0x61, 0x74, 0x63, 0x68, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, + 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x12, 0x49, 0x0a, 0x0f, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, + 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x44, + 0x61, 0x74, 0x61, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0e, 0x73, 0x74, 0x61, 0x72, 0x74, 0x50, 0x6f, + 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x18, 0x04, 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, 0x29, 0x0a, 0x10, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, + 0x52, 0x0f, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x12, 0x46, 0x0a, 0x0d, 0x64, 0x62, 0x5f, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, + 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, + 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0c, 0x64, 0x62, 0x50, + 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x22, 0x4c, 0x0a, 0x15, 0x57, 0x61, 0x74, + 0x63, 0x68, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, + 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0xa9, 0x01, 0x0a, 0x16, 0x53, 0x65, 0x74, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 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, 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, 0x3e, 0x0a, 0x09, 0x6e, 0x65, 0x77, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x08, 0x6e, 0x65, 0x77, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x22, 0x4e, 0x0a, 0x17, 0x53, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, + 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x22, 0xba, 0x01, 0x0a, 0x19, 0x44, 0x72, 0x6f, 0x70, 0x56, 0x69, 0x72, 0x74, + 0x75, 0x61, 0x6c, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 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, 0x21, 0x0a, 0x0c, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x48, 0x0a, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x72, 0x6f, + 0x70, 0x56, 0x69, 0x72, 0x74, 0x75, 0x61, 0x6c, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, + 0x22, 0xdd, 0x03, 0x0a, 0x19, 0x44, 0x72, 0x6f, 0x70, 0x56, 0x69, 0x72, 0x74, 0x75, 0x61, 0x6c, + 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x1c, + 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 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, 0x4c, 0x0a, 0x11, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x32, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, + 0x50, 0x61, 0x74, 0x68, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x11, 0x66, 0x69, 0x65, + 0x6c, 0x64, 0x32, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, 0x12, 0x50, + 0x0a, 0x13, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x32, 0x53, 0x74, 0x61, 0x74, 0x73, 0x6c, 0x6f, 0x67, + 0x50, 0x61, 0x74, 0x68, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x13, 0x66, 0x69, 0x65, + 0x6c, 0x64, 0x32, 0x53, 0x74, 0x61, 0x74, 0x73, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, + 0x12, 0x3c, 0x0a, 0x09, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x05, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, + 0x6c, 0x6f, 0x67, 0x52, 0x09, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x73, 0x12, 0x43, + 0x0a, 0x0d, 0x73, 0x74, 0x61, 0x72, 0x74, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x4d, 0x73, 0x67, 0x50, 0x6f, 0x73, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0d, 0x73, 0x74, 0x61, 0x72, 0x74, 0x50, 0x6f, 0x73, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x3d, 0x0a, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x50, 0x6f, 0x69, 0x6e, + 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x4d, 0x73, 0x67, 0x50, 0x6f, + 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x50, 0x6f, 0x69, + 0x6e, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x75, 0x6d, 0x4f, 0x66, 0x52, 0x6f, 0x77, 0x73, 0x18, + 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x6e, 0x75, 0x6d, 0x4f, 0x66, 0x52, 0x6f, 0x77, 0x73, + 0x22, 0x51, 0x0a, 0x1a, 0x44, 0x72, 0x6f, 0x70, 0x56, 0x69, 0x72, 0x74, 0x75, 0x61, 0x6c, 0x43, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, + 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x22, 0x8b, 0x01, 0x0a, 0x1e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 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, 0x37, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, + 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, + 0x73, 0x22, 0xf9, 0x01, 0x0a, 0x1e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 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, 0x1a, 0x0a, 0x08, 0x76, 0x43, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x76, 0x43, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x12, 0x39, 0x0a, 0x08, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x4d, 0x73, 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4e, 0x0a, + 0x13, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, + 0x69, 0x6e, 0x74, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x4d, 0x73, + 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x12, 0x63, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x22, 0x4d, 0x0a, + 0x19, 0x52, 0x65, 0x73, 0x65, 0x6e, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, + 0x61, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, + 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, + 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x22, 0x70, 0x0a, 0x1a, + 0x52, 0x65, 0x73, 0x65, 0x6e, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, + 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, + 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x5f, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x74, 0x18, 0x02, 0x20, + 0x03, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x52, 0x65, 0x73, 0x65, 0x6e, 0x74, 0x22, 0x6f, + 0x0a, 0x1a, 0x4d, 0x61, 0x72, 0x6b, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x44, 0x72, + 0x6f, 0x70, 0x70, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 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, 0x1f, + 0x0a, 0x0b, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x02, 0x20, + 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x73, 0x22, + 0x66, 0x0a, 0x14, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, + 0x6e, 0x63, 0x65, 0x4c, 0x6f, 0x63, 0x6b, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, + 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, + 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x22, 0xdf, 0x02, 0x0a, 0x16, 0x41, 0x6c, 0x74, 0x65, + 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x18, 0x02, 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, 0x22, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x12, 0x49, 0x0a, 0x0f, 0x73, 0x74, 0x61, + 0x72, 0x74, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x44, 0x61, 0x74, 0x61, + 0x50, 0x61, 0x69, 0x72, 0x52, 0x0e, 0x73, 0x74, 0x61, 0x72, 0x74, 0x50, 0x6f, 0x73, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x41, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, + 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, + 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0a, 0x70, 0x72, 0x6f, + 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x62, 0x49, 0x44, 0x18, + 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x76, + 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, + 0x76, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x22, 0x5a, 0x0a, 0x10, 0x47, 0x63, 0x43, + 0x6f, 0x6e, 0x66, 0x69, 0x72, 0x6d, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 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, 0x69, 0x0a, 0x11, 0x47, 0x63, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x72, 0x6d, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, + 0x1f, 0x0a, 0x0b, 0x67, 0x63, 0x5f, 0x66, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x67, 0x63, 0x46, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, + 0x22, 0x84, 0x01, 0x0a, 0x1b, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x44, 0x61, 0x74, 0x61, 0x4e, + 0x6f, 0x64, 0x65, 0x54, 0x74, 0x4d, 0x73, 0x67, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 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, 0x33, 0x0a, 0x04, 0x6d, 0x73, 0x67, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x6d, 0x73, 0x67, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x4e, 0x6f, 0x64, 0x65, 0x54, 0x74, 0x4d, 0x73, + 0x67, 0x52, 0x04, 0x6d, 0x73, 0x67, 0x73, 0x22, 0xb7, 0x01, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x46, + 0x6c, 0x75, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, + 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, 0x12, 0x17, 0x0a, 0x07, 0x64, + 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, + 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x22, 0x0a, + 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x44, 0x22, 0x55, 0x0a, 0x18, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x39, 0x0a, + 0x05, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x57, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x66, + 0x6f, 0x52, 0x05, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x22, 0xc3, 0x01, 0x0a, 0x20, 0x43, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x6f, + 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, + 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x6f, 0x70, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x04, 0x6f, 0x70, 0x49, 0x44, 0x12, 0x3a, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 0x57, 0x61, 0x74, 0x63, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, + 0x74, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x05, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x22, 0x86, + 0x03, 0x0a, 0x10, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, + 0x44, 0x12, 0x14, 0x0a, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, + 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, + 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x44, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x76, 0x63, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, + 0x07, 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, 0x44, 0x0a, 0x0c, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x5f, 0x66, + 0x69, 0x6c, 0x65, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, + 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x0b, 0x69, + 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x12, 0x3b, 0x0a, 0x07, 0x6f, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x07, + 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x31, 0x0a, 0x07, 0x49, 0x44, 0x52, 0x61, 0x6e, + 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x62, 0x65, 0x67, 0x69, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x05, 0x62, 0x65, 0x67, 0x69, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x72, 0x0a, 0x14, 0x49, 0x6d, + 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, + 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x44, 0x12, 0x1a, 0x0a, 0x08, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x22, 0x91, + 0x04, 0x0a, 0x0d, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x14, + 0x0a, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x6a, + 0x6f, 0x62, 0x49, 0x44, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, + 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, + 0x12, 0x22, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, + 0x18, 0x05, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x44, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, + 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 0x73, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x07, 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, 0x37, 0x0a, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x46, + 0x69, 0x6c, 0x65, 0x52, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x12, 0x3b, 0x0a, 0x07, 0x6f, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x07, + 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x0e, 0x0a, 0x02, 0x74, 0x73, 0x18, 0x0a, 0x20, + 0x01, 0x28, 0x04, 0x52, 0x02, 0x74, 0x73, 0x12, 0x35, 0x0a, 0x08, 0x49, 0x44, 0x5f, 0x72, 0x61, + 0x6e, 0x67, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x44, + 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x07, 0x49, 0x44, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x52, + 0x0a, 0x10, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x73, 0x18, 0x0c, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6d, 0x70, + 0x6f, 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x52, 0x0f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x73, 0x22, 0x63, 0x0a, 0x15, 0x51, 0x75, 0x65, 0x72, 0x79, 0x50, 0x72, 0x65, 0x49, 0x6d, + 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, + 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, + 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x14, 0x0a, 0x05, 0x6a, 0x6f, 0x62, + 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x12, + 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x22, 0xf1, 0x02, 0x0a, 0x14, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, + 0x12, 0x61, 0x0a, 0x0e, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x6f, + 0x77, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x53, 0x74, 0x61, 0x74, + 0x73, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x6f, 0x77, 0x73, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0d, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x6f, 0x77, 0x73, 0x12, 0x6e, 0x0a, 0x13, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x3e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6d, + 0x70, 0x6f, 0x72, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x53, 0x69, 0x7a, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x52, 0x11, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x53, + 0x69, 0x7a, 0x65, 0x1a, 0x40, 0x0a, 0x12, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x6f, 0x77, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x44, 0x0a, 0x16, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x53, 0x69, 0x7a, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, + 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, + 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xfe, 0x02, 0x0a, 0x0f, + 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, + 0x42, 0x0a, 0x0b, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, + 0x6f, 0x72, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x0a, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x46, + 0x69, 0x6c, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x66, 0x69, 0x6c, 0x65, 0x53, 0x69, 0x7a, 0x65, + 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x52, 0x6f, 0x77, 0x73, 0x12, + 0x2a, 0x0a, 0x11, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x5f, + 0x73, 0x69, 0x7a, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0f, 0x74, 0x6f, 0x74, 0x61, + 0x6c, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x56, 0x0a, 0x0c, 0x68, + 0x61, 0x73, 0x68, 0x65, 0x64, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x33, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x46, 0x69, 0x6c, 0x65, + 0x53, 0x74, 0x61, 0x74, 0x73, 0x2e, 0x48, 0x61, 0x73, 0x68, 0x65, 0x64, 0x53, 0x74, 0x61, 0x74, + 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0b, 0x68, 0x61, 0x73, 0x68, 0x65, 0x64, 0x53, 0x74, + 0x61, 0x74, 0x73, 0x1a, 0x67, 0x0a, 0x10, 0x48, 0x61, 0x73, 0x68, 0x65, 0x64, 0x53, 0x74, 0x61, + 0x74, 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, 0x3d, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x53, 0x74, 0x61, 0x74, + 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x92, 0x02, 0x0a, + 0x16, 0x51, 0x75, 0x65, 0x72, 0x79, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x16, 0x0a, 0x06, + 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, + 0x73, 0x6b, 0x49, 0x44, 0x12, 0x3a, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, + 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x65, 0x56, 0x32, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, + 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x6c, 0x6f, 0x74, + 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x73, 0x6c, 0x6f, 0x74, 0x73, 0x12, 0x41, + 0x0a, 0x0a, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x06, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x46, 0x69, 0x6c, + 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, + 0x73, 0x22, 0x7e, 0x0a, 0x12, 0x51, 0x75, 0x65, 0x72, 0x79, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, + 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, + 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x14, 0x0a, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x12, 0x16, 0x0a, 0x06, 0x74, + 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, + 0x6b, 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x71, 0x75, 0x65, 0x72, 0x79, 0x53, 0x6c, 0x6f, 0x74, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x71, 0x75, 0x65, 0x72, 0x79, 0x53, 0x6c, 0x6f, + 0x74, 0x22, 0x8c, 0x02, 0x0a, 0x11, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x65, + 0x64, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x69, 0x6d, + 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x38, 0x0a, 0x07, 0x62, 0x69, + 0x6e, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x07, 0x62, 0x69, 0x6e, + 0x6c, 0x6f, 0x67, 0x73, 0x12, 0x3c, 0x0a, 0x09, 0x73, 0x74, 0x61, 0x74, 0x73, 0x6c, 0x6f, 0x67, + 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, + 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x73, 0x6c, 0x6f, + 0x67, 0x73, 0x12, 0x3c, 0x0a, 0x09, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x73, 0x18, + 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, + 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x09, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x73, + 0x22, 0xa4, 0x02, 0x0a, 0x13, 0x51, 0x75, 0x65, 0x72, 0x79, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x16, 0x0a, + 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, + 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x3a, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, + 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x65, 0x56, 0x32, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, + 0x65, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x6c, 0x6f, + 0x74, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x73, 0x6c, 0x6f, 0x74, 0x73, 0x12, + 0x56, 0x0a, 0x14, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x73, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, + 0x6e, 0x66, 0x6f, 0x52, 0x12, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x73, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x5f, 0x0a, 0x11, 0x44, 0x72, 0x6f, 0x70, 0x49, + 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, + 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x14, 0x0a, 0x05, 0x6a, 0x6f, + 0x62, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, + 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x22, 0xfe, 0x04, 0x0a, 0x09, 0x49, 0x6d, 0x70, + 0x6f, 0x72, 0x74, 0x4a, 0x6f, 0x62, 0x12, 0x14, 0x0a, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x12, 0x12, 0x0a, 0x04, + 0x64, 0x62, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x44, + 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x22, 0x0a, + 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x05, 0x20, + 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, + 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x06, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, + 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x07, 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, 0x1d, + 0x0a, 0x0a, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x5f, 0x74, 0x73, 0x18, 0x08, 0x20, 0x01, + 0x28, 0x04, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x54, 0x73, 0x12, 0x1d, 0x0a, + 0x0a, 0x63, 0x6c, 0x65, 0x61, 0x6e, 0x75, 0x70, 0x5f, 0x74, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, + 0x04, 0x52, 0x09, 0x63, 0x6c, 0x65, 0x61, 0x6e, 0x75, 0x70, 0x54, 0x73, 0x12, 0x2c, 0x0a, 0x11, + 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x44, 0x69, 0x73, 0x6b, 0x53, 0x69, 0x7a, + 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x03, 0x52, 0x11, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x65, 0x64, 0x44, 0x69, 0x73, 0x6b, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x3b, 0x0a, 0x05, 0x73, 0x74, + 0x61, 0x74, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, + 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, + 0x6e, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, + 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x18, 0x0d, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, + 0x54, 0x69, 0x6d, 0x65, 0x12, 0x37, 0x0a, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x0e, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, + 0x72, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x12, 0x3b, 0x0a, + 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x0f, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, + 0x72, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x74, + 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x10, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x22, 0xd8, 0x02, 0x0a, 0x0d, 0x50, 0x72, + 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x14, 0x0a, 0x05, 0x6a, + 0x6f, 0x62, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, + 0x44, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x16, 0x0a, + 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, + 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x3a, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x07, + 0x20, 0x01, 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, + 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x65, 0x56, 0x32, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, + 0x65, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x41, 0x0a, 0x0a, 0x66, 0x69, 0x6c, + 0x65, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, + 0x73, 0x52, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x21, 0x0a, 0x0c, + 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x0b, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x12, + 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, + 0x54, 0x69, 0x6d, 0x65, 0x22, 0xa2, 0x03, 0x0a, 0x0c, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, + 0x61, 0x73, 0x6b, 0x56, 0x32, 0x12, 0x14, 0x0a, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x12, 0x16, 0x0a, 0x06, 0x74, + 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, + 0x6b, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, + 0x44, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, + 0x3a, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x24, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x56, 0x32, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x72, + 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x61, + 0x73, 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x5f, + 0x74, 0x69, 0x6d, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6d, 0x70, + 0x6c, 0x65, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x41, 0x0a, 0x0a, 0x66, 0x69, 0x6c, 0x65, + 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, + 0x52, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x10, 0x73, + 0x74, 0x61, 0x74, 0x73, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, + 0x0a, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x22, 0xb7, 0x01, 0x0a, 0x10, 0x47, 0x63, + 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 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, 0x36, 0x0a, 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0e, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x63, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, + 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x39, 0x0a, 0x06, 0x70, 0x61, 0x72, 0x61, + 0x6d, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, + 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x06, 0x70, 0x61, 0x72, + 0x61, 0x6d, 0x73, 0x22, 0x12, 0x0a, 0x10, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x6c, 0x6f, 0x74, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x65, 0x0a, 0x11, 0x51, 0x75, 0x65, 0x72, 0x79, + 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, + 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x6e, 0x75, 0x6d, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x73, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x6e, 0x75, 0x6d, 0x53, 0x6c, 0x6f, 0x74, 0x73, 0x22, 0xa0, + 0x09, 0x0a, 0x0e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x73, + 0x6b, 0x12, 0x16, 0x0a, 0x06, 0x70, 0x6c, 0x61, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x06, 0x70, 0x6c, 0x61, 0x6e, 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x72, 0x69, + 0x67, 0x67, 0x65, 0x72, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x72, + 0x69, 0x67, 0x67, 0x65, 0x72, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x18, 0x0a, + 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, + 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x35, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, + 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x3c, + 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x26, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x73, 0x6b, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x0b, + 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0a, 0x66, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x1d, 0x0a, + 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x19, 0x0a, 0x08, + 0x65, 0x6e, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, + 0x65, 0x6e, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x74, 0x69, 0x6d, 0x65, 0x6f, + 0x75, 0x74, 0x5f, 0x69, 0x6e, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0b, 0x20, + 0x01, 0x28, 0x05, 0x52, 0x10, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x49, 0x6e, 0x53, 0x65, + 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x65, 0x74, 0x72, 0x79, 0x5f, 0x74, + 0x69, 0x6d, 0x65, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x72, 0x65, 0x74, 0x72, + 0x79, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x74, 0x6c, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, + 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x74, 0x6c, 0x12, 0x1d, 0x0a, + 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x0e, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x09, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x24, 0x0a, 0x0d, + 0x69, 0x6e, 0x70, 0x75, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x0f, 0x20, + 0x03, 0x28, 0x03, 0x52, 0x0d, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x73, 0x12, 0x26, 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x73, 0x18, 0x10, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0e, 0x72, 0x65, 0x73, 0x75, + 0x6c, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x2f, 0x0a, 0x03, 0x70, 0x6f, + 0x73, 0x18, 0x11, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x4d, 0x73, 0x67, 0x50, 0x6f, + 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x03, 0x70, 0x6f, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x6e, + 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x12, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, + 0x65, 0x49, 0x44, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x13, 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, 0x52, 0x0a, 0x14, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, + 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x18, 0x14, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x52, 0x12, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4b, 0x65, + 0x79, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x12, 0x28, 0x0a, 0x10, 0x6d, 0x61, 0x78, 0x5f, 0x73, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x15, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x0e, 0x6d, 0x61, 0x78, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x6f, 0x77, 0x73, + 0x12, 0x2e, 0x0a, 0x13, 0x70, 0x72, 0x65, 0x66, 0x65, 0x72, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x16, 0x20, 0x01, 0x28, 0x03, 0x52, 0x11, 0x70, + 0x72, 0x65, 0x66, 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x6f, 0x77, 0x73, + 0x12, 0x24, 0x0a, 0x0d, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x49, + 0x44, 0x18, 0x17, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, + 0x54, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x26, 0x0a, 0x0e, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, + 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x18, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, + 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x2e, + 0x0a, 0x12, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x74, 0x61, 0x72, 0x74, + 0x54, 0x69, 0x6d, 0x65, 0x18, 0x19, 0x20, 0x01, 0x28, 0x03, 0x52, 0x12, 0x6c, 0x61, 0x73, 0x74, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x19, + 0x0a, 0x08, 0x6d, 0x61, 0x78, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x1a, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x07, 0x6d, 0x61, 0x78, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x74, 0x6d, 0x70, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x1b, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0b, + 0x74, 0x6d, 0x70, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x54, 0x0a, 0x18, 0x70, + 0x72, 0x65, 0x5f, 0x61, 0x6c, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x1c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x49, 0x44, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x16, 0x70, 0x72, 0x65, 0x41, 0x6c, + 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, + 0x73, 0x22, 0xf6, 0x01, 0x0a, 0x12, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, + 0x74, 0x61, 0x74, 0x73, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, + 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, + 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1a, + 0x0a, 0x08, 0x76, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x76, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, + 0x44, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x44, 0x73, 0x12, 0x24, 0x0a, 0x0d, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x54, + 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x61, 0x6e, 0x61, + 0x6c, 0x79, 0x7a, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, + 0x6d, 0x6d, 0x69, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, + 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x22, 0x33, 0x0a, 0x19, 0x44, 0x72, + 0x6f, 0x70, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x70, 0x6c, 0x61, 0x6e, 0x49, + 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x70, 0x6c, 0x61, 0x6e, 0x49, 0x44, 0x2a, + 0x3e, 0x0a, 0x0b, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x07, + 0x0a, 0x03, 0x4e, 0x65, 0x77, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x4e, 0x6f, 0x72, 0x6d, 0x61, + 0x6c, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x10, 0x02, + 0x12, 0x0d, 0x0a, 0x09, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x65, 0x64, 0x10, 0x03, 0x2a, + 0x32, 0x0a, 0x0c, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, + 0x0a, 0x0a, 0x06, 0x4c, 0x65, 0x67, 0x61, 0x63, 0x79, 0x10, 0x00, 0x12, 0x06, 0x0a, 0x02, 0x4c, + 0x30, 0x10, 0x01, 0x12, 0x06, 0x0a, 0x02, 0x4c, 0x31, 0x10, 0x02, 0x12, 0x06, 0x0a, 0x02, 0x4c, + 0x32, 0x10, 0x03, 0x2a, 0x99, 0x01, 0x0a, 0x11, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x57, + 0x61, 0x74, 0x63, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x55, 0x6e, 0x63, + 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x43, 0x6f, 0x6d, + 0x70, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x54, 0x6f, 0x57, 0x61, 0x74, + 0x63, 0x68, 0x10, 0x02, 0x12, 0x10, 0x0a, 0x0c, 0x57, 0x61, 0x74, 0x63, 0x68, 0x53, 0x75, 0x63, + 0x63, 0x65, 0x73, 0x73, 0x10, 0x03, 0x12, 0x10, 0x0a, 0x0c, 0x57, 0x61, 0x74, 0x63, 0x68, 0x46, + 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x54, 0x6f, 0x52, 0x65, + 0x6c, 0x65, 0x61, 0x73, 0x65, 0x10, 0x05, 0x12, 0x12, 0x0a, 0x0e, 0x52, 0x65, 0x6c, 0x65, 0x61, + 0x73, 0x65, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x10, 0x06, 0x12, 0x12, 0x0a, 0x0e, 0x52, + 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x10, 0x07, 0x2a, + 0xcd, 0x01, 0x0a, 0x0e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, + 0x70, 0x65, 0x12, 0x17, 0x0a, 0x13, 0x55, 0x6e, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x65, 0x64, 0x43, + 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x00, 0x12, 0x13, 0x0a, 0x0f, 0x4d, + 0x65, 0x72, 0x67, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x02, + 0x12, 0x11, 0x0a, 0x0d, 0x4d, 0x69, 0x78, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x10, 0x03, 0x12, 0x14, 0x0a, 0x10, 0x53, 0x69, 0x6e, 0x67, 0x6c, 0x65, 0x43, 0x6f, 0x6d, + 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x04, 0x12, 0x13, 0x0a, 0x0f, 0x4d, 0x69, 0x6e, + 0x6f, 0x72, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x05, 0x12, 0x13, + 0x0a, 0x0f, 0x4d, 0x61, 0x6a, 0x6f, 0x72, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x10, 0x06, 0x12, 0x1a, 0x0a, 0x16, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x30, 0x44, 0x65, 0x6c, + 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x07, 0x12, + 0x18, 0x0a, 0x14, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6d, + 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x08, 0x22, 0x04, 0x08, 0x01, 0x10, 0x01, 0x2a, + 0x55, 0x0a, 0x11, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x56, 0x32, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x0b, + 0x0a, 0x07, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x49, + 0x6e, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x46, + 0x61, 0x69, 0x6c, 0x65, 0x64, 0x10, 0x03, 0x12, 0x0d, 0x0a, 0x09, 0x43, 0x6f, 0x6d, 0x70, 0x6c, + 0x65, 0x74, 0x65, 0x64, 0x10, 0x04, 0x2a, 0x29, 0x0a, 0x09, 0x47, 0x63, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x12, 0x05, 0x0a, 0x01, 0x5f, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x50, 0x61, + 0x75, 0x73, 0x65, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x10, + 0x02, 0x2a, 0xb2, 0x01, 0x0a, 0x13, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x75, 0x6e, 0x6b, + 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6e, 0x67, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x69, 0x6e, 0x67, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, + 0x65, 0x64, 0x10, 0x03, 0x12, 0x0a, 0x0a, 0x06, 0x66, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x10, 0x04, + 0x12, 0x0b, 0x0a, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x10, 0x05, 0x12, 0x0d, 0x0a, + 0x09, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x69, 0x6e, 0x67, 0x10, 0x06, 0x12, 0x0c, 0x0a, 0x08, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x69, 0x6e, 0x67, 0x10, 0x07, 0x12, 0x0b, 0x0a, 0x07, 0x63, 0x6c, + 0x65, 0x61, 0x6e, 0x65, 0x64, 0x10, 0x08, 0x12, 0x0e, 0x0a, 0x0a, 0x6d, 0x65, 0x74, 0x61, 0x5f, + 0x73, 0x61, 0x76, 0x65, 0x64, 0x10, 0x09, 0x12, 0x0d, 0x0a, 0x09, 0x73, 0x74, 0x61, 0x74, 0x69, + 0x73, 0x74, 0x69, 0x63, 0x10, 0x0a, 0x32, 0xcf, 0x29, 0x0a, 0x09, 0x44, 0x61, 0x74, 0x61, 0x43, + 0x6f, 0x6f, 0x72, 0x64, 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, 0x4c, 0x0a, 0x05, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x12, 0x1f, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x61, 0x0a, 0x0c, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, + 0x6c, 0x6c, 0x6f, 0x63, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6d, 0x0a, 0x0f, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, 0x73, 0x73, + 0x69, 0x67, 0x6e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x03, 0x88, 0x02, 0x01, 0x12, 0x67, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, + 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6d, 0x0a, + 0x10, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x73, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x79, 0x0a, 0x14, + 0x47, 0x65, 0x74, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x50, + 0x61, 0x74, 0x68, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x73, 0x65, + 0x72, 0x74, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x73, 0x65, + 0x72, 0x74, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x82, 0x01, 0x0a, 0x17, 0x47, 0x65, 0x74, 0x43, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, + 0x69, 0x63, 0x73, 0x12, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x32, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, + 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7f, 0x0a, 0x16, + 0x47, 0x65, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, + 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, + 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6f, 0x0a, + 0x15, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x43, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x74, + 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5b, + 0x0a, 0x0f, 0x53, 0x61, 0x76, 0x65, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, + 0x73, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x61, 0x76, 0x65, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, + 0x50, 0x61, 0x74, 0x68, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x6a, 0x0a, 0x0f, 0x47, + 0x65, 0x74, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x29, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, + 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, + 0x74, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x70, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x52, 0x65, + 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x56, 0x32, 0x12, 0x2b, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x47, 0x65, 0x74, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x56, 0x32, 0x1a, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, + 0x74, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x56, 0x32, 0x22, 0x00, 0x12, 0x7f, 0x0a, 0x16, 0x47, 0x65, 0x74, + 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, + 0x6e, 0x66, 0x6f, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x73, 0x0a, 0x12, 0x47, 0x65, + 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, + 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x53, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x76, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x42, 0x79, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x42, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x42, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x71, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x46, 0x6c, + 0x75, 0x73, 0x68, 0x41, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x2c, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x41, 0x6c, 0x6c, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x41, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7b, 0x0a, 0x12, 0x53, 0x68, + 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x4d, 0x65, + 0x74, 0x72, 0x69, 0x63, 0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d, + 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x71, 0x0a, 0x10, 0x4d, 0x61, 0x6e, 0x75, + 0x61, 0x6c, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2c, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x77, 0x0a, 0x12, 0x47, + 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x80, 0x01, 0x0a, 0x1b, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, + 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x57, 0x69, 0x74, 0x68, 0x50, + 0x6c, 0x61, 0x6e, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, + 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, + 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, 0x73, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x64, 0x0a, 0x0d, 0x57, 0x61, 0x74, 0x63, 0x68, + 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x61, 0x74, + 0x63, 0x68, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x61, 0x74, 0x63, 0x68, 0x43, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x66, 0x0a, + 0x0d, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x27, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, + 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x73, 0x0a, 0x12, 0x44, 0x72, 0x6f, 0x70, 0x56, 0x69, 0x72, + 0x74, 0x75, 0x61, 0x6c, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x2c, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x44, 0x72, 0x6f, 0x70, 0x56, 0x69, 0x72, 0x74, 0x75, 0x61, 0x6c, 0x43, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x72, + 0x6f, 0x70, 0x56, 0x69, 0x72, 0x74, 0x75, 0x61, 0x6c, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6a, 0x0a, 0x0f, 0x53, 0x65, + 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x29, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x53, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x74, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6b, 0x0a, 0x17, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, + 0x73, 0x12, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x22, 0x00, 0x12, 0x6b, 0x0a, 0x17, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x31, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, + 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, + 0x12, 0x63, 0x0a, 0x13, 0x4d, 0x61, 0x72, 0x6b, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, + 0x44, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x61, 0x72, 0x6b, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x44, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x66, 0x0a, 0x1a, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, + 0x73, 0x74, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x65, 0x64, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x43, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x62, 0x0a, + 0x0b, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x12, 0x27, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x68, 0x65, 0x63, + 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x12, 0x54, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, + 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, + 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x52, 0x0a, 0x0a, 0x41, 0x6c, 0x74, 0x65, 0x72, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x41, 0x6c, 0x74, 0x65, 0x72, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x66, 0x0a, 0x0d, 0x47, + 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x28, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, + 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x7b, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x2f, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, + 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x64, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, + 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, + 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, + 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x50, 0x0a, 0x09, 0x44, 0x72, 0x6f, 0x70, 0x49, 0x6e, + 0x64, 0x65, 0x78, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x49, 0x6e, 0x64, + 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x66, 0x0a, 0x0d, 0x44, 0x65, 0x73, 0x63, + 0x72, 0x69, 0x62, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, + 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, + 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x75, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, + 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, + 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, + 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7e, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x49, 0x6e, + 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, + 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, + 0x69, 0x6c, 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, + 0x42, 0x75, 0x69, 0x6c, 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x0b, 0x4c, 0x69, 0x73, 0x74, 0x49, + 0x6e, 0x64, 0x65, 0x78, 0x65, 0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4c, 0x69, 0x73, 0x74, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x58, 0x0a, 0x09, 0x47, 0x63, 0x43, + 0x6f, 0x6e, 0x66, 0x69, 0x72, 0x6d, 0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x63, 0x43, 0x6f, 0x6e, + 0x66, 0x69, 0x72, 0x6d, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x47, 0x63, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x72, 0x6d, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x65, 0x0a, 0x14, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x44, 0x61, 0x74, + 0x61, 0x4e, 0x6f, 0x64, 0x65, 0x54, 0x74, 0x4d, 0x73, 0x67, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x44, 0x61, 0x74, 0x61, 0x4e, 0x6f, 0x64, 0x65, 0x54, 0x74, + 0x4d, 0x73, 0x67, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x4f, 0x0a, 0x09, 0x47, 0x63, + 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x63, 0x43, 0x6f, + 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x61, 0x0a, 0x08, 0x49, + 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x56, 0x32, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, + 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x6e, 0x74, + 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x1a, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, + 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x78, + 0x0a, 0x11, 0x47, 0x65, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x72, 0x6f, 0x67, 0x72, + 0x65, 0x73, 0x73, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x47, 0x65, 0x74, 0x49, + 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x47, 0x65, 0x74, + 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6e, 0x0a, 0x0b, 0x4c, 0x69, 0x73, 0x74, + 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x12, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, + 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, + 0x61, 0x6c, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x32, 0xbf, 0x0f, 0x0a, 0x08, 0x44, 0x61, 0x74, + 0x61, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x6c, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, + 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x73, 0x22, 0x00, 0x12, 0x71, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, + 0x74, 0x69, 0x63, 0x73, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x32, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, + 0x6e, 0x61, 0x6c, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, + 0x73, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5b, 0x0a, 0x0f, 0x57, 0x61, 0x74, 0x63, 0x68, 0x44, + 0x6d, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x61, + 0x74, 0x63, 0x68, 0x44, 0x6d, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0d, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x7b, 0x0a, 0x12, + 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x68, 0x6f, + 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x0a, 0x47, 0x65, 0x74, + 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, + 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x50, 0x0a, 0x0c, 0x43, 0x6f, + 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x12, 0x21, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, + 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, 0x1a, 0x1b, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x6d, 0x0a, 0x12, + 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x55, 0x0a, 0x0c, 0x53, + 0x79, 0x6e, 0x63, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x53, 0x79, 0x6e, 0x63, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x22, 0x00, 0x12, 0x73, 0x0a, 0x12, 0x52, 0x65, 0x73, 0x65, 0x6e, 0x64, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, + 0x65, 0x6e, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x6e, + 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0d, 0x46, 0x6c, 0x75, 0x73, 0x68, + 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x6c, 0x75, + 0x73, 0x68, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, + 0x12, 0x64, 0x0a, 0x16, 0x4e, 0x6f, 0x74, 0x69, 0x66, 0x79, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x7b, 0x0a, 0x1d, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x43, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, + 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x57, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0x33, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x4f, 0x0a, 0x09, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, + 0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x22, 0x00, 0x12, 0x4b, 0x0a, 0x08, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x56, 0x32, + 0x12, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, + 0x00, 0x12, 0x67, 0x0a, 0x0e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, + 0x6f, 0x72, 0x74, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x50, 0x72, 0x65, + 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5e, 0x0a, 0x0b, 0x51, 0x75, + 0x65, 0x72, 0x79, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x51, 0x75, + 0x65, 0x72, 0x79, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0a, 0x44, 0x72, + 0x6f, 0x70, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x72, 0x6f, + 0x70, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x58, 0x0a, + 0x09, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x6c, 0x6f, 0x74, 0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x51, + 0x75, 0x65, 0x72, 0x79, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x61, 0x0a, 0x12, 0x44, 0x72, 0x6f, 0x70, 0x43, + 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, 0x12, 0x2c, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x42, 0x2e, 0x5a, 0x2c, 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, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2f, 0x64, 0x61, 0x74, 0x61, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x33, +} + +var ( + file_data_coord_proto_rawDescOnce sync.Once + file_data_coord_proto_rawDescData = file_data_coord_proto_rawDesc +) + +func file_data_coord_proto_rawDescGZIP() []byte { + file_data_coord_proto_rawDescOnce.Do(func() { + file_data_coord_proto_rawDescData = protoimpl.X.CompressGZIP(file_data_coord_proto_rawDescData) + }) + return file_data_coord_proto_rawDescData +} + +var file_data_coord_proto_enumTypes = make([]protoimpl.EnumInfo, 7) +var file_data_coord_proto_msgTypes = make([]protoimpl.MessageInfo, 108) +var file_data_coord_proto_goTypes = []interface{}{ + (SegmentType)(0), // 0: milvus.proto.data.SegmentType + (SegmentLevel)(0), // 1: milvus.proto.data.SegmentLevel + (ChannelWatchState)(0), // 2: milvus.proto.data.ChannelWatchState + (CompactionType)(0), // 3: milvus.proto.data.CompactionType + (ImportTaskStateV2)(0), // 4: milvus.proto.data.ImportTaskStateV2 + (GcCommand)(0), // 5: milvus.proto.data.GcCommand + (CompactionTaskState)(0), // 6: milvus.proto.data.CompactionTaskState + (*Empty)(nil), // 7: milvus.proto.data.Empty + (*FlushRequest)(nil), // 8: milvus.proto.data.FlushRequest + (*FlushResponse)(nil), // 9: milvus.proto.data.FlushResponse + (*FlushChannelsRequest)(nil), // 10: milvus.proto.data.FlushChannelsRequest + (*SegmentIDRequest)(nil), // 11: milvus.proto.data.SegmentIDRequest + (*AllocSegmentRequest)(nil), // 12: milvus.proto.data.AllocSegmentRequest + (*AllocSegmentResponse)(nil), // 13: milvus.proto.data.AllocSegmentResponse + (*AssignSegmentIDRequest)(nil), // 14: milvus.proto.data.AssignSegmentIDRequest + (*SegmentIDAssignment)(nil), // 15: milvus.proto.data.SegmentIDAssignment + (*AssignSegmentIDResponse)(nil), // 16: milvus.proto.data.AssignSegmentIDResponse + (*GetSegmentStatesRequest)(nil), // 17: milvus.proto.data.GetSegmentStatesRequest + (*SegmentStateInfo)(nil), // 18: milvus.proto.data.SegmentStateInfo + (*GetSegmentStatesResponse)(nil), // 19: milvus.proto.data.GetSegmentStatesResponse + (*GetSegmentInfoRequest)(nil), // 20: milvus.proto.data.GetSegmentInfoRequest + (*GetSegmentInfoResponse)(nil), // 21: milvus.proto.data.GetSegmentInfoResponse + (*GetInsertBinlogPathsRequest)(nil), // 22: milvus.proto.data.GetInsertBinlogPathsRequest + (*GetInsertBinlogPathsResponse)(nil), // 23: milvus.proto.data.GetInsertBinlogPathsResponse + (*GetCollectionStatisticsRequest)(nil), // 24: milvus.proto.data.GetCollectionStatisticsRequest + (*GetCollectionStatisticsResponse)(nil), // 25: milvus.proto.data.GetCollectionStatisticsResponse + (*GetPartitionStatisticsRequest)(nil), // 26: milvus.proto.data.GetPartitionStatisticsRequest + (*GetPartitionStatisticsResponse)(nil), // 27: milvus.proto.data.GetPartitionStatisticsResponse + (*GetSegmentInfoChannelRequest)(nil), // 28: milvus.proto.data.GetSegmentInfoChannelRequest + (*VchannelInfo)(nil), // 29: milvus.proto.data.VchannelInfo + (*WatchDmChannelsRequest)(nil), // 30: milvus.proto.data.WatchDmChannelsRequest + (*FlushSegmentsRequest)(nil), // 31: milvus.proto.data.FlushSegmentsRequest + (*SegmentMsg)(nil), // 32: milvus.proto.data.SegmentMsg + (*SegmentInfo)(nil), // 33: milvus.proto.data.SegmentInfo + (*SegmentStartPosition)(nil), // 34: milvus.proto.data.SegmentStartPosition + (*SaveBinlogPathsRequest)(nil), // 35: milvus.proto.data.SaveBinlogPathsRequest + (*CheckPoint)(nil), // 36: milvus.proto.data.CheckPoint + (*DeltaLogInfo)(nil), // 37: milvus.proto.data.DeltaLogInfo + (*ChannelStatus)(nil), // 38: milvus.proto.data.ChannelStatus + (*DataNodeInfo)(nil), // 39: milvus.proto.data.DataNodeInfo + (*SegmentBinlogs)(nil), // 40: milvus.proto.data.SegmentBinlogs + (*FieldBinlog)(nil), // 41: milvus.proto.data.FieldBinlog + (*TextIndexStats)(nil), // 42: milvus.proto.data.TextIndexStats + (*Binlog)(nil), // 43: milvus.proto.data.Binlog + (*GetRecoveryInfoResponse)(nil), // 44: milvus.proto.data.GetRecoveryInfoResponse + (*GetRecoveryInfoRequest)(nil), // 45: milvus.proto.data.GetRecoveryInfoRequest + (*GetRecoveryInfoResponseV2)(nil), // 46: milvus.proto.data.GetRecoveryInfoResponseV2 + (*GetRecoveryInfoRequestV2)(nil), // 47: milvus.proto.data.GetRecoveryInfoRequestV2 + (*GetChannelRecoveryInfoRequest)(nil), // 48: milvus.proto.data.GetChannelRecoveryInfoRequest + (*GetChannelRecoveryInfoResponse)(nil), // 49: milvus.proto.data.GetChannelRecoveryInfoResponse + (*GetSegmentsByStatesRequest)(nil), // 50: milvus.proto.data.GetSegmentsByStatesRequest + (*GetSegmentsByStatesResponse)(nil), // 51: milvus.proto.data.GetSegmentsByStatesResponse + (*GetFlushedSegmentsRequest)(nil), // 52: milvus.proto.data.GetFlushedSegmentsRequest + (*GetFlushedSegmentsResponse)(nil), // 53: milvus.proto.data.GetFlushedSegmentsResponse + (*SegmentFlushCompletedMsg)(nil), // 54: milvus.proto.data.SegmentFlushCompletedMsg + (*ChannelWatchInfo)(nil), // 55: milvus.proto.data.ChannelWatchInfo + (*CompactionStateRequest)(nil), // 56: milvus.proto.data.CompactionStateRequest + (*SyncSegmentInfo)(nil), // 57: milvus.proto.data.SyncSegmentInfo + (*SyncSegmentsRequest)(nil), // 58: milvus.proto.data.SyncSegmentsRequest + (*CompactionSegmentBinlogs)(nil), // 59: milvus.proto.data.CompactionSegmentBinlogs + (*CompactionPlan)(nil), // 60: milvus.proto.data.CompactionPlan + (*CompactionSegment)(nil), // 61: milvus.proto.data.CompactionSegment + (*CompactionPlanResult)(nil), // 62: milvus.proto.data.CompactionPlanResult + (*CompactionStateResponse)(nil), // 63: milvus.proto.data.CompactionStateResponse + (*SegmentFieldBinlogMeta)(nil), // 64: milvus.proto.data.SegmentFieldBinlogMeta + (*WatchChannelsRequest)(nil), // 65: milvus.proto.data.WatchChannelsRequest + (*WatchChannelsResponse)(nil), // 66: milvus.proto.data.WatchChannelsResponse + (*SetSegmentStateRequest)(nil), // 67: milvus.proto.data.SetSegmentStateRequest + (*SetSegmentStateResponse)(nil), // 68: milvus.proto.data.SetSegmentStateResponse + (*DropVirtualChannelRequest)(nil), // 69: milvus.proto.data.DropVirtualChannelRequest + (*DropVirtualChannelSegment)(nil), // 70: milvus.proto.data.DropVirtualChannelSegment + (*DropVirtualChannelResponse)(nil), // 71: milvus.proto.data.DropVirtualChannelResponse + (*UpdateSegmentStatisticsRequest)(nil), // 72: milvus.proto.data.UpdateSegmentStatisticsRequest + (*UpdateChannelCheckpointRequest)(nil), // 73: milvus.proto.data.UpdateChannelCheckpointRequest + (*ResendSegmentStatsRequest)(nil), // 74: milvus.proto.data.ResendSegmentStatsRequest + (*ResendSegmentStatsResponse)(nil), // 75: milvus.proto.data.ResendSegmentStatsResponse + (*MarkSegmentsDroppedRequest)(nil), // 76: milvus.proto.data.MarkSegmentsDroppedRequest + (*SegmentReferenceLock)(nil), // 77: milvus.proto.data.SegmentReferenceLock + (*AlterCollectionRequest)(nil), // 78: milvus.proto.data.AlterCollectionRequest + (*GcConfirmRequest)(nil), // 79: milvus.proto.data.GcConfirmRequest + (*GcConfirmResponse)(nil), // 80: milvus.proto.data.GcConfirmResponse + (*ReportDataNodeTtMsgsRequest)(nil), // 81: milvus.proto.data.ReportDataNodeTtMsgsRequest + (*GetFlushStateRequest)(nil), // 82: milvus.proto.data.GetFlushStateRequest + (*ChannelOperationsRequest)(nil), // 83: milvus.proto.data.ChannelOperationsRequest + (*ChannelOperationProgressResponse)(nil), // 84: milvus.proto.data.ChannelOperationProgressResponse + (*PreImportRequest)(nil), // 85: milvus.proto.data.PreImportRequest + (*IDRange)(nil), // 86: milvus.proto.data.IDRange + (*ImportRequestSegment)(nil), // 87: milvus.proto.data.ImportRequestSegment + (*ImportRequest)(nil), // 88: milvus.proto.data.ImportRequest + (*QueryPreImportRequest)(nil), // 89: milvus.proto.data.QueryPreImportRequest + (*PartitionImportStats)(nil), // 90: milvus.proto.data.PartitionImportStats + (*ImportFileStats)(nil), // 91: milvus.proto.data.ImportFileStats + (*QueryPreImportResponse)(nil), // 92: milvus.proto.data.QueryPreImportResponse + (*QueryImportRequest)(nil), // 93: milvus.proto.data.QueryImportRequest + (*ImportSegmentInfo)(nil), // 94: milvus.proto.data.ImportSegmentInfo + (*QueryImportResponse)(nil), // 95: milvus.proto.data.QueryImportResponse + (*DropImportRequest)(nil), // 96: milvus.proto.data.DropImportRequest + (*ImportJob)(nil), // 97: milvus.proto.data.ImportJob + (*PreImportTask)(nil), // 98: milvus.proto.data.PreImportTask + (*ImportTaskV2)(nil), // 99: milvus.proto.data.ImportTaskV2 + (*GcControlRequest)(nil), // 100: milvus.proto.data.GcControlRequest + (*QuerySlotRequest)(nil), // 101: milvus.proto.data.QuerySlotRequest + (*QuerySlotResponse)(nil), // 102: milvus.proto.data.QuerySlotResponse + (*CompactionTask)(nil), // 103: milvus.proto.data.CompactionTask + (*PartitionStatsInfo)(nil), // 104: milvus.proto.data.PartitionStatsInfo + (*DropCompactionPlanRequest)(nil), // 105: milvus.proto.data.DropCompactionPlanRequest + nil, // 106: milvus.proto.data.FlushResponse.ChannelCpsEntry + nil, // 107: milvus.proto.data.GetSegmentInfoResponse.ChannelCheckpointEntry + nil, // 108: milvus.proto.data.VchannelInfo.PartitionStatsVersionsEntry + nil, // 109: milvus.proto.data.SegmentInfo.TextStatsLogsEntry + nil, // 110: milvus.proto.data.SegmentBinlogs.TextStatsLogsEntry + nil, // 111: milvus.proto.data.SyncSegmentsRequest.SegmentInfosEntry + nil, // 112: milvus.proto.data.PartitionImportStats.PartitionRowsEntry + nil, // 113: milvus.proto.data.PartitionImportStats.PartitionDataSizeEntry + nil, // 114: milvus.proto.data.ImportFileStats.HashedStatsEntry + (*commonpb.MsgBase)(nil), // 115: milvus.proto.common.MsgBase + (*commonpb.Status)(nil), // 116: milvus.proto.common.Status + (commonpb.SegmentState)(0), // 117: milvus.proto.common.SegmentState + (*msgpb.MsgPosition)(nil), // 118: milvus.proto.msg.MsgPosition + (*internalpb.StringList)(nil), // 119: milvus.proto.internal.StringList + (*commonpb.KeyValuePair)(nil), // 120: milvus.proto.common.KeyValuePair + (*schemapb.CollectionSchema)(nil), // 121: milvus.proto.schema.CollectionSchema + (*commonpb.KeyDataPair)(nil), // 122: milvus.proto.common.KeyDataPair + (*commonpb.SegmentStats)(nil), // 123: milvus.proto.common.SegmentStats + (*msgpb.DataNodeTtMsg)(nil), // 124: milvus.proto.msg.DataNodeTtMsg + (*internalpb.ImportFile)(nil), // 125: milvus.proto.internal.ImportFile + (internalpb.ImportJobState)(0), // 126: milvus.proto.internal.ImportJobState + (*schemapb.FieldSchema)(nil), // 127: milvus.proto.schema.FieldSchema + (*milvuspb.GetComponentStatesRequest)(nil), // 128: milvus.proto.milvus.GetComponentStatesRequest + (*internalpb.GetTimeTickChannelRequest)(nil), // 129: milvus.proto.internal.GetTimeTickChannelRequest + (*internalpb.GetStatisticsChannelRequest)(nil), // 130: milvus.proto.internal.GetStatisticsChannelRequest + (*milvuspb.GetFlushAllStateRequest)(nil), // 131: milvus.proto.milvus.GetFlushAllStateRequest + (*internalpb.ShowConfigurationsRequest)(nil), // 132: milvus.proto.internal.ShowConfigurationsRequest + (*milvuspb.GetMetricsRequest)(nil), // 133: milvus.proto.milvus.GetMetricsRequest + (*milvuspb.ManualCompactionRequest)(nil), // 134: milvus.proto.milvus.ManualCompactionRequest + (*milvuspb.GetCompactionStateRequest)(nil), // 135: milvus.proto.milvus.GetCompactionStateRequest + (*milvuspb.GetCompactionPlansRequest)(nil), // 136: milvus.proto.milvus.GetCompactionPlansRequest + (*milvuspb.CheckHealthRequest)(nil), // 137: milvus.proto.milvus.CheckHealthRequest + (*indexpb.CreateIndexRequest)(nil), // 138: milvus.proto.index.CreateIndexRequest + (*indexpb.AlterIndexRequest)(nil), // 139: milvus.proto.index.AlterIndexRequest + (*indexpb.GetIndexStateRequest)(nil), // 140: milvus.proto.index.GetIndexStateRequest + (*indexpb.GetSegmentIndexStateRequest)(nil), // 141: milvus.proto.index.GetSegmentIndexStateRequest + (*indexpb.GetIndexInfoRequest)(nil), // 142: milvus.proto.index.GetIndexInfoRequest + (*indexpb.DropIndexRequest)(nil), // 143: milvus.proto.index.DropIndexRequest + (*indexpb.DescribeIndexRequest)(nil), // 144: milvus.proto.index.DescribeIndexRequest + (*indexpb.GetIndexStatisticsRequest)(nil), // 145: milvus.proto.index.GetIndexStatisticsRequest + (*indexpb.GetIndexBuildProgressRequest)(nil), // 146: milvus.proto.index.GetIndexBuildProgressRequest + (*indexpb.ListIndexesRequest)(nil), // 147: milvus.proto.index.ListIndexesRequest + (*internalpb.ImportRequestInternal)(nil), // 148: milvus.proto.internal.ImportRequestInternal + (*internalpb.GetImportProgressRequest)(nil), // 149: milvus.proto.internal.GetImportProgressRequest + (*internalpb.ListImportsRequestInternal)(nil), // 150: milvus.proto.internal.ListImportsRequestInternal + (*milvuspb.ComponentStates)(nil), // 151: milvus.proto.milvus.ComponentStates + (*milvuspb.StringResponse)(nil), // 152: milvus.proto.milvus.StringResponse + (*milvuspb.GetFlushAllStateResponse)(nil), // 153: milvus.proto.milvus.GetFlushAllStateResponse + (*internalpb.ShowConfigurationsResponse)(nil), // 154: milvus.proto.internal.ShowConfigurationsResponse + (*milvuspb.GetMetricsResponse)(nil), // 155: milvus.proto.milvus.GetMetricsResponse + (*milvuspb.ManualCompactionResponse)(nil), // 156: milvus.proto.milvus.ManualCompactionResponse + (*milvuspb.GetCompactionStateResponse)(nil), // 157: milvus.proto.milvus.GetCompactionStateResponse + (*milvuspb.GetCompactionPlansResponse)(nil), // 158: milvus.proto.milvus.GetCompactionPlansResponse + (*milvuspb.GetFlushStateResponse)(nil), // 159: milvus.proto.milvus.GetFlushStateResponse + (*milvuspb.CheckHealthResponse)(nil), // 160: milvus.proto.milvus.CheckHealthResponse + (*indexpb.GetIndexStateResponse)(nil), // 161: milvus.proto.index.GetIndexStateResponse + (*indexpb.GetSegmentIndexStateResponse)(nil), // 162: milvus.proto.index.GetSegmentIndexStateResponse + (*indexpb.GetIndexInfoResponse)(nil), // 163: milvus.proto.index.GetIndexInfoResponse + (*indexpb.DescribeIndexResponse)(nil), // 164: milvus.proto.index.DescribeIndexResponse + (*indexpb.GetIndexStatisticsResponse)(nil), // 165: milvus.proto.index.GetIndexStatisticsResponse + (*indexpb.GetIndexBuildProgressResponse)(nil), // 166: milvus.proto.index.GetIndexBuildProgressResponse + (*indexpb.ListIndexesResponse)(nil), // 167: milvus.proto.index.ListIndexesResponse + (*internalpb.ImportResponse)(nil), // 168: milvus.proto.internal.ImportResponse + (*internalpb.GetImportProgressResponse)(nil), // 169: milvus.proto.internal.GetImportProgressResponse + (*internalpb.ListImportsResponse)(nil), // 170: milvus.proto.internal.ListImportsResponse +} +var file_data_coord_proto_depIdxs = []int32{ + 115, // 0: milvus.proto.data.FlushRequest.base:type_name -> milvus.proto.common.MsgBase + 116, // 1: milvus.proto.data.FlushResponse.status:type_name -> milvus.proto.common.Status + 106, // 2: milvus.proto.data.FlushResponse.channel_cps:type_name -> milvus.proto.data.FlushResponse.ChannelCpsEntry + 115, // 3: milvus.proto.data.FlushChannelsRequest.base:type_name -> milvus.proto.common.MsgBase + 1, // 4: milvus.proto.data.SegmentIDRequest.level:type_name -> milvus.proto.data.SegmentLevel + 33, // 5: milvus.proto.data.AllocSegmentResponse.segment_info:type_name -> milvus.proto.data.SegmentInfo + 116, // 6: milvus.proto.data.AllocSegmentResponse.status:type_name -> milvus.proto.common.Status + 11, // 7: milvus.proto.data.AssignSegmentIDRequest.segmentIDRequests:type_name -> milvus.proto.data.SegmentIDRequest + 116, // 8: milvus.proto.data.SegmentIDAssignment.status:type_name -> milvus.proto.common.Status + 15, // 9: milvus.proto.data.AssignSegmentIDResponse.segIDAssignments:type_name -> milvus.proto.data.SegmentIDAssignment + 116, // 10: milvus.proto.data.AssignSegmentIDResponse.status:type_name -> milvus.proto.common.Status + 115, // 11: milvus.proto.data.GetSegmentStatesRequest.base:type_name -> milvus.proto.common.MsgBase + 117, // 12: milvus.proto.data.SegmentStateInfo.state:type_name -> milvus.proto.common.SegmentState + 118, // 13: milvus.proto.data.SegmentStateInfo.start_position:type_name -> milvus.proto.msg.MsgPosition + 118, // 14: milvus.proto.data.SegmentStateInfo.end_position:type_name -> milvus.proto.msg.MsgPosition + 116, // 15: milvus.proto.data.SegmentStateInfo.status:type_name -> milvus.proto.common.Status + 116, // 16: milvus.proto.data.GetSegmentStatesResponse.status:type_name -> milvus.proto.common.Status + 18, // 17: milvus.proto.data.GetSegmentStatesResponse.states:type_name -> milvus.proto.data.SegmentStateInfo + 115, // 18: milvus.proto.data.GetSegmentInfoRequest.base:type_name -> milvus.proto.common.MsgBase + 116, // 19: milvus.proto.data.GetSegmentInfoResponse.status:type_name -> milvus.proto.common.Status + 33, // 20: milvus.proto.data.GetSegmentInfoResponse.infos:type_name -> milvus.proto.data.SegmentInfo + 107, // 21: milvus.proto.data.GetSegmentInfoResponse.channel_checkpoint:type_name -> milvus.proto.data.GetSegmentInfoResponse.ChannelCheckpointEntry + 115, // 22: milvus.proto.data.GetInsertBinlogPathsRequest.base:type_name -> milvus.proto.common.MsgBase + 119, // 23: milvus.proto.data.GetInsertBinlogPathsResponse.paths:type_name -> milvus.proto.internal.StringList + 116, // 24: milvus.proto.data.GetInsertBinlogPathsResponse.status:type_name -> milvus.proto.common.Status + 115, // 25: milvus.proto.data.GetCollectionStatisticsRequest.base:type_name -> milvus.proto.common.MsgBase + 120, // 26: milvus.proto.data.GetCollectionStatisticsResponse.stats:type_name -> milvus.proto.common.KeyValuePair + 116, // 27: milvus.proto.data.GetCollectionStatisticsResponse.status:type_name -> milvus.proto.common.Status + 115, // 28: milvus.proto.data.GetPartitionStatisticsRequest.base:type_name -> milvus.proto.common.MsgBase + 120, // 29: milvus.proto.data.GetPartitionStatisticsResponse.stats:type_name -> milvus.proto.common.KeyValuePair + 116, // 30: milvus.proto.data.GetPartitionStatisticsResponse.status:type_name -> milvus.proto.common.Status + 118, // 31: milvus.proto.data.VchannelInfo.seek_position:type_name -> milvus.proto.msg.MsgPosition + 33, // 32: milvus.proto.data.VchannelInfo.unflushedSegments:type_name -> milvus.proto.data.SegmentInfo + 33, // 33: milvus.proto.data.VchannelInfo.flushedSegments:type_name -> milvus.proto.data.SegmentInfo + 33, // 34: milvus.proto.data.VchannelInfo.dropped_segments:type_name -> milvus.proto.data.SegmentInfo + 33, // 35: milvus.proto.data.VchannelInfo.indexed_segments:type_name -> milvus.proto.data.SegmentInfo + 108, // 36: milvus.proto.data.VchannelInfo.partition_stats_versions:type_name -> milvus.proto.data.VchannelInfo.PartitionStatsVersionsEntry + 115, // 37: milvus.proto.data.WatchDmChannelsRequest.base:type_name -> milvus.proto.common.MsgBase + 29, // 38: milvus.proto.data.WatchDmChannelsRequest.vchannels:type_name -> milvus.proto.data.VchannelInfo + 115, // 39: milvus.proto.data.FlushSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase + 115, // 40: milvus.proto.data.SegmentMsg.base:type_name -> milvus.proto.common.MsgBase + 33, // 41: milvus.proto.data.SegmentMsg.segment:type_name -> milvus.proto.data.SegmentInfo + 117, // 42: milvus.proto.data.SegmentInfo.state:type_name -> milvus.proto.common.SegmentState + 118, // 43: milvus.proto.data.SegmentInfo.start_position:type_name -> milvus.proto.msg.MsgPosition + 118, // 44: milvus.proto.data.SegmentInfo.dml_position:type_name -> milvus.proto.msg.MsgPosition + 41, // 45: milvus.proto.data.SegmentInfo.binlogs:type_name -> milvus.proto.data.FieldBinlog + 41, // 46: milvus.proto.data.SegmentInfo.statslogs:type_name -> milvus.proto.data.FieldBinlog + 41, // 47: milvus.proto.data.SegmentInfo.deltalogs:type_name -> milvus.proto.data.FieldBinlog + 1, // 48: milvus.proto.data.SegmentInfo.level:type_name -> milvus.proto.data.SegmentLevel + 1, // 49: milvus.proto.data.SegmentInfo.last_level:type_name -> milvus.proto.data.SegmentLevel + 109, // 50: milvus.proto.data.SegmentInfo.textStatsLogs:type_name -> milvus.proto.data.SegmentInfo.TextStatsLogsEntry + 41, // 51: milvus.proto.data.SegmentInfo.bm25statslogs:type_name -> milvus.proto.data.FieldBinlog + 118, // 52: milvus.proto.data.SegmentStartPosition.start_position:type_name -> milvus.proto.msg.MsgPosition + 115, // 53: milvus.proto.data.SaveBinlogPathsRequest.base:type_name -> milvus.proto.common.MsgBase + 41, // 54: milvus.proto.data.SaveBinlogPathsRequest.field2BinlogPaths:type_name -> milvus.proto.data.FieldBinlog + 36, // 55: milvus.proto.data.SaveBinlogPathsRequest.checkPoints:type_name -> milvus.proto.data.CheckPoint + 34, // 56: milvus.proto.data.SaveBinlogPathsRequest.start_positions:type_name -> milvus.proto.data.SegmentStartPosition + 41, // 57: milvus.proto.data.SaveBinlogPathsRequest.field2StatslogPaths:type_name -> milvus.proto.data.FieldBinlog + 41, // 58: milvus.proto.data.SaveBinlogPathsRequest.deltalogs:type_name -> milvus.proto.data.FieldBinlog + 1, // 59: milvus.proto.data.SaveBinlogPathsRequest.seg_level:type_name -> milvus.proto.data.SegmentLevel + 41, // 60: milvus.proto.data.SaveBinlogPathsRequest.field2Bm25logPaths:type_name -> milvus.proto.data.FieldBinlog + 118, // 61: milvus.proto.data.CheckPoint.position:type_name -> milvus.proto.msg.MsgPosition + 2, // 62: milvus.proto.data.ChannelStatus.state:type_name -> milvus.proto.data.ChannelWatchState + 38, // 63: milvus.proto.data.DataNodeInfo.channels:type_name -> milvus.proto.data.ChannelStatus + 41, // 64: milvus.proto.data.SegmentBinlogs.fieldBinlogs:type_name -> milvus.proto.data.FieldBinlog + 41, // 65: milvus.proto.data.SegmentBinlogs.statslogs:type_name -> milvus.proto.data.FieldBinlog + 41, // 66: milvus.proto.data.SegmentBinlogs.deltalogs:type_name -> milvus.proto.data.FieldBinlog + 110, // 67: milvus.proto.data.SegmentBinlogs.textStatsLogs:type_name -> milvus.proto.data.SegmentBinlogs.TextStatsLogsEntry + 43, // 68: milvus.proto.data.FieldBinlog.binlogs:type_name -> milvus.proto.data.Binlog + 116, // 69: milvus.proto.data.GetRecoveryInfoResponse.status:type_name -> milvus.proto.common.Status + 29, // 70: milvus.proto.data.GetRecoveryInfoResponse.channels:type_name -> milvus.proto.data.VchannelInfo + 40, // 71: milvus.proto.data.GetRecoveryInfoResponse.binlogs:type_name -> milvus.proto.data.SegmentBinlogs + 115, // 72: milvus.proto.data.GetRecoveryInfoRequest.base:type_name -> milvus.proto.common.MsgBase + 116, // 73: milvus.proto.data.GetRecoveryInfoResponseV2.status:type_name -> milvus.proto.common.Status + 29, // 74: milvus.proto.data.GetRecoveryInfoResponseV2.channels:type_name -> milvus.proto.data.VchannelInfo + 33, // 75: milvus.proto.data.GetRecoveryInfoResponseV2.segments:type_name -> milvus.proto.data.SegmentInfo + 115, // 76: milvus.proto.data.GetRecoveryInfoRequestV2.base:type_name -> milvus.proto.common.MsgBase + 115, // 77: milvus.proto.data.GetChannelRecoveryInfoRequest.base:type_name -> milvus.proto.common.MsgBase + 116, // 78: milvus.proto.data.GetChannelRecoveryInfoResponse.status:type_name -> milvus.proto.common.Status + 29, // 79: milvus.proto.data.GetChannelRecoveryInfoResponse.info:type_name -> milvus.proto.data.VchannelInfo + 121, // 80: milvus.proto.data.GetChannelRecoveryInfoResponse.schema:type_name -> milvus.proto.schema.CollectionSchema + 115, // 81: milvus.proto.data.GetSegmentsByStatesRequest.base:type_name -> milvus.proto.common.MsgBase + 117, // 82: milvus.proto.data.GetSegmentsByStatesRequest.states:type_name -> milvus.proto.common.SegmentState + 116, // 83: milvus.proto.data.GetSegmentsByStatesResponse.status:type_name -> milvus.proto.common.Status + 115, // 84: milvus.proto.data.GetFlushedSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase + 116, // 85: milvus.proto.data.GetFlushedSegmentsResponse.status:type_name -> milvus.proto.common.Status + 115, // 86: milvus.proto.data.SegmentFlushCompletedMsg.base:type_name -> milvus.proto.common.MsgBase + 33, // 87: milvus.proto.data.SegmentFlushCompletedMsg.segment:type_name -> milvus.proto.data.SegmentInfo + 29, // 88: milvus.proto.data.ChannelWatchInfo.vchan:type_name -> milvus.proto.data.VchannelInfo + 2, // 89: milvus.proto.data.ChannelWatchInfo.state:type_name -> milvus.proto.data.ChannelWatchState + 121, // 90: milvus.proto.data.ChannelWatchInfo.schema:type_name -> milvus.proto.schema.CollectionSchema + 120, // 91: milvus.proto.data.ChannelWatchInfo.dbProperties:type_name -> milvus.proto.common.KeyValuePair + 115, // 92: milvus.proto.data.CompactionStateRequest.base:type_name -> milvus.proto.common.MsgBase + 41, // 93: milvus.proto.data.SyncSegmentInfo.pk_stats_log:type_name -> milvus.proto.data.FieldBinlog + 117, // 94: milvus.proto.data.SyncSegmentInfo.state:type_name -> milvus.proto.common.SegmentState + 1, // 95: milvus.proto.data.SyncSegmentInfo.level:type_name -> milvus.proto.data.SegmentLevel + 41, // 96: milvus.proto.data.SyncSegmentsRequest.stats_logs:type_name -> milvus.proto.data.FieldBinlog + 111, // 97: milvus.proto.data.SyncSegmentsRequest.segment_infos:type_name -> milvus.proto.data.SyncSegmentsRequest.SegmentInfosEntry + 41, // 98: milvus.proto.data.CompactionSegmentBinlogs.fieldBinlogs:type_name -> milvus.proto.data.FieldBinlog + 41, // 99: milvus.proto.data.CompactionSegmentBinlogs.field2StatslogPaths:type_name -> milvus.proto.data.FieldBinlog + 41, // 100: milvus.proto.data.CompactionSegmentBinlogs.deltalogs:type_name -> milvus.proto.data.FieldBinlog + 1, // 101: milvus.proto.data.CompactionSegmentBinlogs.level:type_name -> milvus.proto.data.SegmentLevel + 59, // 102: milvus.proto.data.CompactionPlan.segmentBinlogs:type_name -> milvus.proto.data.CompactionSegmentBinlogs + 3, // 103: milvus.proto.data.CompactionPlan.type:type_name -> milvus.proto.data.CompactionType + 121, // 104: milvus.proto.data.CompactionPlan.schema:type_name -> milvus.proto.schema.CollectionSchema + 86, // 105: milvus.proto.data.CompactionPlan.pre_allocated_segmentIDs:type_name -> milvus.proto.data.IDRange + 41, // 106: milvus.proto.data.CompactionSegment.insert_logs:type_name -> milvus.proto.data.FieldBinlog + 41, // 107: milvus.proto.data.CompactionSegment.field2StatslogPaths:type_name -> milvus.proto.data.FieldBinlog + 41, // 108: milvus.proto.data.CompactionSegment.deltalogs:type_name -> milvus.proto.data.FieldBinlog + 41, // 109: milvus.proto.data.CompactionSegment.bm25logs:type_name -> milvus.proto.data.FieldBinlog + 6, // 110: milvus.proto.data.CompactionPlanResult.state:type_name -> milvus.proto.data.CompactionTaskState + 61, // 111: milvus.proto.data.CompactionPlanResult.segments:type_name -> milvus.proto.data.CompactionSegment + 3, // 112: milvus.proto.data.CompactionPlanResult.type:type_name -> milvus.proto.data.CompactionType + 116, // 113: milvus.proto.data.CompactionStateResponse.status:type_name -> milvus.proto.common.Status + 62, // 114: milvus.proto.data.CompactionStateResponse.results:type_name -> milvus.proto.data.CompactionPlanResult + 122, // 115: milvus.proto.data.WatchChannelsRequest.start_positions:type_name -> milvus.proto.common.KeyDataPair + 121, // 116: milvus.proto.data.WatchChannelsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 120, // 117: milvus.proto.data.WatchChannelsRequest.db_properties:type_name -> milvus.proto.common.KeyValuePair + 116, // 118: milvus.proto.data.WatchChannelsResponse.status:type_name -> milvus.proto.common.Status + 115, // 119: milvus.proto.data.SetSegmentStateRequest.base:type_name -> milvus.proto.common.MsgBase + 117, // 120: milvus.proto.data.SetSegmentStateRequest.new_state:type_name -> milvus.proto.common.SegmentState + 116, // 121: milvus.proto.data.SetSegmentStateResponse.status:type_name -> milvus.proto.common.Status + 115, // 122: milvus.proto.data.DropVirtualChannelRequest.base:type_name -> milvus.proto.common.MsgBase + 70, // 123: milvus.proto.data.DropVirtualChannelRequest.segments:type_name -> milvus.proto.data.DropVirtualChannelSegment + 41, // 124: milvus.proto.data.DropVirtualChannelSegment.field2BinlogPaths:type_name -> milvus.proto.data.FieldBinlog + 41, // 125: milvus.proto.data.DropVirtualChannelSegment.field2StatslogPaths:type_name -> milvus.proto.data.FieldBinlog + 41, // 126: milvus.proto.data.DropVirtualChannelSegment.deltalogs:type_name -> milvus.proto.data.FieldBinlog + 118, // 127: milvus.proto.data.DropVirtualChannelSegment.startPosition:type_name -> milvus.proto.msg.MsgPosition + 118, // 128: milvus.proto.data.DropVirtualChannelSegment.checkPoint:type_name -> milvus.proto.msg.MsgPosition + 116, // 129: milvus.proto.data.DropVirtualChannelResponse.status:type_name -> milvus.proto.common.Status + 115, // 130: milvus.proto.data.UpdateSegmentStatisticsRequest.base:type_name -> milvus.proto.common.MsgBase + 123, // 131: milvus.proto.data.UpdateSegmentStatisticsRequest.stats:type_name -> milvus.proto.common.SegmentStats + 115, // 132: milvus.proto.data.UpdateChannelCheckpointRequest.base:type_name -> milvus.proto.common.MsgBase + 118, // 133: milvus.proto.data.UpdateChannelCheckpointRequest.position:type_name -> milvus.proto.msg.MsgPosition + 118, // 134: milvus.proto.data.UpdateChannelCheckpointRequest.channel_checkpoints:type_name -> milvus.proto.msg.MsgPosition + 115, // 135: milvus.proto.data.ResendSegmentStatsRequest.base:type_name -> milvus.proto.common.MsgBase + 116, // 136: milvus.proto.data.ResendSegmentStatsResponse.status:type_name -> milvus.proto.common.Status + 115, // 137: milvus.proto.data.MarkSegmentsDroppedRequest.base:type_name -> milvus.proto.common.MsgBase + 121, // 138: milvus.proto.data.AlterCollectionRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 122, // 139: milvus.proto.data.AlterCollectionRequest.start_positions:type_name -> milvus.proto.common.KeyDataPair + 120, // 140: milvus.proto.data.AlterCollectionRequest.properties:type_name -> milvus.proto.common.KeyValuePair + 116, // 141: milvus.proto.data.GcConfirmResponse.status:type_name -> milvus.proto.common.Status + 115, // 142: milvus.proto.data.ReportDataNodeTtMsgsRequest.base:type_name -> milvus.proto.common.MsgBase + 124, // 143: milvus.proto.data.ReportDataNodeTtMsgsRequest.msgs:type_name -> milvus.proto.msg.DataNodeTtMsg + 55, // 144: milvus.proto.data.ChannelOperationsRequest.infos:type_name -> milvus.proto.data.ChannelWatchInfo + 116, // 145: milvus.proto.data.ChannelOperationProgressResponse.status:type_name -> milvus.proto.common.Status + 2, // 146: milvus.proto.data.ChannelOperationProgressResponse.state:type_name -> milvus.proto.data.ChannelWatchState + 121, // 147: milvus.proto.data.PreImportRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 125, // 148: milvus.proto.data.PreImportRequest.import_files:type_name -> milvus.proto.internal.ImportFile + 120, // 149: milvus.proto.data.PreImportRequest.options:type_name -> milvus.proto.common.KeyValuePair + 121, // 150: milvus.proto.data.ImportRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 125, // 151: milvus.proto.data.ImportRequest.files:type_name -> milvus.proto.internal.ImportFile + 120, // 152: milvus.proto.data.ImportRequest.options:type_name -> milvus.proto.common.KeyValuePair + 86, // 153: milvus.proto.data.ImportRequest.ID_range:type_name -> milvus.proto.data.IDRange + 87, // 154: milvus.proto.data.ImportRequest.request_segments:type_name -> milvus.proto.data.ImportRequestSegment + 112, // 155: milvus.proto.data.PartitionImportStats.partition_rows:type_name -> milvus.proto.data.PartitionImportStats.PartitionRowsEntry + 113, // 156: milvus.proto.data.PartitionImportStats.partition_data_size:type_name -> milvus.proto.data.PartitionImportStats.PartitionDataSizeEntry + 125, // 157: milvus.proto.data.ImportFileStats.import_file:type_name -> milvus.proto.internal.ImportFile + 114, // 158: milvus.proto.data.ImportFileStats.hashed_stats:type_name -> milvus.proto.data.ImportFileStats.HashedStatsEntry + 116, // 159: milvus.proto.data.QueryPreImportResponse.status:type_name -> milvus.proto.common.Status + 4, // 160: milvus.proto.data.QueryPreImportResponse.state:type_name -> milvus.proto.data.ImportTaskStateV2 + 91, // 161: milvus.proto.data.QueryPreImportResponse.file_stats:type_name -> milvus.proto.data.ImportFileStats + 41, // 162: milvus.proto.data.ImportSegmentInfo.binlogs:type_name -> milvus.proto.data.FieldBinlog + 41, // 163: milvus.proto.data.ImportSegmentInfo.statslogs:type_name -> milvus.proto.data.FieldBinlog + 41, // 164: milvus.proto.data.ImportSegmentInfo.deltalogs:type_name -> milvus.proto.data.FieldBinlog + 116, // 165: milvus.proto.data.QueryImportResponse.status:type_name -> milvus.proto.common.Status + 4, // 166: milvus.proto.data.QueryImportResponse.state:type_name -> milvus.proto.data.ImportTaskStateV2 + 94, // 167: milvus.proto.data.QueryImportResponse.import_segments_info:type_name -> milvus.proto.data.ImportSegmentInfo + 121, // 168: milvus.proto.data.ImportJob.schema:type_name -> milvus.proto.schema.CollectionSchema + 126, // 169: milvus.proto.data.ImportJob.state:type_name -> milvus.proto.internal.ImportJobState + 125, // 170: milvus.proto.data.ImportJob.files:type_name -> milvus.proto.internal.ImportFile + 120, // 171: milvus.proto.data.ImportJob.options:type_name -> milvus.proto.common.KeyValuePair + 4, // 172: milvus.proto.data.PreImportTask.state:type_name -> milvus.proto.data.ImportTaskStateV2 + 91, // 173: milvus.proto.data.PreImportTask.file_stats:type_name -> milvus.proto.data.ImportFileStats + 4, // 174: milvus.proto.data.ImportTaskV2.state:type_name -> milvus.proto.data.ImportTaskStateV2 + 91, // 175: milvus.proto.data.ImportTaskV2.file_stats:type_name -> milvus.proto.data.ImportFileStats + 115, // 176: milvus.proto.data.GcControlRequest.base:type_name -> milvus.proto.common.MsgBase + 5, // 177: milvus.proto.data.GcControlRequest.command:type_name -> milvus.proto.data.GcCommand + 120, // 178: milvus.proto.data.GcControlRequest.params:type_name -> milvus.proto.common.KeyValuePair + 116, // 179: milvus.proto.data.QuerySlotResponse.status:type_name -> milvus.proto.common.Status + 3, // 180: milvus.proto.data.CompactionTask.type:type_name -> milvus.proto.data.CompactionType + 6, // 181: milvus.proto.data.CompactionTask.state:type_name -> milvus.proto.data.CompactionTaskState + 118, // 182: milvus.proto.data.CompactionTask.pos:type_name -> milvus.proto.msg.MsgPosition + 121, // 183: milvus.proto.data.CompactionTask.schema:type_name -> milvus.proto.schema.CollectionSchema + 127, // 184: milvus.proto.data.CompactionTask.clustering_key_field:type_name -> milvus.proto.schema.FieldSchema + 86, // 185: milvus.proto.data.CompactionTask.pre_allocated_segmentIDs:type_name -> milvus.proto.data.IDRange + 118, // 186: milvus.proto.data.FlushResponse.ChannelCpsEntry.value:type_name -> milvus.proto.msg.MsgPosition + 118, // 187: milvus.proto.data.GetSegmentInfoResponse.ChannelCheckpointEntry.value:type_name -> milvus.proto.msg.MsgPosition + 42, // 188: milvus.proto.data.SegmentInfo.TextStatsLogsEntry.value:type_name -> milvus.proto.data.TextIndexStats + 42, // 189: milvus.proto.data.SegmentBinlogs.TextStatsLogsEntry.value:type_name -> milvus.proto.data.TextIndexStats + 57, // 190: milvus.proto.data.SyncSegmentsRequest.SegmentInfosEntry.value:type_name -> milvus.proto.data.SyncSegmentInfo + 90, // 191: milvus.proto.data.ImportFileStats.HashedStatsEntry.value:type_name -> milvus.proto.data.PartitionImportStats + 128, // 192: milvus.proto.data.DataCoord.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest + 129, // 193: milvus.proto.data.DataCoord.GetTimeTickChannel:input_type -> milvus.proto.internal.GetTimeTickChannelRequest + 130, // 194: milvus.proto.data.DataCoord.GetStatisticsChannel:input_type -> milvus.proto.internal.GetStatisticsChannelRequest + 8, // 195: milvus.proto.data.DataCoord.Flush:input_type -> milvus.proto.data.FlushRequest + 12, // 196: milvus.proto.data.DataCoord.AllocSegment:input_type -> milvus.proto.data.AllocSegmentRequest + 14, // 197: milvus.proto.data.DataCoord.AssignSegmentID:input_type -> milvus.proto.data.AssignSegmentIDRequest + 20, // 198: milvus.proto.data.DataCoord.GetSegmentInfo:input_type -> milvus.proto.data.GetSegmentInfoRequest + 17, // 199: milvus.proto.data.DataCoord.GetSegmentStates:input_type -> milvus.proto.data.GetSegmentStatesRequest + 22, // 200: milvus.proto.data.DataCoord.GetInsertBinlogPaths:input_type -> milvus.proto.data.GetInsertBinlogPathsRequest + 24, // 201: milvus.proto.data.DataCoord.GetCollectionStatistics:input_type -> milvus.proto.data.GetCollectionStatisticsRequest + 26, // 202: milvus.proto.data.DataCoord.GetPartitionStatistics:input_type -> milvus.proto.data.GetPartitionStatisticsRequest + 28, // 203: milvus.proto.data.DataCoord.GetSegmentInfoChannel:input_type -> milvus.proto.data.GetSegmentInfoChannelRequest + 35, // 204: milvus.proto.data.DataCoord.SaveBinlogPaths:input_type -> milvus.proto.data.SaveBinlogPathsRequest + 45, // 205: milvus.proto.data.DataCoord.GetRecoveryInfo:input_type -> milvus.proto.data.GetRecoveryInfoRequest + 47, // 206: milvus.proto.data.DataCoord.GetRecoveryInfoV2:input_type -> milvus.proto.data.GetRecoveryInfoRequestV2 + 48, // 207: milvus.proto.data.DataCoord.GetChannelRecoveryInfo:input_type -> milvus.proto.data.GetChannelRecoveryInfoRequest + 52, // 208: milvus.proto.data.DataCoord.GetFlushedSegments:input_type -> milvus.proto.data.GetFlushedSegmentsRequest + 50, // 209: milvus.proto.data.DataCoord.GetSegmentsByStates:input_type -> milvus.proto.data.GetSegmentsByStatesRequest + 131, // 210: milvus.proto.data.DataCoord.GetFlushAllState:input_type -> milvus.proto.milvus.GetFlushAllStateRequest + 132, // 211: milvus.proto.data.DataCoord.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest + 133, // 212: milvus.proto.data.DataCoord.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest + 134, // 213: milvus.proto.data.DataCoord.ManualCompaction:input_type -> milvus.proto.milvus.ManualCompactionRequest + 135, // 214: milvus.proto.data.DataCoord.GetCompactionState:input_type -> milvus.proto.milvus.GetCompactionStateRequest + 136, // 215: milvus.proto.data.DataCoord.GetCompactionStateWithPlans:input_type -> milvus.proto.milvus.GetCompactionPlansRequest + 65, // 216: milvus.proto.data.DataCoord.WatchChannels:input_type -> milvus.proto.data.WatchChannelsRequest + 82, // 217: milvus.proto.data.DataCoord.GetFlushState:input_type -> milvus.proto.data.GetFlushStateRequest + 69, // 218: milvus.proto.data.DataCoord.DropVirtualChannel:input_type -> milvus.proto.data.DropVirtualChannelRequest + 67, // 219: milvus.proto.data.DataCoord.SetSegmentState:input_type -> milvus.proto.data.SetSegmentStateRequest + 72, // 220: milvus.proto.data.DataCoord.UpdateSegmentStatistics:input_type -> milvus.proto.data.UpdateSegmentStatisticsRequest + 73, // 221: milvus.proto.data.DataCoord.UpdateChannelCheckpoint:input_type -> milvus.proto.data.UpdateChannelCheckpointRequest + 76, // 222: milvus.proto.data.DataCoord.MarkSegmentsDropped:input_type -> milvus.proto.data.MarkSegmentsDroppedRequest + 78, // 223: milvus.proto.data.DataCoord.BroadcastAlteredCollection:input_type -> milvus.proto.data.AlterCollectionRequest + 137, // 224: milvus.proto.data.DataCoord.CheckHealth:input_type -> milvus.proto.milvus.CheckHealthRequest + 138, // 225: milvus.proto.data.DataCoord.CreateIndex:input_type -> milvus.proto.index.CreateIndexRequest + 139, // 226: milvus.proto.data.DataCoord.AlterIndex:input_type -> milvus.proto.index.AlterIndexRequest + 140, // 227: milvus.proto.data.DataCoord.GetIndexState:input_type -> milvus.proto.index.GetIndexStateRequest + 141, // 228: milvus.proto.data.DataCoord.GetSegmentIndexState:input_type -> milvus.proto.index.GetSegmentIndexStateRequest + 142, // 229: milvus.proto.data.DataCoord.GetIndexInfos:input_type -> milvus.proto.index.GetIndexInfoRequest + 143, // 230: milvus.proto.data.DataCoord.DropIndex:input_type -> milvus.proto.index.DropIndexRequest + 144, // 231: milvus.proto.data.DataCoord.DescribeIndex:input_type -> milvus.proto.index.DescribeIndexRequest + 145, // 232: milvus.proto.data.DataCoord.GetIndexStatistics:input_type -> milvus.proto.index.GetIndexStatisticsRequest + 146, // 233: milvus.proto.data.DataCoord.GetIndexBuildProgress:input_type -> milvus.proto.index.GetIndexBuildProgressRequest + 147, // 234: milvus.proto.data.DataCoord.ListIndexes:input_type -> milvus.proto.index.ListIndexesRequest + 79, // 235: milvus.proto.data.DataCoord.GcConfirm:input_type -> milvus.proto.data.GcConfirmRequest + 81, // 236: milvus.proto.data.DataCoord.ReportDataNodeTtMsgs:input_type -> milvus.proto.data.ReportDataNodeTtMsgsRequest + 100, // 237: milvus.proto.data.DataCoord.GcControl:input_type -> milvus.proto.data.GcControlRequest + 148, // 238: milvus.proto.data.DataCoord.ImportV2:input_type -> milvus.proto.internal.ImportRequestInternal + 149, // 239: milvus.proto.data.DataCoord.GetImportProgress:input_type -> milvus.proto.internal.GetImportProgressRequest + 150, // 240: milvus.proto.data.DataCoord.ListImports:input_type -> milvus.proto.internal.ListImportsRequestInternal + 128, // 241: milvus.proto.data.DataNode.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest + 130, // 242: milvus.proto.data.DataNode.GetStatisticsChannel:input_type -> milvus.proto.internal.GetStatisticsChannelRequest + 30, // 243: milvus.proto.data.DataNode.WatchDmChannels:input_type -> milvus.proto.data.WatchDmChannelsRequest + 31, // 244: milvus.proto.data.DataNode.FlushSegments:input_type -> milvus.proto.data.FlushSegmentsRequest + 132, // 245: milvus.proto.data.DataNode.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest + 133, // 246: milvus.proto.data.DataNode.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest + 60, // 247: milvus.proto.data.DataNode.CompactionV2:input_type -> milvus.proto.data.CompactionPlan + 56, // 248: milvus.proto.data.DataNode.GetCompactionState:input_type -> milvus.proto.data.CompactionStateRequest + 58, // 249: milvus.proto.data.DataNode.SyncSegments:input_type -> milvus.proto.data.SyncSegmentsRequest + 74, // 250: milvus.proto.data.DataNode.ResendSegmentStats:input_type -> milvus.proto.data.ResendSegmentStatsRequest + 10, // 251: milvus.proto.data.DataNode.FlushChannels:input_type -> milvus.proto.data.FlushChannelsRequest + 83, // 252: milvus.proto.data.DataNode.NotifyChannelOperation:input_type -> milvus.proto.data.ChannelOperationsRequest + 55, // 253: milvus.proto.data.DataNode.CheckChannelOperationProgress:input_type -> milvus.proto.data.ChannelWatchInfo + 85, // 254: milvus.proto.data.DataNode.PreImport:input_type -> milvus.proto.data.PreImportRequest + 88, // 255: milvus.proto.data.DataNode.ImportV2:input_type -> milvus.proto.data.ImportRequest + 89, // 256: milvus.proto.data.DataNode.QueryPreImport:input_type -> milvus.proto.data.QueryPreImportRequest + 93, // 257: milvus.proto.data.DataNode.QueryImport:input_type -> milvus.proto.data.QueryImportRequest + 96, // 258: milvus.proto.data.DataNode.DropImport:input_type -> milvus.proto.data.DropImportRequest + 101, // 259: milvus.proto.data.DataNode.QuerySlot:input_type -> milvus.proto.data.QuerySlotRequest + 105, // 260: milvus.proto.data.DataNode.DropCompactionPlan:input_type -> milvus.proto.data.DropCompactionPlanRequest + 151, // 261: milvus.proto.data.DataCoord.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates + 152, // 262: milvus.proto.data.DataCoord.GetTimeTickChannel:output_type -> milvus.proto.milvus.StringResponse + 152, // 263: milvus.proto.data.DataCoord.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse + 9, // 264: milvus.proto.data.DataCoord.Flush:output_type -> milvus.proto.data.FlushResponse + 13, // 265: milvus.proto.data.DataCoord.AllocSegment:output_type -> milvus.proto.data.AllocSegmentResponse + 16, // 266: milvus.proto.data.DataCoord.AssignSegmentID:output_type -> milvus.proto.data.AssignSegmentIDResponse + 21, // 267: milvus.proto.data.DataCoord.GetSegmentInfo:output_type -> milvus.proto.data.GetSegmentInfoResponse + 19, // 268: milvus.proto.data.DataCoord.GetSegmentStates:output_type -> milvus.proto.data.GetSegmentStatesResponse + 23, // 269: milvus.proto.data.DataCoord.GetInsertBinlogPaths:output_type -> milvus.proto.data.GetInsertBinlogPathsResponse + 25, // 270: milvus.proto.data.DataCoord.GetCollectionStatistics:output_type -> milvus.proto.data.GetCollectionStatisticsResponse + 27, // 271: milvus.proto.data.DataCoord.GetPartitionStatistics:output_type -> milvus.proto.data.GetPartitionStatisticsResponse + 152, // 272: milvus.proto.data.DataCoord.GetSegmentInfoChannel:output_type -> milvus.proto.milvus.StringResponse + 116, // 273: milvus.proto.data.DataCoord.SaveBinlogPaths:output_type -> milvus.proto.common.Status + 44, // 274: milvus.proto.data.DataCoord.GetRecoveryInfo:output_type -> milvus.proto.data.GetRecoveryInfoResponse + 46, // 275: milvus.proto.data.DataCoord.GetRecoveryInfoV2:output_type -> milvus.proto.data.GetRecoveryInfoResponseV2 + 49, // 276: milvus.proto.data.DataCoord.GetChannelRecoveryInfo:output_type -> milvus.proto.data.GetChannelRecoveryInfoResponse + 53, // 277: milvus.proto.data.DataCoord.GetFlushedSegments:output_type -> milvus.proto.data.GetFlushedSegmentsResponse + 51, // 278: milvus.proto.data.DataCoord.GetSegmentsByStates:output_type -> milvus.proto.data.GetSegmentsByStatesResponse + 153, // 279: milvus.proto.data.DataCoord.GetFlushAllState:output_type -> milvus.proto.milvus.GetFlushAllStateResponse + 154, // 280: milvus.proto.data.DataCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse + 155, // 281: milvus.proto.data.DataCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse + 156, // 282: milvus.proto.data.DataCoord.ManualCompaction:output_type -> milvus.proto.milvus.ManualCompactionResponse + 157, // 283: milvus.proto.data.DataCoord.GetCompactionState:output_type -> milvus.proto.milvus.GetCompactionStateResponse + 158, // 284: milvus.proto.data.DataCoord.GetCompactionStateWithPlans:output_type -> milvus.proto.milvus.GetCompactionPlansResponse + 66, // 285: milvus.proto.data.DataCoord.WatchChannels:output_type -> milvus.proto.data.WatchChannelsResponse + 159, // 286: milvus.proto.data.DataCoord.GetFlushState:output_type -> milvus.proto.milvus.GetFlushStateResponse + 71, // 287: milvus.proto.data.DataCoord.DropVirtualChannel:output_type -> milvus.proto.data.DropVirtualChannelResponse + 68, // 288: milvus.proto.data.DataCoord.SetSegmentState:output_type -> milvus.proto.data.SetSegmentStateResponse + 116, // 289: milvus.proto.data.DataCoord.UpdateSegmentStatistics:output_type -> milvus.proto.common.Status + 116, // 290: milvus.proto.data.DataCoord.UpdateChannelCheckpoint:output_type -> milvus.proto.common.Status + 116, // 291: milvus.proto.data.DataCoord.MarkSegmentsDropped:output_type -> milvus.proto.common.Status + 116, // 292: milvus.proto.data.DataCoord.BroadcastAlteredCollection:output_type -> milvus.proto.common.Status + 160, // 293: milvus.proto.data.DataCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse + 116, // 294: milvus.proto.data.DataCoord.CreateIndex:output_type -> milvus.proto.common.Status + 116, // 295: milvus.proto.data.DataCoord.AlterIndex:output_type -> milvus.proto.common.Status + 161, // 296: milvus.proto.data.DataCoord.GetIndexState:output_type -> milvus.proto.index.GetIndexStateResponse + 162, // 297: milvus.proto.data.DataCoord.GetSegmentIndexState:output_type -> milvus.proto.index.GetSegmentIndexStateResponse + 163, // 298: milvus.proto.data.DataCoord.GetIndexInfos:output_type -> milvus.proto.index.GetIndexInfoResponse + 116, // 299: milvus.proto.data.DataCoord.DropIndex:output_type -> milvus.proto.common.Status + 164, // 300: milvus.proto.data.DataCoord.DescribeIndex:output_type -> milvus.proto.index.DescribeIndexResponse + 165, // 301: milvus.proto.data.DataCoord.GetIndexStatistics:output_type -> milvus.proto.index.GetIndexStatisticsResponse + 166, // 302: milvus.proto.data.DataCoord.GetIndexBuildProgress:output_type -> milvus.proto.index.GetIndexBuildProgressResponse + 167, // 303: milvus.proto.data.DataCoord.ListIndexes:output_type -> milvus.proto.index.ListIndexesResponse + 80, // 304: milvus.proto.data.DataCoord.GcConfirm:output_type -> milvus.proto.data.GcConfirmResponse + 116, // 305: milvus.proto.data.DataCoord.ReportDataNodeTtMsgs:output_type -> milvus.proto.common.Status + 116, // 306: milvus.proto.data.DataCoord.GcControl:output_type -> milvus.proto.common.Status + 168, // 307: milvus.proto.data.DataCoord.ImportV2:output_type -> milvus.proto.internal.ImportResponse + 169, // 308: milvus.proto.data.DataCoord.GetImportProgress:output_type -> milvus.proto.internal.GetImportProgressResponse + 170, // 309: milvus.proto.data.DataCoord.ListImports:output_type -> milvus.proto.internal.ListImportsResponse + 151, // 310: milvus.proto.data.DataNode.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates + 152, // 311: milvus.proto.data.DataNode.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse + 116, // 312: milvus.proto.data.DataNode.WatchDmChannels:output_type -> milvus.proto.common.Status + 116, // 313: milvus.proto.data.DataNode.FlushSegments:output_type -> milvus.proto.common.Status + 154, // 314: milvus.proto.data.DataNode.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse + 155, // 315: milvus.proto.data.DataNode.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse + 116, // 316: milvus.proto.data.DataNode.CompactionV2:output_type -> milvus.proto.common.Status + 63, // 317: milvus.proto.data.DataNode.GetCompactionState:output_type -> milvus.proto.data.CompactionStateResponse + 116, // 318: milvus.proto.data.DataNode.SyncSegments:output_type -> milvus.proto.common.Status + 75, // 319: milvus.proto.data.DataNode.ResendSegmentStats:output_type -> milvus.proto.data.ResendSegmentStatsResponse + 116, // 320: milvus.proto.data.DataNode.FlushChannels:output_type -> milvus.proto.common.Status + 116, // 321: milvus.proto.data.DataNode.NotifyChannelOperation:output_type -> milvus.proto.common.Status + 84, // 322: milvus.proto.data.DataNode.CheckChannelOperationProgress:output_type -> milvus.proto.data.ChannelOperationProgressResponse + 116, // 323: milvus.proto.data.DataNode.PreImport:output_type -> milvus.proto.common.Status + 116, // 324: milvus.proto.data.DataNode.ImportV2:output_type -> milvus.proto.common.Status + 92, // 325: milvus.proto.data.DataNode.QueryPreImport:output_type -> milvus.proto.data.QueryPreImportResponse + 95, // 326: milvus.proto.data.DataNode.QueryImport:output_type -> milvus.proto.data.QueryImportResponse + 116, // 327: milvus.proto.data.DataNode.DropImport:output_type -> milvus.proto.common.Status + 102, // 328: milvus.proto.data.DataNode.QuerySlot:output_type -> milvus.proto.data.QuerySlotResponse + 116, // 329: milvus.proto.data.DataNode.DropCompactionPlan:output_type -> milvus.proto.common.Status + 261, // [261:330] is the sub-list for method output_type + 192, // [192:261] is the sub-list for method input_type + 192, // [192:192] is the sub-list for extension type_name + 192, // [192:192] is the sub-list for extension extendee + 0, // [0:192] is the sub-list for field type_name +} + +func init() { file_data_coord_proto_init() } +func file_data_coord_proto_init() { + if File_data_coord_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_data_coord_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Empty); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*FlushRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*FlushResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*FlushChannelsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentIDRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AllocSegmentRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AllocSegmentResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AssignSegmentIDRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentIDAssignment); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AssignSegmentIDResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetSegmentStatesRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentStateInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetSegmentStatesResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetSegmentInfoRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetSegmentInfoResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetInsertBinlogPathsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetInsertBinlogPathsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetCollectionStatisticsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetCollectionStatisticsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetPartitionStatisticsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetPartitionStatisticsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetSegmentInfoChannelRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*VchannelInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*WatchDmChannelsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*FlushSegmentsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentMsg); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentStartPosition); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SaveBinlogPathsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CheckPoint); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DeltaLogInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ChannelStatus); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DataNodeInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentBinlogs); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*FieldBinlog); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*TextIndexStats); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Binlog); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetRecoveryInfoResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetRecoveryInfoRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetRecoveryInfoResponseV2); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetRecoveryInfoRequestV2); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetChannelRecoveryInfoRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetChannelRecoveryInfoResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetSegmentsByStatesRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetSegmentsByStatesResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetFlushedSegmentsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetFlushedSegmentsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentFlushCompletedMsg); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ChannelWatchInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CompactionStateRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[50].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SyncSegmentInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SyncSegmentsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CompactionSegmentBinlogs); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CompactionPlan); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CompactionSegment); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[55].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CompactionPlanResult); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[56].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CompactionStateResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[57].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentFieldBinlogMeta); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[58].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*WatchChannelsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[59].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*WatchChannelsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[60].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SetSegmentStateRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[61].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SetSegmentStateResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[62].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DropVirtualChannelRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[63].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DropVirtualChannelSegment); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[64].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DropVirtualChannelResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[65].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*UpdateSegmentStatisticsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[66].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*UpdateChannelCheckpointRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[67].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ResendSegmentStatsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[68].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ResendSegmentStatsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[69].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*MarkSegmentsDroppedRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[70].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentReferenceLock); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[71].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AlterCollectionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[72].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GcConfirmRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[73].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GcConfirmResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[74].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ReportDataNodeTtMsgsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[75].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetFlushStateRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[76].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ChannelOperationsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[77].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ChannelOperationProgressResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[78].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PreImportRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[79].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*IDRange); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[80].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ImportRequestSegment); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[81].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ImportRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[82].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QueryPreImportRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[83].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PartitionImportStats); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[84].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ImportFileStats); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[85].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QueryPreImportResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[86].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QueryImportRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[87].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ImportSegmentInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[88].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QueryImportResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[89].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DropImportRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[90].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ImportJob); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[91].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PreImportTask); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[92].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ImportTaskV2); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[93].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GcControlRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[94].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QuerySlotRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[95].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QuerySlotResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[96].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CompactionTask); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[97].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PartitionStatsInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[98].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DropCompactionPlanRequest); 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{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_data_coord_proto_rawDesc, + NumEnums: 7, + NumMessages: 108, + NumExtensions: 0, + NumServices: 2, + }, + GoTypes: file_data_coord_proto_goTypes, + DependencyIndexes: file_data_coord_proto_depIdxs, + EnumInfos: file_data_coord_proto_enumTypes, + MessageInfos: file_data_coord_proto_msgTypes, + }.Build() + File_data_coord_proto = out.File + file_data_coord_proto_rawDesc = nil + file_data_coord_proto_goTypes = nil + file_data_coord_proto_depIdxs = nil +} diff --git a/pkg/proto/datapb/data_coord_grpc.pb.go b/pkg/proto/datapb/data_coord_grpc.pb.go new file mode 100644 index 0000000000000..fc64e5fa91b90 --- /dev/null +++ b/pkg/proto/datapb/data_coord_grpc.pb.go @@ -0,0 +1,2697 @@ +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. +// versions: +// - protoc-gen-go-grpc v1.3.0 +// - protoc v3.21.4 +// source: data_coord.proto + +package datapb + +import ( + context "context" + commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + indexpb "github.com/milvus-io/milvus/pkg/proto/indexpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" +) + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +// Requires gRPC-Go v1.32.0 or later. +const _ = grpc.SupportPackageIsVersion7 + +const ( + DataCoord_GetComponentStates_FullMethodName = "/milvus.proto.data.DataCoord/GetComponentStates" + DataCoord_GetTimeTickChannel_FullMethodName = "/milvus.proto.data.DataCoord/GetTimeTickChannel" + DataCoord_GetStatisticsChannel_FullMethodName = "/milvus.proto.data.DataCoord/GetStatisticsChannel" + DataCoord_Flush_FullMethodName = "/milvus.proto.data.DataCoord/Flush" + DataCoord_AllocSegment_FullMethodName = "/milvus.proto.data.DataCoord/AllocSegment" + DataCoord_AssignSegmentID_FullMethodName = "/milvus.proto.data.DataCoord/AssignSegmentID" + DataCoord_GetSegmentInfo_FullMethodName = "/milvus.proto.data.DataCoord/GetSegmentInfo" + DataCoord_GetSegmentStates_FullMethodName = "/milvus.proto.data.DataCoord/GetSegmentStates" + DataCoord_GetInsertBinlogPaths_FullMethodName = "/milvus.proto.data.DataCoord/GetInsertBinlogPaths" + DataCoord_GetCollectionStatistics_FullMethodName = "/milvus.proto.data.DataCoord/GetCollectionStatistics" + DataCoord_GetPartitionStatistics_FullMethodName = "/milvus.proto.data.DataCoord/GetPartitionStatistics" + DataCoord_GetSegmentInfoChannel_FullMethodName = "/milvus.proto.data.DataCoord/GetSegmentInfoChannel" + DataCoord_SaveBinlogPaths_FullMethodName = "/milvus.proto.data.DataCoord/SaveBinlogPaths" + DataCoord_GetRecoveryInfo_FullMethodName = "/milvus.proto.data.DataCoord/GetRecoveryInfo" + DataCoord_GetRecoveryInfoV2_FullMethodName = "/milvus.proto.data.DataCoord/GetRecoveryInfoV2" + DataCoord_GetChannelRecoveryInfo_FullMethodName = "/milvus.proto.data.DataCoord/GetChannelRecoveryInfo" + DataCoord_GetFlushedSegments_FullMethodName = "/milvus.proto.data.DataCoord/GetFlushedSegments" + DataCoord_GetSegmentsByStates_FullMethodName = "/milvus.proto.data.DataCoord/GetSegmentsByStates" + DataCoord_GetFlushAllState_FullMethodName = "/milvus.proto.data.DataCoord/GetFlushAllState" + DataCoord_ShowConfigurations_FullMethodName = "/milvus.proto.data.DataCoord/ShowConfigurations" + DataCoord_GetMetrics_FullMethodName = "/milvus.proto.data.DataCoord/GetMetrics" + DataCoord_ManualCompaction_FullMethodName = "/milvus.proto.data.DataCoord/ManualCompaction" + DataCoord_GetCompactionState_FullMethodName = "/milvus.proto.data.DataCoord/GetCompactionState" + DataCoord_GetCompactionStateWithPlans_FullMethodName = "/milvus.proto.data.DataCoord/GetCompactionStateWithPlans" + DataCoord_WatchChannels_FullMethodName = "/milvus.proto.data.DataCoord/WatchChannels" + DataCoord_GetFlushState_FullMethodName = "/milvus.proto.data.DataCoord/GetFlushState" + DataCoord_DropVirtualChannel_FullMethodName = "/milvus.proto.data.DataCoord/DropVirtualChannel" + DataCoord_SetSegmentState_FullMethodName = "/milvus.proto.data.DataCoord/SetSegmentState" + DataCoord_UpdateSegmentStatistics_FullMethodName = "/milvus.proto.data.DataCoord/UpdateSegmentStatistics" + DataCoord_UpdateChannelCheckpoint_FullMethodName = "/milvus.proto.data.DataCoord/UpdateChannelCheckpoint" + DataCoord_MarkSegmentsDropped_FullMethodName = "/milvus.proto.data.DataCoord/MarkSegmentsDropped" + DataCoord_BroadcastAlteredCollection_FullMethodName = "/milvus.proto.data.DataCoord/BroadcastAlteredCollection" + DataCoord_CheckHealth_FullMethodName = "/milvus.proto.data.DataCoord/CheckHealth" + DataCoord_CreateIndex_FullMethodName = "/milvus.proto.data.DataCoord/CreateIndex" + DataCoord_AlterIndex_FullMethodName = "/milvus.proto.data.DataCoord/AlterIndex" + DataCoord_GetIndexState_FullMethodName = "/milvus.proto.data.DataCoord/GetIndexState" + DataCoord_GetSegmentIndexState_FullMethodName = "/milvus.proto.data.DataCoord/GetSegmentIndexState" + DataCoord_GetIndexInfos_FullMethodName = "/milvus.proto.data.DataCoord/GetIndexInfos" + DataCoord_DropIndex_FullMethodName = "/milvus.proto.data.DataCoord/DropIndex" + DataCoord_DescribeIndex_FullMethodName = "/milvus.proto.data.DataCoord/DescribeIndex" + DataCoord_GetIndexStatistics_FullMethodName = "/milvus.proto.data.DataCoord/GetIndexStatistics" + DataCoord_GetIndexBuildProgress_FullMethodName = "/milvus.proto.data.DataCoord/GetIndexBuildProgress" + DataCoord_ListIndexes_FullMethodName = "/milvus.proto.data.DataCoord/ListIndexes" + DataCoord_GcConfirm_FullMethodName = "/milvus.proto.data.DataCoord/GcConfirm" + DataCoord_ReportDataNodeTtMsgs_FullMethodName = "/milvus.proto.data.DataCoord/ReportDataNodeTtMsgs" + DataCoord_GcControl_FullMethodName = "/milvus.proto.data.DataCoord/GcControl" + DataCoord_ImportV2_FullMethodName = "/milvus.proto.data.DataCoord/ImportV2" + DataCoord_GetImportProgress_FullMethodName = "/milvus.proto.data.DataCoord/GetImportProgress" + DataCoord_ListImports_FullMethodName = "/milvus.proto.data.DataCoord/ListImports" +) + +// DataCoordClient is the client API for DataCoord service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +type DataCoordClient interface { + GetComponentStates(ctx context.Context, in *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) + GetTimeTickChannel(ctx context.Context, in *internalpb.GetTimeTickChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) + GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) + Flush(ctx context.Context, in *FlushRequest, opts ...grpc.CallOption) (*FlushResponse, error) + // AllocSegment alloc a new growing segment, add it into segment meta. + AllocSegment(ctx context.Context, in *AllocSegmentRequest, opts ...grpc.CallOption) (*AllocSegmentResponse, error) + // Deprecated: Do not use. + AssignSegmentID(ctx context.Context, in *AssignSegmentIDRequest, opts ...grpc.CallOption) (*AssignSegmentIDResponse, error) + GetSegmentInfo(ctx context.Context, in *GetSegmentInfoRequest, opts ...grpc.CallOption) (*GetSegmentInfoResponse, error) + GetSegmentStates(ctx context.Context, in *GetSegmentStatesRequest, opts ...grpc.CallOption) (*GetSegmentStatesResponse, error) + GetInsertBinlogPaths(ctx context.Context, in *GetInsertBinlogPathsRequest, opts ...grpc.CallOption) (*GetInsertBinlogPathsResponse, error) + GetCollectionStatistics(ctx context.Context, in *GetCollectionStatisticsRequest, opts ...grpc.CallOption) (*GetCollectionStatisticsResponse, error) + GetPartitionStatistics(ctx context.Context, in *GetPartitionStatisticsRequest, opts ...grpc.CallOption) (*GetPartitionStatisticsResponse, error) + GetSegmentInfoChannel(ctx context.Context, in *GetSegmentInfoChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) + SaveBinlogPaths(ctx context.Context, in *SaveBinlogPathsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + GetRecoveryInfo(ctx context.Context, in *GetRecoveryInfoRequest, opts ...grpc.CallOption) (*GetRecoveryInfoResponse, error) + GetRecoveryInfoV2(ctx context.Context, in *GetRecoveryInfoRequestV2, opts ...grpc.CallOption) (*GetRecoveryInfoResponseV2, error) + GetChannelRecoveryInfo(ctx context.Context, in *GetChannelRecoveryInfoRequest, opts ...grpc.CallOption) (*GetChannelRecoveryInfoResponse, error) + GetFlushedSegments(ctx context.Context, in *GetFlushedSegmentsRequest, opts ...grpc.CallOption) (*GetFlushedSegmentsResponse, error) + GetSegmentsByStates(ctx context.Context, in *GetSegmentsByStatesRequest, opts ...grpc.CallOption) (*GetSegmentsByStatesResponse, error) + GetFlushAllState(ctx context.Context, in *milvuspb.GetFlushAllStateRequest, opts ...grpc.CallOption) (*milvuspb.GetFlushAllStateResponse, error) + ShowConfigurations(ctx context.Context, in *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error) + // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy + GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) + ManualCompaction(ctx context.Context, in *milvuspb.ManualCompactionRequest, opts ...grpc.CallOption) (*milvuspb.ManualCompactionResponse, error) + GetCompactionState(ctx context.Context, in *milvuspb.GetCompactionStateRequest, opts ...grpc.CallOption) (*milvuspb.GetCompactionStateResponse, error) + GetCompactionStateWithPlans(ctx context.Context, in *milvuspb.GetCompactionPlansRequest, opts ...grpc.CallOption) (*milvuspb.GetCompactionPlansResponse, error) + WatchChannels(ctx context.Context, in *WatchChannelsRequest, opts ...grpc.CallOption) (*WatchChannelsResponse, error) + GetFlushState(ctx context.Context, in *GetFlushStateRequest, opts ...grpc.CallOption) (*milvuspb.GetFlushStateResponse, error) + DropVirtualChannel(ctx context.Context, in *DropVirtualChannelRequest, opts ...grpc.CallOption) (*DropVirtualChannelResponse, error) + SetSegmentState(ctx context.Context, in *SetSegmentStateRequest, opts ...grpc.CallOption) (*SetSegmentStateResponse, error) + UpdateSegmentStatistics(ctx context.Context, in *UpdateSegmentStatisticsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + UpdateChannelCheckpoint(ctx context.Context, in *UpdateChannelCheckpointRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + MarkSegmentsDropped(ctx context.Context, in *MarkSegmentsDroppedRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + BroadcastAlteredCollection(ctx context.Context, in *AlterCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + CheckHealth(ctx context.Context, in *milvuspb.CheckHealthRequest, opts ...grpc.CallOption) (*milvuspb.CheckHealthResponse, error) + CreateIndex(ctx context.Context, in *indexpb.CreateIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + AlterIndex(ctx context.Context, in *indexpb.AlterIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + // Deprecated: use DescribeIndex instead + GetIndexState(ctx context.Context, in *indexpb.GetIndexStateRequest, opts ...grpc.CallOption) (*indexpb.GetIndexStateResponse, error) + GetSegmentIndexState(ctx context.Context, in *indexpb.GetSegmentIndexStateRequest, opts ...grpc.CallOption) (*indexpb.GetSegmentIndexStateResponse, error) + GetIndexInfos(ctx context.Context, in *indexpb.GetIndexInfoRequest, opts ...grpc.CallOption) (*indexpb.GetIndexInfoResponse, error) + DropIndex(ctx context.Context, in *indexpb.DropIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + DescribeIndex(ctx context.Context, in *indexpb.DescribeIndexRequest, opts ...grpc.CallOption) (*indexpb.DescribeIndexResponse, error) + GetIndexStatistics(ctx context.Context, in *indexpb.GetIndexStatisticsRequest, opts ...grpc.CallOption) (*indexpb.GetIndexStatisticsResponse, error) + // Deprecated: use DescribeIndex instead + GetIndexBuildProgress(ctx context.Context, in *indexpb.GetIndexBuildProgressRequest, opts ...grpc.CallOption) (*indexpb.GetIndexBuildProgressResponse, error) + ListIndexes(ctx context.Context, in *indexpb.ListIndexesRequest, opts ...grpc.CallOption) (*indexpb.ListIndexesResponse, error) + GcConfirm(ctx context.Context, in *GcConfirmRequest, opts ...grpc.CallOption) (*GcConfirmResponse, error) + ReportDataNodeTtMsgs(ctx context.Context, in *ReportDataNodeTtMsgsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + GcControl(ctx context.Context, in *GcControlRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + // importV2 + ImportV2(ctx context.Context, in *internalpb.ImportRequestInternal, opts ...grpc.CallOption) (*internalpb.ImportResponse, error) + GetImportProgress(ctx context.Context, in *internalpb.GetImportProgressRequest, opts ...grpc.CallOption) (*internalpb.GetImportProgressResponse, error) + ListImports(ctx context.Context, in *internalpb.ListImportsRequestInternal, opts ...grpc.CallOption) (*internalpb.ListImportsResponse, error) +} + +type dataCoordClient struct { + cc grpc.ClientConnInterface +} + +func NewDataCoordClient(cc grpc.ClientConnInterface) DataCoordClient { + return &dataCoordClient{cc} +} + +func (c *dataCoordClient) GetComponentStates(ctx context.Context, in *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) { + out := new(milvuspb.ComponentStates) + err := c.cc.Invoke(ctx, DataCoord_GetComponentStates_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GetTimeTickChannel(ctx context.Context, in *internalpb.GetTimeTickChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) { + out := new(milvuspb.StringResponse) + err := c.cc.Invoke(ctx, DataCoord_GetTimeTickChannel_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) { + out := new(milvuspb.StringResponse) + err := c.cc.Invoke(ctx, DataCoord_GetStatisticsChannel_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) Flush(ctx context.Context, in *FlushRequest, opts ...grpc.CallOption) (*FlushResponse, error) { + out := new(FlushResponse) + err := c.cc.Invoke(ctx, DataCoord_Flush_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) AllocSegment(ctx context.Context, in *AllocSegmentRequest, opts ...grpc.CallOption) (*AllocSegmentResponse, error) { + out := new(AllocSegmentResponse) + err := c.cc.Invoke(ctx, DataCoord_AllocSegment_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// Deprecated: Do not use. +func (c *dataCoordClient) AssignSegmentID(ctx context.Context, in *AssignSegmentIDRequest, opts ...grpc.CallOption) (*AssignSegmentIDResponse, error) { + out := new(AssignSegmentIDResponse) + err := c.cc.Invoke(ctx, DataCoord_AssignSegmentID_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GetSegmentInfo(ctx context.Context, in *GetSegmentInfoRequest, opts ...grpc.CallOption) (*GetSegmentInfoResponse, error) { + out := new(GetSegmentInfoResponse) + err := c.cc.Invoke(ctx, DataCoord_GetSegmentInfo_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GetSegmentStates(ctx context.Context, in *GetSegmentStatesRequest, opts ...grpc.CallOption) (*GetSegmentStatesResponse, error) { + out := new(GetSegmentStatesResponse) + err := c.cc.Invoke(ctx, DataCoord_GetSegmentStates_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GetInsertBinlogPaths(ctx context.Context, in *GetInsertBinlogPathsRequest, opts ...grpc.CallOption) (*GetInsertBinlogPathsResponse, error) { + out := new(GetInsertBinlogPathsResponse) + err := c.cc.Invoke(ctx, DataCoord_GetInsertBinlogPaths_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GetCollectionStatistics(ctx context.Context, in *GetCollectionStatisticsRequest, opts ...grpc.CallOption) (*GetCollectionStatisticsResponse, error) { + out := new(GetCollectionStatisticsResponse) + err := c.cc.Invoke(ctx, DataCoord_GetCollectionStatistics_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GetPartitionStatistics(ctx context.Context, in *GetPartitionStatisticsRequest, opts ...grpc.CallOption) (*GetPartitionStatisticsResponse, error) { + out := new(GetPartitionStatisticsResponse) + err := c.cc.Invoke(ctx, DataCoord_GetPartitionStatistics_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GetSegmentInfoChannel(ctx context.Context, in *GetSegmentInfoChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) { + out := new(milvuspb.StringResponse) + err := c.cc.Invoke(ctx, DataCoord_GetSegmentInfoChannel_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) SaveBinlogPaths(ctx context.Context, in *SaveBinlogPathsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, DataCoord_SaveBinlogPaths_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GetRecoveryInfo(ctx context.Context, in *GetRecoveryInfoRequest, opts ...grpc.CallOption) (*GetRecoveryInfoResponse, error) { + out := new(GetRecoveryInfoResponse) + err := c.cc.Invoke(ctx, DataCoord_GetRecoveryInfo_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GetRecoveryInfoV2(ctx context.Context, in *GetRecoveryInfoRequestV2, opts ...grpc.CallOption) (*GetRecoveryInfoResponseV2, error) { + out := new(GetRecoveryInfoResponseV2) + err := c.cc.Invoke(ctx, DataCoord_GetRecoveryInfoV2_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GetChannelRecoveryInfo(ctx context.Context, in *GetChannelRecoveryInfoRequest, opts ...grpc.CallOption) (*GetChannelRecoveryInfoResponse, error) { + out := new(GetChannelRecoveryInfoResponse) + err := c.cc.Invoke(ctx, DataCoord_GetChannelRecoveryInfo_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GetFlushedSegments(ctx context.Context, in *GetFlushedSegmentsRequest, opts ...grpc.CallOption) (*GetFlushedSegmentsResponse, error) { + out := new(GetFlushedSegmentsResponse) + err := c.cc.Invoke(ctx, DataCoord_GetFlushedSegments_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GetSegmentsByStates(ctx context.Context, in *GetSegmentsByStatesRequest, opts ...grpc.CallOption) (*GetSegmentsByStatesResponse, error) { + out := new(GetSegmentsByStatesResponse) + err := c.cc.Invoke(ctx, DataCoord_GetSegmentsByStates_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GetFlushAllState(ctx context.Context, in *milvuspb.GetFlushAllStateRequest, opts ...grpc.CallOption) (*milvuspb.GetFlushAllStateResponse, error) { + out := new(milvuspb.GetFlushAllStateResponse) + err := c.cc.Invoke(ctx, DataCoord_GetFlushAllState_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) ShowConfigurations(ctx context.Context, in *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error) { + out := new(internalpb.ShowConfigurationsResponse) + err := c.cc.Invoke(ctx, DataCoord_ShowConfigurations_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) { + out := new(milvuspb.GetMetricsResponse) + err := c.cc.Invoke(ctx, DataCoord_GetMetrics_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) ManualCompaction(ctx context.Context, in *milvuspb.ManualCompactionRequest, opts ...grpc.CallOption) (*milvuspb.ManualCompactionResponse, error) { + out := new(milvuspb.ManualCompactionResponse) + err := c.cc.Invoke(ctx, DataCoord_ManualCompaction_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GetCompactionState(ctx context.Context, in *milvuspb.GetCompactionStateRequest, opts ...grpc.CallOption) (*milvuspb.GetCompactionStateResponse, error) { + out := new(milvuspb.GetCompactionStateResponse) + err := c.cc.Invoke(ctx, DataCoord_GetCompactionState_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GetCompactionStateWithPlans(ctx context.Context, in *milvuspb.GetCompactionPlansRequest, opts ...grpc.CallOption) (*milvuspb.GetCompactionPlansResponse, error) { + out := new(milvuspb.GetCompactionPlansResponse) + err := c.cc.Invoke(ctx, DataCoord_GetCompactionStateWithPlans_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) WatchChannels(ctx context.Context, in *WatchChannelsRequest, opts ...grpc.CallOption) (*WatchChannelsResponse, error) { + out := new(WatchChannelsResponse) + err := c.cc.Invoke(ctx, DataCoord_WatchChannels_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GetFlushState(ctx context.Context, in *GetFlushStateRequest, opts ...grpc.CallOption) (*milvuspb.GetFlushStateResponse, error) { + out := new(milvuspb.GetFlushStateResponse) + err := c.cc.Invoke(ctx, DataCoord_GetFlushState_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) DropVirtualChannel(ctx context.Context, in *DropVirtualChannelRequest, opts ...grpc.CallOption) (*DropVirtualChannelResponse, error) { + out := new(DropVirtualChannelResponse) + err := c.cc.Invoke(ctx, DataCoord_DropVirtualChannel_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) SetSegmentState(ctx context.Context, in *SetSegmentStateRequest, opts ...grpc.CallOption) (*SetSegmentStateResponse, error) { + out := new(SetSegmentStateResponse) + err := c.cc.Invoke(ctx, DataCoord_SetSegmentState_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) UpdateSegmentStatistics(ctx context.Context, in *UpdateSegmentStatisticsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, DataCoord_UpdateSegmentStatistics_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) UpdateChannelCheckpoint(ctx context.Context, in *UpdateChannelCheckpointRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, DataCoord_UpdateChannelCheckpoint_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) MarkSegmentsDropped(ctx context.Context, in *MarkSegmentsDroppedRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, DataCoord_MarkSegmentsDropped_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) BroadcastAlteredCollection(ctx context.Context, in *AlterCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, DataCoord_BroadcastAlteredCollection_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) CheckHealth(ctx context.Context, in *milvuspb.CheckHealthRequest, opts ...grpc.CallOption) (*milvuspb.CheckHealthResponse, error) { + out := new(milvuspb.CheckHealthResponse) + err := c.cc.Invoke(ctx, DataCoord_CheckHealth_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) CreateIndex(ctx context.Context, in *indexpb.CreateIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, DataCoord_CreateIndex_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) AlterIndex(ctx context.Context, in *indexpb.AlterIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, DataCoord_AlterIndex_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GetIndexState(ctx context.Context, in *indexpb.GetIndexStateRequest, opts ...grpc.CallOption) (*indexpb.GetIndexStateResponse, error) { + out := new(indexpb.GetIndexStateResponse) + err := c.cc.Invoke(ctx, DataCoord_GetIndexState_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GetSegmentIndexState(ctx context.Context, in *indexpb.GetSegmentIndexStateRequest, opts ...grpc.CallOption) (*indexpb.GetSegmentIndexStateResponse, error) { + out := new(indexpb.GetSegmentIndexStateResponse) + err := c.cc.Invoke(ctx, DataCoord_GetSegmentIndexState_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GetIndexInfos(ctx context.Context, in *indexpb.GetIndexInfoRequest, opts ...grpc.CallOption) (*indexpb.GetIndexInfoResponse, error) { + out := new(indexpb.GetIndexInfoResponse) + err := c.cc.Invoke(ctx, DataCoord_GetIndexInfos_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) DropIndex(ctx context.Context, in *indexpb.DropIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, DataCoord_DropIndex_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) DescribeIndex(ctx context.Context, in *indexpb.DescribeIndexRequest, opts ...grpc.CallOption) (*indexpb.DescribeIndexResponse, error) { + out := new(indexpb.DescribeIndexResponse) + err := c.cc.Invoke(ctx, DataCoord_DescribeIndex_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GetIndexStatistics(ctx context.Context, in *indexpb.GetIndexStatisticsRequest, opts ...grpc.CallOption) (*indexpb.GetIndexStatisticsResponse, error) { + out := new(indexpb.GetIndexStatisticsResponse) + err := c.cc.Invoke(ctx, DataCoord_GetIndexStatistics_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GetIndexBuildProgress(ctx context.Context, in *indexpb.GetIndexBuildProgressRequest, opts ...grpc.CallOption) (*indexpb.GetIndexBuildProgressResponse, error) { + out := new(indexpb.GetIndexBuildProgressResponse) + err := c.cc.Invoke(ctx, DataCoord_GetIndexBuildProgress_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) ListIndexes(ctx context.Context, in *indexpb.ListIndexesRequest, opts ...grpc.CallOption) (*indexpb.ListIndexesResponse, error) { + out := new(indexpb.ListIndexesResponse) + err := c.cc.Invoke(ctx, DataCoord_ListIndexes_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GcConfirm(ctx context.Context, in *GcConfirmRequest, opts ...grpc.CallOption) (*GcConfirmResponse, error) { + out := new(GcConfirmResponse) + err := c.cc.Invoke(ctx, DataCoord_GcConfirm_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) ReportDataNodeTtMsgs(ctx context.Context, in *ReportDataNodeTtMsgsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, DataCoord_ReportDataNodeTtMsgs_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GcControl(ctx context.Context, in *GcControlRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, DataCoord_GcControl_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) ImportV2(ctx context.Context, in *internalpb.ImportRequestInternal, opts ...grpc.CallOption) (*internalpb.ImportResponse, error) { + out := new(internalpb.ImportResponse) + err := c.cc.Invoke(ctx, DataCoord_ImportV2_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GetImportProgress(ctx context.Context, in *internalpb.GetImportProgressRequest, opts ...grpc.CallOption) (*internalpb.GetImportProgressResponse, error) { + out := new(internalpb.GetImportProgressResponse) + err := c.cc.Invoke(ctx, DataCoord_GetImportProgress_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) ListImports(ctx context.Context, in *internalpb.ListImportsRequestInternal, opts ...grpc.CallOption) (*internalpb.ListImportsResponse, error) { + out := new(internalpb.ListImportsResponse) + err := c.cc.Invoke(ctx, DataCoord_ListImports_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// DataCoordServer is the server API for DataCoord service. +// All implementations should embed UnimplementedDataCoordServer +// for forward compatibility +type DataCoordServer interface { + GetComponentStates(context.Context, *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) + GetTimeTickChannel(context.Context, *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error) + GetStatisticsChannel(context.Context, *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) + Flush(context.Context, *FlushRequest) (*FlushResponse, error) + // AllocSegment alloc a new growing segment, add it into segment meta. + AllocSegment(context.Context, *AllocSegmentRequest) (*AllocSegmentResponse, error) + // Deprecated: Do not use. + AssignSegmentID(context.Context, *AssignSegmentIDRequest) (*AssignSegmentIDResponse, error) + GetSegmentInfo(context.Context, *GetSegmentInfoRequest) (*GetSegmentInfoResponse, error) + GetSegmentStates(context.Context, *GetSegmentStatesRequest) (*GetSegmentStatesResponse, error) + GetInsertBinlogPaths(context.Context, *GetInsertBinlogPathsRequest) (*GetInsertBinlogPathsResponse, error) + GetCollectionStatistics(context.Context, *GetCollectionStatisticsRequest) (*GetCollectionStatisticsResponse, error) + GetPartitionStatistics(context.Context, *GetPartitionStatisticsRequest) (*GetPartitionStatisticsResponse, error) + GetSegmentInfoChannel(context.Context, *GetSegmentInfoChannelRequest) (*milvuspb.StringResponse, error) + SaveBinlogPaths(context.Context, *SaveBinlogPathsRequest) (*commonpb.Status, error) + GetRecoveryInfo(context.Context, *GetRecoveryInfoRequest) (*GetRecoveryInfoResponse, error) + GetRecoveryInfoV2(context.Context, *GetRecoveryInfoRequestV2) (*GetRecoveryInfoResponseV2, error) + GetChannelRecoveryInfo(context.Context, *GetChannelRecoveryInfoRequest) (*GetChannelRecoveryInfoResponse, error) + GetFlushedSegments(context.Context, *GetFlushedSegmentsRequest) (*GetFlushedSegmentsResponse, error) + GetSegmentsByStates(context.Context, *GetSegmentsByStatesRequest) (*GetSegmentsByStatesResponse, error) + GetFlushAllState(context.Context, *milvuspb.GetFlushAllStateRequest) (*milvuspb.GetFlushAllStateResponse, error) + ShowConfigurations(context.Context, *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) + // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy + GetMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) + ManualCompaction(context.Context, *milvuspb.ManualCompactionRequest) (*milvuspb.ManualCompactionResponse, error) + GetCompactionState(context.Context, *milvuspb.GetCompactionStateRequest) (*milvuspb.GetCompactionStateResponse, error) + GetCompactionStateWithPlans(context.Context, *milvuspb.GetCompactionPlansRequest) (*milvuspb.GetCompactionPlansResponse, error) + WatchChannels(context.Context, *WatchChannelsRequest) (*WatchChannelsResponse, error) + GetFlushState(context.Context, *GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error) + DropVirtualChannel(context.Context, *DropVirtualChannelRequest) (*DropVirtualChannelResponse, error) + SetSegmentState(context.Context, *SetSegmentStateRequest) (*SetSegmentStateResponse, error) + UpdateSegmentStatistics(context.Context, *UpdateSegmentStatisticsRequest) (*commonpb.Status, error) + UpdateChannelCheckpoint(context.Context, *UpdateChannelCheckpointRequest) (*commonpb.Status, error) + MarkSegmentsDropped(context.Context, *MarkSegmentsDroppedRequest) (*commonpb.Status, error) + BroadcastAlteredCollection(context.Context, *AlterCollectionRequest) (*commonpb.Status, error) + CheckHealth(context.Context, *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error) + CreateIndex(context.Context, *indexpb.CreateIndexRequest) (*commonpb.Status, error) + AlterIndex(context.Context, *indexpb.AlterIndexRequest) (*commonpb.Status, error) + // Deprecated: use DescribeIndex instead + GetIndexState(context.Context, *indexpb.GetIndexStateRequest) (*indexpb.GetIndexStateResponse, error) + GetSegmentIndexState(context.Context, *indexpb.GetSegmentIndexStateRequest) (*indexpb.GetSegmentIndexStateResponse, error) + GetIndexInfos(context.Context, *indexpb.GetIndexInfoRequest) (*indexpb.GetIndexInfoResponse, error) + DropIndex(context.Context, *indexpb.DropIndexRequest) (*commonpb.Status, error) + DescribeIndex(context.Context, *indexpb.DescribeIndexRequest) (*indexpb.DescribeIndexResponse, error) + GetIndexStatistics(context.Context, *indexpb.GetIndexStatisticsRequest) (*indexpb.GetIndexStatisticsResponse, error) + // Deprecated: use DescribeIndex instead + GetIndexBuildProgress(context.Context, *indexpb.GetIndexBuildProgressRequest) (*indexpb.GetIndexBuildProgressResponse, error) + ListIndexes(context.Context, *indexpb.ListIndexesRequest) (*indexpb.ListIndexesResponse, error) + GcConfirm(context.Context, *GcConfirmRequest) (*GcConfirmResponse, error) + ReportDataNodeTtMsgs(context.Context, *ReportDataNodeTtMsgsRequest) (*commonpb.Status, error) + GcControl(context.Context, *GcControlRequest) (*commonpb.Status, error) + // importV2 + ImportV2(context.Context, *internalpb.ImportRequestInternal) (*internalpb.ImportResponse, error) + GetImportProgress(context.Context, *internalpb.GetImportProgressRequest) (*internalpb.GetImportProgressResponse, error) + ListImports(context.Context, *internalpb.ListImportsRequestInternal) (*internalpb.ListImportsResponse, error) +} + +// UnimplementedDataCoordServer should be embedded to have forward compatible implementations. +type UnimplementedDataCoordServer struct { +} + +func (UnimplementedDataCoordServer) GetComponentStates(context.Context, *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetComponentStates not implemented") +} +func (UnimplementedDataCoordServer) GetTimeTickChannel(context.Context, *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetTimeTickChannel not implemented") +} +func (UnimplementedDataCoordServer) GetStatisticsChannel(context.Context, *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetStatisticsChannel not implemented") +} +func (UnimplementedDataCoordServer) Flush(context.Context, *FlushRequest) (*FlushResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method Flush not implemented") +} +func (UnimplementedDataCoordServer) AllocSegment(context.Context, *AllocSegmentRequest) (*AllocSegmentResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method AllocSegment not implemented") +} +func (UnimplementedDataCoordServer) AssignSegmentID(context.Context, *AssignSegmentIDRequest) (*AssignSegmentIDResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method AssignSegmentID not implemented") +} +func (UnimplementedDataCoordServer) GetSegmentInfo(context.Context, *GetSegmentInfoRequest) (*GetSegmentInfoResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetSegmentInfo not implemented") +} +func (UnimplementedDataCoordServer) GetSegmentStates(context.Context, *GetSegmentStatesRequest) (*GetSegmentStatesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetSegmentStates not implemented") +} +func (UnimplementedDataCoordServer) GetInsertBinlogPaths(context.Context, *GetInsertBinlogPathsRequest) (*GetInsertBinlogPathsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetInsertBinlogPaths not implemented") +} +func (UnimplementedDataCoordServer) GetCollectionStatistics(context.Context, *GetCollectionStatisticsRequest) (*GetCollectionStatisticsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetCollectionStatistics not implemented") +} +func (UnimplementedDataCoordServer) GetPartitionStatistics(context.Context, *GetPartitionStatisticsRequest) (*GetPartitionStatisticsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetPartitionStatistics not implemented") +} +func (UnimplementedDataCoordServer) GetSegmentInfoChannel(context.Context, *GetSegmentInfoChannelRequest) (*milvuspb.StringResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetSegmentInfoChannel not implemented") +} +func (UnimplementedDataCoordServer) SaveBinlogPaths(context.Context, *SaveBinlogPathsRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method SaveBinlogPaths not implemented") +} +func (UnimplementedDataCoordServer) GetRecoveryInfo(context.Context, *GetRecoveryInfoRequest) (*GetRecoveryInfoResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetRecoveryInfo not implemented") +} +func (UnimplementedDataCoordServer) GetRecoveryInfoV2(context.Context, *GetRecoveryInfoRequestV2) (*GetRecoveryInfoResponseV2, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetRecoveryInfoV2 not implemented") +} +func (UnimplementedDataCoordServer) GetChannelRecoveryInfo(context.Context, *GetChannelRecoveryInfoRequest) (*GetChannelRecoveryInfoResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetChannelRecoveryInfo not implemented") +} +func (UnimplementedDataCoordServer) GetFlushedSegments(context.Context, *GetFlushedSegmentsRequest) (*GetFlushedSegmentsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetFlushedSegments not implemented") +} +func (UnimplementedDataCoordServer) GetSegmentsByStates(context.Context, *GetSegmentsByStatesRequest) (*GetSegmentsByStatesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetSegmentsByStates not implemented") +} +func (UnimplementedDataCoordServer) GetFlushAllState(context.Context, *milvuspb.GetFlushAllStateRequest) (*milvuspb.GetFlushAllStateResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetFlushAllState not implemented") +} +func (UnimplementedDataCoordServer) ShowConfigurations(context.Context, *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ShowConfigurations not implemented") +} +func (UnimplementedDataCoordServer) GetMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetMetrics not implemented") +} +func (UnimplementedDataCoordServer) ManualCompaction(context.Context, *milvuspb.ManualCompactionRequest) (*milvuspb.ManualCompactionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ManualCompaction not implemented") +} +func (UnimplementedDataCoordServer) GetCompactionState(context.Context, *milvuspb.GetCompactionStateRequest) (*milvuspb.GetCompactionStateResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetCompactionState not implemented") +} +func (UnimplementedDataCoordServer) GetCompactionStateWithPlans(context.Context, *milvuspb.GetCompactionPlansRequest) (*milvuspb.GetCompactionPlansResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetCompactionStateWithPlans not implemented") +} +func (UnimplementedDataCoordServer) WatchChannels(context.Context, *WatchChannelsRequest) (*WatchChannelsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method WatchChannels not implemented") +} +func (UnimplementedDataCoordServer) GetFlushState(context.Context, *GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetFlushState not implemented") +} +func (UnimplementedDataCoordServer) DropVirtualChannel(context.Context, *DropVirtualChannelRequest) (*DropVirtualChannelResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DropVirtualChannel not implemented") +} +func (UnimplementedDataCoordServer) SetSegmentState(context.Context, *SetSegmentStateRequest) (*SetSegmentStateResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method SetSegmentState not implemented") +} +func (UnimplementedDataCoordServer) UpdateSegmentStatistics(context.Context, *UpdateSegmentStatisticsRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateSegmentStatistics not implemented") +} +func (UnimplementedDataCoordServer) UpdateChannelCheckpoint(context.Context, *UpdateChannelCheckpointRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateChannelCheckpoint not implemented") +} +func (UnimplementedDataCoordServer) MarkSegmentsDropped(context.Context, *MarkSegmentsDroppedRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method MarkSegmentsDropped not implemented") +} +func (UnimplementedDataCoordServer) BroadcastAlteredCollection(context.Context, *AlterCollectionRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method BroadcastAlteredCollection not implemented") +} +func (UnimplementedDataCoordServer) CheckHealth(context.Context, *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method CheckHealth not implemented") +} +func (UnimplementedDataCoordServer) CreateIndex(context.Context, *indexpb.CreateIndexRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method CreateIndex not implemented") +} +func (UnimplementedDataCoordServer) AlterIndex(context.Context, *indexpb.AlterIndexRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method AlterIndex not implemented") +} +func (UnimplementedDataCoordServer) GetIndexState(context.Context, *indexpb.GetIndexStateRequest) (*indexpb.GetIndexStateResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetIndexState not implemented") +} +func (UnimplementedDataCoordServer) GetSegmentIndexState(context.Context, *indexpb.GetSegmentIndexStateRequest) (*indexpb.GetSegmentIndexStateResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetSegmentIndexState not implemented") +} +func (UnimplementedDataCoordServer) GetIndexInfos(context.Context, *indexpb.GetIndexInfoRequest) (*indexpb.GetIndexInfoResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetIndexInfos not implemented") +} +func (UnimplementedDataCoordServer) DropIndex(context.Context, *indexpb.DropIndexRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method DropIndex not implemented") +} +func (UnimplementedDataCoordServer) DescribeIndex(context.Context, *indexpb.DescribeIndexRequest) (*indexpb.DescribeIndexResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DescribeIndex not implemented") +} +func (UnimplementedDataCoordServer) GetIndexStatistics(context.Context, *indexpb.GetIndexStatisticsRequest) (*indexpb.GetIndexStatisticsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetIndexStatistics not implemented") +} +func (UnimplementedDataCoordServer) GetIndexBuildProgress(context.Context, *indexpb.GetIndexBuildProgressRequest) (*indexpb.GetIndexBuildProgressResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetIndexBuildProgress not implemented") +} +func (UnimplementedDataCoordServer) ListIndexes(context.Context, *indexpb.ListIndexesRequest) (*indexpb.ListIndexesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListIndexes not implemented") +} +func (UnimplementedDataCoordServer) GcConfirm(context.Context, *GcConfirmRequest) (*GcConfirmResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GcConfirm not implemented") +} +func (UnimplementedDataCoordServer) ReportDataNodeTtMsgs(context.Context, *ReportDataNodeTtMsgsRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method ReportDataNodeTtMsgs not implemented") +} +func (UnimplementedDataCoordServer) GcControl(context.Context, *GcControlRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method GcControl not implemented") +} +func (UnimplementedDataCoordServer) ImportV2(context.Context, *internalpb.ImportRequestInternal) (*internalpb.ImportResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ImportV2 not implemented") +} +func (UnimplementedDataCoordServer) GetImportProgress(context.Context, *internalpb.GetImportProgressRequest) (*internalpb.GetImportProgressResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetImportProgress not implemented") +} +func (UnimplementedDataCoordServer) ListImports(context.Context, *internalpb.ListImportsRequestInternal) (*internalpb.ListImportsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListImports not implemented") +} + +// UnsafeDataCoordServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to DataCoordServer will +// result in compilation errors. +type UnsafeDataCoordServer interface { + mustEmbedUnimplementedDataCoordServer() +} + +func RegisterDataCoordServer(s grpc.ServiceRegistrar, srv DataCoordServer) { + s.RegisterService(&DataCoord_ServiceDesc, srv) +} + +func _DataCoord_GetComponentStates_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.GetComponentStatesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetComponentStates(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GetComponentStates_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetComponentStates(ctx, req.(*milvuspb.GetComponentStatesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GetTimeTickChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.GetTimeTickChannelRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetTimeTickChannel(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GetTimeTickChannel_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetTimeTickChannel(ctx, req.(*internalpb.GetTimeTickChannelRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GetStatisticsChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.GetStatisticsChannelRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetStatisticsChannel(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GetStatisticsChannel_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetStatisticsChannel(ctx, req.(*internalpb.GetStatisticsChannelRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_Flush_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(FlushRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).Flush(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_Flush_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).Flush(ctx, req.(*FlushRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_AllocSegment_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(AllocSegmentRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).AllocSegment(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_AllocSegment_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).AllocSegment(ctx, req.(*AllocSegmentRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_AssignSegmentID_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(AssignSegmentIDRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).AssignSegmentID(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_AssignSegmentID_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).AssignSegmentID(ctx, req.(*AssignSegmentIDRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GetSegmentInfo_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetSegmentInfoRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetSegmentInfo(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GetSegmentInfo_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetSegmentInfo(ctx, req.(*GetSegmentInfoRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GetSegmentStates_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetSegmentStatesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetSegmentStates(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GetSegmentStates_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetSegmentStates(ctx, req.(*GetSegmentStatesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GetInsertBinlogPaths_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetInsertBinlogPathsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetInsertBinlogPaths(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GetInsertBinlogPaths_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetInsertBinlogPaths(ctx, req.(*GetInsertBinlogPathsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GetCollectionStatistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetCollectionStatisticsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetCollectionStatistics(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GetCollectionStatistics_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetCollectionStatistics(ctx, req.(*GetCollectionStatisticsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GetPartitionStatistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetPartitionStatisticsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetPartitionStatistics(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GetPartitionStatistics_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetPartitionStatistics(ctx, req.(*GetPartitionStatisticsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GetSegmentInfoChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetSegmentInfoChannelRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetSegmentInfoChannel(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GetSegmentInfoChannel_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetSegmentInfoChannel(ctx, req.(*GetSegmentInfoChannelRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_SaveBinlogPaths_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(SaveBinlogPathsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).SaveBinlogPaths(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_SaveBinlogPaths_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).SaveBinlogPaths(ctx, req.(*SaveBinlogPathsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GetRecoveryInfo_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetRecoveryInfoRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetRecoveryInfo(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GetRecoveryInfo_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetRecoveryInfo(ctx, req.(*GetRecoveryInfoRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GetRecoveryInfoV2_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetRecoveryInfoRequestV2) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetRecoveryInfoV2(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GetRecoveryInfoV2_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetRecoveryInfoV2(ctx, req.(*GetRecoveryInfoRequestV2)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GetChannelRecoveryInfo_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetChannelRecoveryInfoRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetChannelRecoveryInfo(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GetChannelRecoveryInfo_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetChannelRecoveryInfo(ctx, req.(*GetChannelRecoveryInfoRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GetFlushedSegments_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetFlushedSegmentsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetFlushedSegments(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GetFlushedSegments_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetFlushedSegments(ctx, req.(*GetFlushedSegmentsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GetSegmentsByStates_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetSegmentsByStatesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetSegmentsByStates(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GetSegmentsByStates_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetSegmentsByStates(ctx, req.(*GetSegmentsByStatesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GetFlushAllState_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.GetFlushAllStateRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetFlushAllState(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GetFlushAllState_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetFlushAllState(ctx, req.(*milvuspb.GetFlushAllStateRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_ShowConfigurations_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.ShowConfigurationsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).ShowConfigurations(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_ShowConfigurations_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).ShowConfigurations(ctx, req.(*internalpb.ShowConfigurationsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GetMetrics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.GetMetricsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetMetrics(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GetMetrics_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetMetrics(ctx, req.(*milvuspb.GetMetricsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_ManualCompaction_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.ManualCompactionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).ManualCompaction(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_ManualCompaction_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).ManualCompaction(ctx, req.(*milvuspb.ManualCompactionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GetCompactionState_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.GetCompactionStateRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetCompactionState(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GetCompactionState_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetCompactionState(ctx, req.(*milvuspb.GetCompactionStateRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GetCompactionStateWithPlans_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.GetCompactionPlansRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetCompactionStateWithPlans(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GetCompactionStateWithPlans_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetCompactionStateWithPlans(ctx, req.(*milvuspb.GetCompactionPlansRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_WatchChannels_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(WatchChannelsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).WatchChannels(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_WatchChannels_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).WatchChannels(ctx, req.(*WatchChannelsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GetFlushState_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetFlushStateRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetFlushState(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GetFlushState_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetFlushState(ctx, req.(*GetFlushStateRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_DropVirtualChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DropVirtualChannelRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).DropVirtualChannel(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_DropVirtualChannel_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).DropVirtualChannel(ctx, req.(*DropVirtualChannelRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_SetSegmentState_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(SetSegmentStateRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).SetSegmentState(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_SetSegmentState_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).SetSegmentState(ctx, req.(*SetSegmentStateRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_UpdateSegmentStatistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(UpdateSegmentStatisticsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).UpdateSegmentStatistics(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_UpdateSegmentStatistics_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).UpdateSegmentStatistics(ctx, req.(*UpdateSegmentStatisticsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_UpdateChannelCheckpoint_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(UpdateChannelCheckpointRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).UpdateChannelCheckpoint(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_UpdateChannelCheckpoint_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).UpdateChannelCheckpoint(ctx, req.(*UpdateChannelCheckpointRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_MarkSegmentsDropped_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(MarkSegmentsDroppedRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).MarkSegmentsDropped(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_MarkSegmentsDropped_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).MarkSegmentsDropped(ctx, req.(*MarkSegmentsDroppedRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_BroadcastAlteredCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(AlterCollectionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).BroadcastAlteredCollection(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_BroadcastAlteredCollection_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).BroadcastAlteredCollection(ctx, req.(*AlterCollectionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_CheckHealth_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.CheckHealthRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).CheckHealth(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_CheckHealth_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).CheckHealth(ctx, req.(*milvuspb.CheckHealthRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_CreateIndex_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(indexpb.CreateIndexRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).CreateIndex(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_CreateIndex_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).CreateIndex(ctx, req.(*indexpb.CreateIndexRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_AlterIndex_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(indexpb.AlterIndexRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).AlterIndex(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_AlterIndex_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).AlterIndex(ctx, req.(*indexpb.AlterIndexRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GetIndexState_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(indexpb.GetIndexStateRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetIndexState(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GetIndexState_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetIndexState(ctx, req.(*indexpb.GetIndexStateRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GetSegmentIndexState_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(indexpb.GetSegmentIndexStateRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetSegmentIndexState(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GetSegmentIndexState_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetSegmentIndexState(ctx, req.(*indexpb.GetSegmentIndexStateRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GetIndexInfos_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(indexpb.GetIndexInfoRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetIndexInfos(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GetIndexInfos_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetIndexInfos(ctx, req.(*indexpb.GetIndexInfoRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_DropIndex_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(indexpb.DropIndexRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).DropIndex(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_DropIndex_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).DropIndex(ctx, req.(*indexpb.DropIndexRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_DescribeIndex_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(indexpb.DescribeIndexRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).DescribeIndex(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_DescribeIndex_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).DescribeIndex(ctx, req.(*indexpb.DescribeIndexRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GetIndexStatistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(indexpb.GetIndexStatisticsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetIndexStatistics(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GetIndexStatistics_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetIndexStatistics(ctx, req.(*indexpb.GetIndexStatisticsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GetIndexBuildProgress_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(indexpb.GetIndexBuildProgressRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetIndexBuildProgress(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GetIndexBuildProgress_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetIndexBuildProgress(ctx, req.(*indexpb.GetIndexBuildProgressRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_ListIndexes_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(indexpb.ListIndexesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).ListIndexes(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_ListIndexes_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).ListIndexes(ctx, req.(*indexpb.ListIndexesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GcConfirm_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GcConfirmRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GcConfirm(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GcConfirm_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GcConfirm(ctx, req.(*GcConfirmRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_ReportDataNodeTtMsgs_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ReportDataNodeTtMsgsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).ReportDataNodeTtMsgs(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_ReportDataNodeTtMsgs_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).ReportDataNodeTtMsgs(ctx, req.(*ReportDataNodeTtMsgsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GcControl_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GcControlRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GcControl(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GcControl_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GcControl(ctx, req.(*GcControlRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_ImportV2_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.ImportRequestInternal) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).ImportV2(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_ImportV2_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).ImportV2(ctx, req.(*internalpb.ImportRequestInternal)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GetImportProgress_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.GetImportProgressRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetImportProgress(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_GetImportProgress_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetImportProgress(ctx, req.(*internalpb.GetImportProgressRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_ListImports_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.ListImportsRequestInternal) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).ListImports(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataCoord_ListImports_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).ListImports(ctx, req.(*internalpb.ListImportsRequestInternal)) + } + return interceptor(ctx, in, info, handler) +} + +// DataCoord_ServiceDesc is the grpc.ServiceDesc for DataCoord service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var DataCoord_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "milvus.proto.data.DataCoord", + HandlerType: (*DataCoordServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "GetComponentStates", + Handler: _DataCoord_GetComponentStates_Handler, + }, + { + MethodName: "GetTimeTickChannel", + Handler: _DataCoord_GetTimeTickChannel_Handler, + }, + { + MethodName: "GetStatisticsChannel", + Handler: _DataCoord_GetStatisticsChannel_Handler, + }, + { + MethodName: "Flush", + Handler: _DataCoord_Flush_Handler, + }, + { + MethodName: "AllocSegment", + Handler: _DataCoord_AllocSegment_Handler, + }, + { + MethodName: "AssignSegmentID", + Handler: _DataCoord_AssignSegmentID_Handler, + }, + { + MethodName: "GetSegmentInfo", + Handler: _DataCoord_GetSegmentInfo_Handler, + }, + { + MethodName: "GetSegmentStates", + Handler: _DataCoord_GetSegmentStates_Handler, + }, + { + MethodName: "GetInsertBinlogPaths", + Handler: _DataCoord_GetInsertBinlogPaths_Handler, + }, + { + MethodName: "GetCollectionStatistics", + Handler: _DataCoord_GetCollectionStatistics_Handler, + }, + { + MethodName: "GetPartitionStatistics", + Handler: _DataCoord_GetPartitionStatistics_Handler, + }, + { + MethodName: "GetSegmentInfoChannel", + Handler: _DataCoord_GetSegmentInfoChannel_Handler, + }, + { + MethodName: "SaveBinlogPaths", + Handler: _DataCoord_SaveBinlogPaths_Handler, + }, + { + MethodName: "GetRecoveryInfo", + Handler: _DataCoord_GetRecoveryInfo_Handler, + }, + { + MethodName: "GetRecoveryInfoV2", + Handler: _DataCoord_GetRecoveryInfoV2_Handler, + }, + { + MethodName: "GetChannelRecoveryInfo", + Handler: _DataCoord_GetChannelRecoveryInfo_Handler, + }, + { + MethodName: "GetFlushedSegments", + Handler: _DataCoord_GetFlushedSegments_Handler, + }, + { + MethodName: "GetSegmentsByStates", + Handler: _DataCoord_GetSegmentsByStates_Handler, + }, + { + MethodName: "GetFlushAllState", + Handler: _DataCoord_GetFlushAllState_Handler, + }, + { + MethodName: "ShowConfigurations", + Handler: _DataCoord_ShowConfigurations_Handler, + }, + { + MethodName: "GetMetrics", + Handler: _DataCoord_GetMetrics_Handler, + }, + { + MethodName: "ManualCompaction", + Handler: _DataCoord_ManualCompaction_Handler, + }, + { + MethodName: "GetCompactionState", + Handler: _DataCoord_GetCompactionState_Handler, + }, + { + MethodName: "GetCompactionStateWithPlans", + Handler: _DataCoord_GetCompactionStateWithPlans_Handler, + }, + { + MethodName: "WatchChannels", + Handler: _DataCoord_WatchChannels_Handler, + }, + { + MethodName: "GetFlushState", + Handler: _DataCoord_GetFlushState_Handler, + }, + { + MethodName: "DropVirtualChannel", + Handler: _DataCoord_DropVirtualChannel_Handler, + }, + { + MethodName: "SetSegmentState", + Handler: _DataCoord_SetSegmentState_Handler, + }, + { + MethodName: "UpdateSegmentStatistics", + Handler: _DataCoord_UpdateSegmentStatistics_Handler, + }, + { + MethodName: "UpdateChannelCheckpoint", + Handler: _DataCoord_UpdateChannelCheckpoint_Handler, + }, + { + MethodName: "MarkSegmentsDropped", + Handler: _DataCoord_MarkSegmentsDropped_Handler, + }, + { + MethodName: "BroadcastAlteredCollection", + Handler: _DataCoord_BroadcastAlteredCollection_Handler, + }, + { + MethodName: "CheckHealth", + Handler: _DataCoord_CheckHealth_Handler, + }, + { + MethodName: "CreateIndex", + Handler: _DataCoord_CreateIndex_Handler, + }, + { + MethodName: "AlterIndex", + Handler: _DataCoord_AlterIndex_Handler, + }, + { + MethodName: "GetIndexState", + Handler: _DataCoord_GetIndexState_Handler, + }, + { + MethodName: "GetSegmentIndexState", + Handler: _DataCoord_GetSegmentIndexState_Handler, + }, + { + MethodName: "GetIndexInfos", + Handler: _DataCoord_GetIndexInfos_Handler, + }, + { + MethodName: "DropIndex", + Handler: _DataCoord_DropIndex_Handler, + }, + { + MethodName: "DescribeIndex", + Handler: _DataCoord_DescribeIndex_Handler, + }, + { + MethodName: "GetIndexStatistics", + Handler: _DataCoord_GetIndexStatistics_Handler, + }, + { + MethodName: "GetIndexBuildProgress", + Handler: _DataCoord_GetIndexBuildProgress_Handler, + }, + { + MethodName: "ListIndexes", + Handler: _DataCoord_ListIndexes_Handler, + }, + { + MethodName: "GcConfirm", + Handler: _DataCoord_GcConfirm_Handler, + }, + { + MethodName: "ReportDataNodeTtMsgs", + Handler: _DataCoord_ReportDataNodeTtMsgs_Handler, + }, + { + MethodName: "GcControl", + Handler: _DataCoord_GcControl_Handler, + }, + { + MethodName: "ImportV2", + Handler: _DataCoord_ImportV2_Handler, + }, + { + MethodName: "GetImportProgress", + Handler: _DataCoord_GetImportProgress_Handler, + }, + { + MethodName: "ListImports", + Handler: _DataCoord_ListImports_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "data_coord.proto", +} + +const ( + DataNode_GetComponentStates_FullMethodName = "/milvus.proto.data.DataNode/GetComponentStates" + DataNode_GetStatisticsChannel_FullMethodName = "/milvus.proto.data.DataNode/GetStatisticsChannel" + DataNode_WatchDmChannels_FullMethodName = "/milvus.proto.data.DataNode/WatchDmChannels" + DataNode_FlushSegments_FullMethodName = "/milvus.proto.data.DataNode/FlushSegments" + DataNode_ShowConfigurations_FullMethodName = "/milvus.proto.data.DataNode/ShowConfigurations" + DataNode_GetMetrics_FullMethodName = "/milvus.proto.data.DataNode/GetMetrics" + DataNode_CompactionV2_FullMethodName = "/milvus.proto.data.DataNode/CompactionV2" + DataNode_GetCompactionState_FullMethodName = "/milvus.proto.data.DataNode/GetCompactionState" + DataNode_SyncSegments_FullMethodName = "/milvus.proto.data.DataNode/SyncSegments" + DataNode_ResendSegmentStats_FullMethodName = "/milvus.proto.data.DataNode/ResendSegmentStats" + DataNode_FlushChannels_FullMethodName = "/milvus.proto.data.DataNode/FlushChannels" + DataNode_NotifyChannelOperation_FullMethodName = "/milvus.proto.data.DataNode/NotifyChannelOperation" + DataNode_CheckChannelOperationProgress_FullMethodName = "/milvus.proto.data.DataNode/CheckChannelOperationProgress" + DataNode_PreImport_FullMethodName = "/milvus.proto.data.DataNode/PreImport" + DataNode_ImportV2_FullMethodName = "/milvus.proto.data.DataNode/ImportV2" + DataNode_QueryPreImport_FullMethodName = "/milvus.proto.data.DataNode/QueryPreImport" + DataNode_QueryImport_FullMethodName = "/milvus.proto.data.DataNode/QueryImport" + DataNode_DropImport_FullMethodName = "/milvus.proto.data.DataNode/DropImport" + DataNode_QuerySlot_FullMethodName = "/milvus.proto.data.DataNode/QuerySlot" + DataNode_DropCompactionPlan_FullMethodName = "/milvus.proto.data.DataNode/DropCompactionPlan" +) + +// DataNodeClient is the client API for DataNode service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +type DataNodeClient interface { + GetComponentStates(ctx context.Context, in *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) + GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) + WatchDmChannels(ctx context.Context, in *WatchDmChannelsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + FlushSegments(ctx context.Context, in *FlushSegmentsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + ShowConfigurations(ctx context.Context, in *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error) + // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy + GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) + CompactionV2(ctx context.Context, in *CompactionPlan, opts ...grpc.CallOption) (*commonpb.Status, error) + GetCompactionState(ctx context.Context, in *CompactionStateRequest, opts ...grpc.CallOption) (*CompactionStateResponse, error) + SyncSegments(ctx context.Context, in *SyncSegmentsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + // Deprecated + ResendSegmentStats(ctx context.Context, in *ResendSegmentStatsRequest, opts ...grpc.CallOption) (*ResendSegmentStatsResponse, error) + FlushChannels(ctx context.Context, in *FlushChannelsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + NotifyChannelOperation(ctx context.Context, in *ChannelOperationsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + CheckChannelOperationProgress(ctx context.Context, in *ChannelWatchInfo, opts ...grpc.CallOption) (*ChannelOperationProgressResponse, error) + // import v2 + PreImport(ctx context.Context, in *PreImportRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + ImportV2(ctx context.Context, in *ImportRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + QueryPreImport(ctx context.Context, in *QueryPreImportRequest, opts ...grpc.CallOption) (*QueryPreImportResponse, error) + QueryImport(ctx context.Context, in *QueryImportRequest, opts ...grpc.CallOption) (*QueryImportResponse, error) + DropImport(ctx context.Context, in *DropImportRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + QuerySlot(ctx context.Context, in *QuerySlotRequest, opts ...grpc.CallOption) (*QuerySlotResponse, error) + DropCompactionPlan(ctx context.Context, in *DropCompactionPlanRequest, opts ...grpc.CallOption) (*commonpb.Status, error) +} + +type dataNodeClient struct { + cc grpc.ClientConnInterface +} + +func NewDataNodeClient(cc grpc.ClientConnInterface) DataNodeClient { + return &dataNodeClient{cc} +} + +func (c *dataNodeClient) GetComponentStates(ctx context.Context, in *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) { + out := new(milvuspb.ComponentStates) + err := c.cc.Invoke(ctx, DataNode_GetComponentStates_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataNodeClient) GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) { + out := new(milvuspb.StringResponse) + err := c.cc.Invoke(ctx, DataNode_GetStatisticsChannel_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataNodeClient) WatchDmChannels(ctx context.Context, in *WatchDmChannelsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, DataNode_WatchDmChannels_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataNodeClient) FlushSegments(ctx context.Context, in *FlushSegmentsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, DataNode_FlushSegments_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataNodeClient) ShowConfigurations(ctx context.Context, in *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error) { + out := new(internalpb.ShowConfigurationsResponse) + err := c.cc.Invoke(ctx, DataNode_ShowConfigurations_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataNodeClient) GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) { + out := new(milvuspb.GetMetricsResponse) + err := c.cc.Invoke(ctx, DataNode_GetMetrics_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataNodeClient) CompactionV2(ctx context.Context, in *CompactionPlan, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, DataNode_CompactionV2_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataNodeClient) GetCompactionState(ctx context.Context, in *CompactionStateRequest, opts ...grpc.CallOption) (*CompactionStateResponse, error) { + out := new(CompactionStateResponse) + err := c.cc.Invoke(ctx, DataNode_GetCompactionState_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataNodeClient) SyncSegments(ctx context.Context, in *SyncSegmentsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, DataNode_SyncSegments_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataNodeClient) ResendSegmentStats(ctx context.Context, in *ResendSegmentStatsRequest, opts ...grpc.CallOption) (*ResendSegmentStatsResponse, error) { + out := new(ResendSegmentStatsResponse) + err := c.cc.Invoke(ctx, DataNode_ResendSegmentStats_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataNodeClient) FlushChannels(ctx context.Context, in *FlushChannelsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, DataNode_FlushChannels_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataNodeClient) NotifyChannelOperation(ctx context.Context, in *ChannelOperationsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, DataNode_NotifyChannelOperation_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataNodeClient) CheckChannelOperationProgress(ctx context.Context, in *ChannelWatchInfo, opts ...grpc.CallOption) (*ChannelOperationProgressResponse, error) { + out := new(ChannelOperationProgressResponse) + err := c.cc.Invoke(ctx, DataNode_CheckChannelOperationProgress_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataNodeClient) PreImport(ctx context.Context, in *PreImportRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, DataNode_PreImport_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataNodeClient) ImportV2(ctx context.Context, in *ImportRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, DataNode_ImportV2_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataNodeClient) QueryPreImport(ctx context.Context, in *QueryPreImportRequest, opts ...grpc.CallOption) (*QueryPreImportResponse, error) { + out := new(QueryPreImportResponse) + err := c.cc.Invoke(ctx, DataNode_QueryPreImport_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataNodeClient) QueryImport(ctx context.Context, in *QueryImportRequest, opts ...grpc.CallOption) (*QueryImportResponse, error) { + out := new(QueryImportResponse) + err := c.cc.Invoke(ctx, DataNode_QueryImport_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataNodeClient) DropImport(ctx context.Context, in *DropImportRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, DataNode_DropImport_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataNodeClient) QuerySlot(ctx context.Context, in *QuerySlotRequest, opts ...grpc.CallOption) (*QuerySlotResponse, error) { + out := new(QuerySlotResponse) + err := c.cc.Invoke(ctx, DataNode_QuerySlot_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataNodeClient) DropCompactionPlan(ctx context.Context, in *DropCompactionPlanRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, DataNode_DropCompactionPlan_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// DataNodeServer is the server API for DataNode service. +// All implementations should embed UnimplementedDataNodeServer +// for forward compatibility +type DataNodeServer interface { + GetComponentStates(context.Context, *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) + GetStatisticsChannel(context.Context, *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) + WatchDmChannels(context.Context, *WatchDmChannelsRequest) (*commonpb.Status, error) + FlushSegments(context.Context, *FlushSegmentsRequest) (*commonpb.Status, error) + ShowConfigurations(context.Context, *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) + // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy + GetMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) + CompactionV2(context.Context, *CompactionPlan) (*commonpb.Status, error) + GetCompactionState(context.Context, *CompactionStateRequest) (*CompactionStateResponse, error) + SyncSegments(context.Context, *SyncSegmentsRequest) (*commonpb.Status, error) + // Deprecated + ResendSegmentStats(context.Context, *ResendSegmentStatsRequest) (*ResendSegmentStatsResponse, error) + FlushChannels(context.Context, *FlushChannelsRequest) (*commonpb.Status, error) + NotifyChannelOperation(context.Context, *ChannelOperationsRequest) (*commonpb.Status, error) + CheckChannelOperationProgress(context.Context, *ChannelWatchInfo) (*ChannelOperationProgressResponse, error) + // import v2 + PreImport(context.Context, *PreImportRequest) (*commonpb.Status, error) + ImportV2(context.Context, *ImportRequest) (*commonpb.Status, error) + QueryPreImport(context.Context, *QueryPreImportRequest) (*QueryPreImportResponse, error) + QueryImport(context.Context, *QueryImportRequest) (*QueryImportResponse, error) + DropImport(context.Context, *DropImportRequest) (*commonpb.Status, error) + QuerySlot(context.Context, *QuerySlotRequest) (*QuerySlotResponse, error) + DropCompactionPlan(context.Context, *DropCompactionPlanRequest) (*commonpb.Status, error) +} + +// UnimplementedDataNodeServer should be embedded to have forward compatible implementations. +type UnimplementedDataNodeServer struct { +} + +func (UnimplementedDataNodeServer) GetComponentStates(context.Context, *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetComponentStates not implemented") +} +func (UnimplementedDataNodeServer) GetStatisticsChannel(context.Context, *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetStatisticsChannel not implemented") +} +func (UnimplementedDataNodeServer) WatchDmChannels(context.Context, *WatchDmChannelsRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method WatchDmChannels not implemented") +} +func (UnimplementedDataNodeServer) FlushSegments(context.Context, *FlushSegmentsRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method FlushSegments not implemented") +} +func (UnimplementedDataNodeServer) ShowConfigurations(context.Context, *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ShowConfigurations not implemented") +} +func (UnimplementedDataNodeServer) GetMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetMetrics not implemented") +} +func (UnimplementedDataNodeServer) CompactionV2(context.Context, *CompactionPlan) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method CompactionV2 not implemented") +} +func (UnimplementedDataNodeServer) GetCompactionState(context.Context, *CompactionStateRequest) (*CompactionStateResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetCompactionState not implemented") +} +func (UnimplementedDataNodeServer) SyncSegments(context.Context, *SyncSegmentsRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method SyncSegments not implemented") +} +func (UnimplementedDataNodeServer) ResendSegmentStats(context.Context, *ResendSegmentStatsRequest) (*ResendSegmentStatsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResendSegmentStats not implemented") +} +func (UnimplementedDataNodeServer) FlushChannels(context.Context, *FlushChannelsRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method FlushChannels not implemented") +} +func (UnimplementedDataNodeServer) NotifyChannelOperation(context.Context, *ChannelOperationsRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method NotifyChannelOperation not implemented") +} +func (UnimplementedDataNodeServer) CheckChannelOperationProgress(context.Context, *ChannelWatchInfo) (*ChannelOperationProgressResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method CheckChannelOperationProgress not implemented") +} +func (UnimplementedDataNodeServer) PreImport(context.Context, *PreImportRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method PreImport not implemented") +} +func (UnimplementedDataNodeServer) ImportV2(context.Context, *ImportRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method ImportV2 not implemented") +} +func (UnimplementedDataNodeServer) QueryPreImport(context.Context, *QueryPreImportRequest) (*QueryPreImportResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryPreImport not implemented") +} +func (UnimplementedDataNodeServer) QueryImport(context.Context, *QueryImportRequest) (*QueryImportResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryImport not implemented") +} +func (UnimplementedDataNodeServer) DropImport(context.Context, *DropImportRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method DropImport not implemented") +} +func (UnimplementedDataNodeServer) QuerySlot(context.Context, *QuerySlotRequest) (*QuerySlotResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QuerySlot not implemented") +} +func (UnimplementedDataNodeServer) DropCompactionPlan(context.Context, *DropCompactionPlanRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method DropCompactionPlan not implemented") +} + +// UnsafeDataNodeServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to DataNodeServer will +// result in compilation errors. +type UnsafeDataNodeServer interface { + mustEmbedUnimplementedDataNodeServer() +} + +func RegisterDataNodeServer(s grpc.ServiceRegistrar, srv DataNodeServer) { + s.RegisterService(&DataNode_ServiceDesc, srv) +} + +func _DataNode_GetComponentStates_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.GetComponentStatesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataNodeServer).GetComponentStates(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataNode_GetComponentStates_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataNodeServer).GetComponentStates(ctx, req.(*milvuspb.GetComponentStatesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataNode_GetStatisticsChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.GetStatisticsChannelRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataNodeServer).GetStatisticsChannel(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataNode_GetStatisticsChannel_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataNodeServer).GetStatisticsChannel(ctx, req.(*internalpb.GetStatisticsChannelRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataNode_WatchDmChannels_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(WatchDmChannelsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataNodeServer).WatchDmChannels(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataNode_WatchDmChannels_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataNodeServer).WatchDmChannels(ctx, req.(*WatchDmChannelsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataNode_FlushSegments_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(FlushSegmentsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataNodeServer).FlushSegments(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataNode_FlushSegments_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataNodeServer).FlushSegments(ctx, req.(*FlushSegmentsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataNode_ShowConfigurations_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.ShowConfigurationsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataNodeServer).ShowConfigurations(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataNode_ShowConfigurations_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataNodeServer).ShowConfigurations(ctx, req.(*internalpb.ShowConfigurationsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataNode_GetMetrics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.GetMetricsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataNodeServer).GetMetrics(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataNode_GetMetrics_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataNodeServer).GetMetrics(ctx, req.(*milvuspb.GetMetricsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataNode_CompactionV2_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CompactionPlan) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataNodeServer).CompactionV2(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataNode_CompactionV2_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataNodeServer).CompactionV2(ctx, req.(*CompactionPlan)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataNode_GetCompactionState_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CompactionStateRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataNodeServer).GetCompactionState(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataNode_GetCompactionState_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataNodeServer).GetCompactionState(ctx, req.(*CompactionStateRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataNode_SyncSegments_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(SyncSegmentsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataNodeServer).SyncSegments(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataNode_SyncSegments_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataNodeServer).SyncSegments(ctx, req.(*SyncSegmentsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataNode_ResendSegmentStats_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResendSegmentStatsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataNodeServer).ResendSegmentStats(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataNode_ResendSegmentStats_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataNodeServer).ResendSegmentStats(ctx, req.(*ResendSegmentStatsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataNode_FlushChannels_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(FlushChannelsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataNodeServer).FlushChannels(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataNode_FlushChannels_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataNodeServer).FlushChannels(ctx, req.(*FlushChannelsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataNode_NotifyChannelOperation_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ChannelOperationsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataNodeServer).NotifyChannelOperation(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataNode_NotifyChannelOperation_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataNodeServer).NotifyChannelOperation(ctx, req.(*ChannelOperationsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataNode_CheckChannelOperationProgress_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ChannelWatchInfo) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataNodeServer).CheckChannelOperationProgress(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataNode_CheckChannelOperationProgress_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataNodeServer).CheckChannelOperationProgress(ctx, req.(*ChannelWatchInfo)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataNode_PreImport_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(PreImportRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataNodeServer).PreImport(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataNode_PreImport_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataNodeServer).PreImport(ctx, req.(*PreImportRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataNode_ImportV2_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ImportRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataNodeServer).ImportV2(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataNode_ImportV2_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataNodeServer).ImportV2(ctx, req.(*ImportRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataNode_QueryPreImport_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryPreImportRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataNodeServer).QueryPreImport(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataNode_QueryPreImport_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataNodeServer).QueryPreImport(ctx, req.(*QueryPreImportRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataNode_QueryImport_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryImportRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataNodeServer).QueryImport(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataNode_QueryImport_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataNodeServer).QueryImport(ctx, req.(*QueryImportRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataNode_DropImport_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DropImportRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataNodeServer).DropImport(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataNode_DropImport_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataNodeServer).DropImport(ctx, req.(*DropImportRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataNode_QuerySlot_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QuerySlotRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataNodeServer).QuerySlot(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataNode_QuerySlot_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataNodeServer).QuerySlot(ctx, req.(*QuerySlotRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataNode_DropCompactionPlan_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DropCompactionPlanRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataNodeServer).DropCompactionPlan(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataNode_DropCompactionPlan_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataNodeServer).DropCompactionPlan(ctx, req.(*DropCompactionPlanRequest)) + } + return interceptor(ctx, in, info, handler) +} + +// DataNode_ServiceDesc is the grpc.ServiceDesc for DataNode service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var DataNode_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "milvus.proto.data.DataNode", + HandlerType: (*DataNodeServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "GetComponentStates", + Handler: _DataNode_GetComponentStates_Handler, + }, + { + MethodName: "GetStatisticsChannel", + Handler: _DataNode_GetStatisticsChannel_Handler, + }, + { + MethodName: "WatchDmChannels", + Handler: _DataNode_WatchDmChannels_Handler, + }, + { + MethodName: "FlushSegments", + Handler: _DataNode_FlushSegments_Handler, + }, + { + MethodName: "ShowConfigurations", + Handler: _DataNode_ShowConfigurations_Handler, + }, + { + MethodName: "GetMetrics", + Handler: _DataNode_GetMetrics_Handler, + }, + { + MethodName: "CompactionV2", + Handler: _DataNode_CompactionV2_Handler, + }, + { + MethodName: "GetCompactionState", + Handler: _DataNode_GetCompactionState_Handler, + }, + { + MethodName: "SyncSegments", + Handler: _DataNode_SyncSegments_Handler, + }, + { + MethodName: "ResendSegmentStats", + Handler: _DataNode_ResendSegmentStats_Handler, + }, + { + MethodName: "FlushChannels", + Handler: _DataNode_FlushChannels_Handler, + }, + { + MethodName: "NotifyChannelOperation", + Handler: _DataNode_NotifyChannelOperation_Handler, + }, + { + MethodName: "CheckChannelOperationProgress", + Handler: _DataNode_CheckChannelOperationProgress_Handler, + }, + { + MethodName: "PreImport", + Handler: _DataNode_PreImport_Handler, + }, + { + MethodName: "ImportV2", + Handler: _DataNode_ImportV2_Handler, + }, + { + MethodName: "QueryPreImport", + Handler: _DataNode_QueryPreImport_Handler, + }, + { + MethodName: "QueryImport", + Handler: _DataNode_QueryImport_Handler, + }, + { + MethodName: "DropImport", + Handler: _DataNode_DropImport_Handler, + }, + { + MethodName: "QuerySlot", + Handler: _DataNode_QuerySlot_Handler, + }, + { + MethodName: "DropCompactionPlan", + Handler: _DataNode_DropCompactionPlan_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "data_coord.proto", +} diff --git a/internal/proto/etcd_meta.proto b/pkg/proto/etcd_meta.proto similarity index 97% rename from internal/proto/etcd_meta.proto rename to pkg/proto/etcd_meta.proto index 40f3fa5e4aec6..6f6495835443d 100644 --- a/internal/proto/etcd_meta.proto +++ b/pkg/proto/etcd_meta.proto @@ -1,6 +1,6 @@ syntax = "proto3"; package milvus.proto.etcd; -option go_package="github.com/milvus-io/milvus/internal/proto/etcdpb"; +option go_package="github.com/milvus-io/milvus/pkg/proto/etcdpb"; import "common.proto"; import "schema.proto"; diff --git a/pkg/proto/etcdpb/etcd_meta.pb.go b/pkg/proto/etcdpb/etcd_meta.pb.go new file mode 100644 index 0000000000000..43b9396bbe484 --- /dev/null +++ b/pkg/proto/etcdpb/etcd_meta.pb.go @@ -0,0 +1,1429 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.33.0 +// protoc v3.21.4 +// source: etcd_meta.proto + +package etcdpb + +import ( + commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + 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" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type DatabaseState int32 + +const ( + DatabaseState_DatabaseUnknown DatabaseState = 0 + DatabaseState_DatabaseCreated DatabaseState = 1 + DatabaseState_DatabaseCreating DatabaseState = 2 + DatabaseState_DatabaseDropping DatabaseState = 3 + DatabaseState_DatabaseDropped DatabaseState = 4 +) + +// Enum value maps for DatabaseState. +var ( + DatabaseState_name = map[int32]string{ + 0: "DatabaseUnknown", + 1: "DatabaseCreated", + 2: "DatabaseCreating", + 3: "DatabaseDropping", + 4: "DatabaseDropped", + } + DatabaseState_value = map[string]int32{ + "DatabaseUnknown": 0, + "DatabaseCreated": 1, + "DatabaseCreating": 2, + "DatabaseDropping": 3, + "DatabaseDropped": 4, + } +) + +func (x DatabaseState) Enum() *DatabaseState { + p := new(DatabaseState) + *p = x + return p +} + +func (x DatabaseState) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (DatabaseState) Descriptor() protoreflect.EnumDescriptor { + return file_etcd_meta_proto_enumTypes[0].Descriptor() +} + +func (DatabaseState) Type() protoreflect.EnumType { + return &file_etcd_meta_proto_enumTypes[0] +} + +func (x DatabaseState) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use DatabaseState.Descriptor instead. +func (DatabaseState) EnumDescriptor() ([]byte, []int) { + return file_etcd_meta_proto_rawDescGZIP(), []int{0} +} + +type CollectionState int32 + +const ( + CollectionState_CollectionCreated CollectionState = 0 + CollectionState_CollectionCreating CollectionState = 1 + CollectionState_CollectionDropping CollectionState = 2 + CollectionState_CollectionDropped CollectionState = 3 +) + +// Enum value maps for CollectionState. +var ( + CollectionState_name = map[int32]string{ + 0: "CollectionCreated", + 1: "CollectionCreating", + 2: "CollectionDropping", + 3: "CollectionDropped", + } + CollectionState_value = map[string]int32{ + "CollectionCreated": 0, + "CollectionCreating": 1, + "CollectionDropping": 2, + "CollectionDropped": 3, + } +) + +func (x CollectionState) Enum() *CollectionState { + p := new(CollectionState) + *p = x + return p +} + +func (x CollectionState) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (CollectionState) Descriptor() protoreflect.EnumDescriptor { + return file_etcd_meta_proto_enumTypes[1].Descriptor() +} + +func (CollectionState) Type() protoreflect.EnumType { + return &file_etcd_meta_proto_enumTypes[1] +} + +func (x CollectionState) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use CollectionState.Descriptor instead. +func (CollectionState) EnumDescriptor() ([]byte, []int) { + return file_etcd_meta_proto_rawDescGZIP(), []int{1} +} + +type PartitionState int32 + +const ( + PartitionState_PartitionCreated PartitionState = 0 + PartitionState_PartitionCreating PartitionState = 1 + PartitionState_PartitionDropping PartitionState = 2 + PartitionState_PartitionDropped PartitionState = 3 +) + +// Enum value maps for PartitionState. +var ( + PartitionState_name = map[int32]string{ + 0: "PartitionCreated", + 1: "PartitionCreating", + 2: "PartitionDropping", + 3: "PartitionDropped", + } + PartitionState_value = map[string]int32{ + "PartitionCreated": 0, + "PartitionCreating": 1, + "PartitionDropping": 2, + "PartitionDropped": 3, + } +) + +func (x PartitionState) Enum() *PartitionState { + p := new(PartitionState) + *p = x + return p +} + +func (x PartitionState) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (PartitionState) Descriptor() protoreflect.EnumDescriptor { + return file_etcd_meta_proto_enumTypes[2].Descriptor() +} + +func (PartitionState) Type() protoreflect.EnumType { + return &file_etcd_meta_proto_enumTypes[2] +} + +func (x PartitionState) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use PartitionState.Descriptor instead. +func (PartitionState) EnumDescriptor() ([]byte, []int) { + return file_etcd_meta_proto_rawDescGZIP(), []int{2} +} + +type AliasState int32 + +const ( + AliasState_AliasCreated AliasState = 0 + AliasState_AliasCreating AliasState = 1 + AliasState_AliasDropping AliasState = 2 + AliasState_AliasDropped AliasState = 3 +) + +// Enum value maps for AliasState. +var ( + AliasState_name = map[int32]string{ + 0: "AliasCreated", + 1: "AliasCreating", + 2: "AliasDropping", + 3: "AliasDropped", + } + AliasState_value = map[string]int32{ + "AliasCreated": 0, + "AliasCreating": 1, + "AliasDropping": 2, + "AliasDropped": 3, + } +) + +func (x AliasState) Enum() *AliasState { + p := new(AliasState) + *p = x + return p +} + +func (x AliasState) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (AliasState) Descriptor() protoreflect.EnumDescriptor { + return file_etcd_meta_proto_enumTypes[3].Descriptor() +} + +func (AliasState) Type() protoreflect.EnumType { + return &file_etcd_meta_proto_enumTypes[3] +} + +func (x AliasState) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use AliasState.Descriptor instead. +func (AliasState) EnumDescriptor() ([]byte, []int) { + return file_etcd_meta_proto_rawDescGZIP(), []int{3} +} + +type IndexInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + IndexName string `protobuf:"bytes,1,opt,name=index_name,json=indexName,proto3" json:"index_name,omitempty"` + IndexID int64 `protobuf:"varint,2,opt,name=indexID,proto3" json:"indexID,omitempty"` + IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,3,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"` + Deleted bool `protobuf:"varint,4,opt,name=deleted,proto3" json:"deleted,omitempty"` + CreateTime uint64 `protobuf:"varint,5,opt,name=create_time,json=createTime,proto3" json:"create_time,omitempty"` +} + +func (x *IndexInfo) Reset() { + *x = IndexInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_etcd_meta_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *IndexInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*IndexInfo) ProtoMessage() {} + +func (x *IndexInfo) ProtoReflect() protoreflect.Message { + mi := &file_etcd_meta_proto_msgTypes[0] + 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 IndexInfo.ProtoReflect.Descriptor instead. +func (*IndexInfo) Descriptor() ([]byte, []int) { + return file_etcd_meta_proto_rawDescGZIP(), []int{0} +} + +func (x *IndexInfo) GetIndexName() string { + if x != nil { + return x.IndexName + } + return "" +} + +func (x *IndexInfo) GetIndexID() int64 { + if x != nil { + return x.IndexID + } + return 0 +} + +func (x *IndexInfo) GetIndexParams() []*commonpb.KeyValuePair { + if x != nil { + return x.IndexParams + } + return nil +} + +func (x *IndexInfo) GetDeleted() bool { + if x != nil { + return x.Deleted + } + return false +} + +func (x *IndexInfo) GetCreateTime() uint64 { + if x != nil { + return x.CreateTime + } + return 0 +} + +type FieldIndexInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + FiledID int64 `protobuf:"varint,1,opt,name=filedID,proto3" json:"filedID,omitempty"` + IndexID int64 `protobuf:"varint,2,opt,name=indexID,proto3" json:"indexID,omitempty"` +} + +func (x *FieldIndexInfo) Reset() { + *x = FieldIndexInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_etcd_meta_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *FieldIndexInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FieldIndexInfo) ProtoMessage() {} + +func (x *FieldIndexInfo) ProtoReflect() protoreflect.Message { + mi := &file_etcd_meta_proto_msgTypes[1] + 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 FieldIndexInfo.ProtoReflect.Descriptor instead. +func (*FieldIndexInfo) Descriptor() ([]byte, []int) { + return file_etcd_meta_proto_rawDescGZIP(), []int{1} +} + +func (x *FieldIndexInfo) GetFiledID() int64 { + if x != nil { + return x.FiledID + } + return 0 +} + +func (x *FieldIndexInfo) GetIndexID() int64 { + if x != nil { + return x.IndexID + } + return 0 +} + +type CollectionInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ID int64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"` + Schema *schemapb.CollectionSchema `protobuf:"bytes,2,opt,name=schema,proto3" json:"schema,omitempty"` + CreateTime uint64 `protobuf:"varint,3,opt,name=create_time,json=createTime,proto3" json:"create_time,omitempty"` + // deprecate + PartitionIDs []int64 `protobuf:"varint,4,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` + // deprecate + PartitionNames []string `protobuf:"bytes,5,rep,name=partitionNames,proto3" json:"partitionNames,omitempty"` + // deprecate + FieldIndexes []*FieldIndexInfo `protobuf:"bytes,6,rep,name=field_indexes,json=fieldIndexes,proto3" json:"field_indexes,omitempty"` + VirtualChannelNames []string `protobuf:"bytes,7,rep,name=virtual_channel_names,json=virtualChannelNames,proto3" json:"virtual_channel_names,omitempty"` + PhysicalChannelNames []string `protobuf:"bytes,8,rep,name=physical_channel_names,json=physicalChannelNames,proto3" json:"physical_channel_names,omitempty"` + // deprecate + PartitionCreatedTimestamps []uint64 `protobuf:"varint,9,rep,packed,name=partition_created_timestamps,json=partitionCreatedTimestamps,proto3" json:"partition_created_timestamps,omitempty"` + ShardsNum int32 `protobuf:"varint,10,opt,name=shards_num,json=shardsNum,proto3" json:"shards_num,omitempty"` + StartPositions []*commonpb.KeyDataPair `protobuf:"bytes,11,rep,name=start_positions,json=startPositions,proto3" json:"start_positions,omitempty"` + ConsistencyLevel commonpb.ConsistencyLevel `protobuf:"varint,12,opt,name=consistency_level,json=consistencyLevel,proto3,enum=milvus.proto.common.ConsistencyLevel" json:"consistency_level,omitempty"` + State CollectionState `protobuf:"varint,13,opt,name=state,proto3,enum=milvus.proto.etcd.CollectionState" json:"state,omitempty"` // To keep compatible with older version, default state is `Created`. + Properties []*commonpb.KeyValuePair `protobuf:"bytes,14,rep,name=properties,proto3" json:"properties,omitempty"` + DbId int64 `protobuf:"varint,15,opt,name=db_id,json=dbId,proto3" json:"db_id,omitempty"` +} + +func (x *CollectionInfo) Reset() { + *x = CollectionInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_etcd_meta_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CollectionInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CollectionInfo) ProtoMessage() {} + +func (x *CollectionInfo) ProtoReflect() protoreflect.Message { + mi := &file_etcd_meta_proto_msgTypes[2] + 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 CollectionInfo.ProtoReflect.Descriptor instead. +func (*CollectionInfo) Descriptor() ([]byte, []int) { + return file_etcd_meta_proto_rawDescGZIP(), []int{2} +} + +func (x *CollectionInfo) GetID() int64 { + if x != nil { + return x.ID + } + return 0 +} + +func (x *CollectionInfo) GetSchema() *schemapb.CollectionSchema { + if x != nil { + return x.Schema + } + return nil +} + +func (x *CollectionInfo) GetCreateTime() uint64 { + if x != nil { + return x.CreateTime + } + return 0 +} + +func (x *CollectionInfo) GetPartitionIDs() []int64 { + if x != nil { + return x.PartitionIDs + } + return nil +} + +func (x *CollectionInfo) GetPartitionNames() []string { + if x != nil { + return x.PartitionNames + } + return nil +} + +func (x *CollectionInfo) GetFieldIndexes() []*FieldIndexInfo { + if x != nil { + return x.FieldIndexes + } + return nil +} + +func (x *CollectionInfo) GetVirtualChannelNames() []string { + if x != nil { + return x.VirtualChannelNames + } + return nil +} + +func (x *CollectionInfo) GetPhysicalChannelNames() []string { + if x != nil { + return x.PhysicalChannelNames + } + return nil +} + +func (x *CollectionInfo) GetPartitionCreatedTimestamps() []uint64 { + if x != nil { + return x.PartitionCreatedTimestamps + } + return nil +} + +func (x *CollectionInfo) GetShardsNum() int32 { + if x != nil { + return x.ShardsNum + } + return 0 +} + +func (x *CollectionInfo) GetStartPositions() []*commonpb.KeyDataPair { + if x != nil { + return x.StartPositions + } + return nil +} + +func (x *CollectionInfo) GetConsistencyLevel() commonpb.ConsistencyLevel { + if x != nil { + return x.ConsistencyLevel + } + return commonpb.ConsistencyLevel(0) +} + +func (x *CollectionInfo) GetState() CollectionState { + if x != nil { + return x.State + } + return CollectionState_CollectionCreated +} + +func (x *CollectionInfo) GetProperties() []*commonpb.KeyValuePair { + if x != nil { + return x.Properties + } + return nil +} + +func (x *CollectionInfo) GetDbId() int64 { + if x != nil { + return x.DbId + } + return 0 +} + +type PartitionInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + PartitionID int64 `protobuf:"varint,1,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + PartitionName string `protobuf:"bytes,2,opt,name=partitionName,proto3" json:"partitionName,omitempty"` + PartitionCreatedTimestamp uint64 `protobuf:"varint,3,opt,name=partition_created_timestamp,json=partitionCreatedTimestamp,proto3" json:"partition_created_timestamp,omitempty"` + CollectionId int64 `protobuf:"varint,4,opt,name=collection_id,json=collectionId,proto3" json:"collection_id,omitempty"` + State PartitionState `protobuf:"varint,5,opt,name=state,proto3,enum=milvus.proto.etcd.PartitionState" json:"state,omitempty"` // To keep compatible with older version, default state is `Created`. +} + +func (x *PartitionInfo) Reset() { + *x = PartitionInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_etcd_meta_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PartitionInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PartitionInfo) ProtoMessage() {} + +func (x *PartitionInfo) ProtoReflect() protoreflect.Message { + mi := &file_etcd_meta_proto_msgTypes[3] + 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 PartitionInfo.ProtoReflect.Descriptor instead. +func (*PartitionInfo) Descriptor() ([]byte, []int) { + return file_etcd_meta_proto_rawDescGZIP(), []int{3} +} + +func (x *PartitionInfo) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *PartitionInfo) GetPartitionName() string { + if x != nil { + return x.PartitionName + } + return "" +} + +func (x *PartitionInfo) GetPartitionCreatedTimestamp() uint64 { + if x != nil { + return x.PartitionCreatedTimestamp + } + return 0 +} + +func (x *PartitionInfo) GetCollectionId() int64 { + if x != nil { + return x.CollectionId + } + return 0 +} + +func (x *PartitionInfo) GetState() PartitionState { + if x != nil { + return x.State + } + return PartitionState_PartitionCreated +} + +type AliasInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + AliasName string `protobuf:"bytes,1,opt,name=alias_name,json=aliasName,proto3" json:"alias_name,omitempty"` + CollectionId int64 `protobuf:"varint,2,opt,name=collection_id,json=collectionId,proto3" json:"collection_id,omitempty"` + CreatedTime uint64 `protobuf:"varint,3,opt,name=created_time,json=createdTime,proto3" json:"created_time,omitempty"` + State AliasState `protobuf:"varint,4,opt,name=state,proto3,enum=milvus.proto.etcd.AliasState" json:"state,omitempty"` // To keep compatible with older version, default state is `Created`. + DbId int64 `protobuf:"varint,5,opt,name=db_id,json=dbId,proto3" json:"db_id,omitempty"` +} + +func (x *AliasInfo) Reset() { + *x = AliasInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_etcd_meta_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AliasInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AliasInfo) ProtoMessage() {} + +func (x *AliasInfo) ProtoReflect() protoreflect.Message { + mi := &file_etcd_meta_proto_msgTypes[4] + 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 AliasInfo.ProtoReflect.Descriptor instead. +func (*AliasInfo) Descriptor() ([]byte, []int) { + return file_etcd_meta_proto_rawDescGZIP(), []int{4} +} + +func (x *AliasInfo) GetAliasName() string { + if x != nil { + return x.AliasName + } + return "" +} + +func (x *AliasInfo) GetCollectionId() int64 { + if x != nil { + return x.CollectionId + } + return 0 +} + +func (x *AliasInfo) GetCreatedTime() uint64 { + if x != nil { + return x.CreatedTime + } + return 0 +} + +func (x *AliasInfo) GetState() AliasState { + if x != nil { + return x.State + } + return AliasState_AliasCreated +} + +func (x *AliasInfo) GetDbId() int64 { + if x != nil { + return x.DbId + } + return 0 +} + +type DatabaseInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + TenantId string `protobuf:"bytes,1,opt,name=tenant_id,json=tenantId,proto3" json:"tenant_id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Id int64 `protobuf:"varint,3,opt,name=id,proto3" json:"id,omitempty"` + State DatabaseState `protobuf:"varint,4,opt,name=state,proto3,enum=milvus.proto.etcd.DatabaseState" json:"state,omitempty"` + CreatedTime uint64 `protobuf:"varint,5,opt,name=created_time,json=createdTime,proto3" json:"created_time,omitempty"` + Properties []*commonpb.KeyValuePair `protobuf:"bytes,6,rep,name=properties,proto3" json:"properties,omitempty"` +} + +func (x *DatabaseInfo) Reset() { + *x = DatabaseInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_etcd_meta_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DatabaseInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DatabaseInfo) ProtoMessage() {} + +func (x *DatabaseInfo) ProtoReflect() protoreflect.Message { + mi := &file_etcd_meta_proto_msgTypes[5] + 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 DatabaseInfo.ProtoReflect.Descriptor instead. +func (*DatabaseInfo) Descriptor() ([]byte, []int) { + return file_etcd_meta_proto_rawDescGZIP(), []int{5} +} + +func (x *DatabaseInfo) GetTenantId() string { + if x != nil { + return x.TenantId + } + return "" +} + +func (x *DatabaseInfo) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *DatabaseInfo) GetId() int64 { + if x != nil { + return x.Id + } + return 0 +} + +func (x *DatabaseInfo) GetState() DatabaseState { + if x != nil { + return x.State + } + return DatabaseState_DatabaseUnknown +} + +func (x *DatabaseInfo) GetCreatedTime() uint64 { + if x != nil { + return x.CreatedTime + } + return 0 +} + +func (x *DatabaseInfo) GetProperties() []*commonpb.KeyValuePair { + if x != nil { + return x.Properties + } + return nil +} + +type SegmentIndexInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionID int64 `protobuf:"varint,2,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + SegmentID int64 `protobuf:"varint,3,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + FieldID int64 `protobuf:"varint,4,opt,name=fieldID,proto3" json:"fieldID,omitempty"` + IndexID int64 `protobuf:"varint,5,opt,name=indexID,proto3" json:"indexID,omitempty"` + BuildID int64 `protobuf:"varint,6,opt,name=buildID,proto3" json:"buildID,omitempty"` + EnableIndex bool `protobuf:"varint,7,opt,name=enable_index,json=enableIndex,proto3" json:"enable_index,omitempty"` + CreateTime uint64 `protobuf:"varint,8,opt,name=create_time,json=createTime,proto3" json:"create_time,omitempty"` +} + +func (x *SegmentIndexInfo) Reset() { + *x = SegmentIndexInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_etcd_meta_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentIndexInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentIndexInfo) ProtoMessage() {} + +func (x *SegmentIndexInfo) ProtoReflect() protoreflect.Message { + mi := &file_etcd_meta_proto_msgTypes[6] + 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 SegmentIndexInfo.ProtoReflect.Descriptor instead. +func (*SegmentIndexInfo) Descriptor() ([]byte, []int) { + return file_etcd_meta_proto_rawDescGZIP(), []int{6} +} + +func (x *SegmentIndexInfo) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *SegmentIndexInfo) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *SegmentIndexInfo) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *SegmentIndexInfo) GetFieldID() int64 { + if x != nil { + return x.FieldID + } + return 0 +} + +func (x *SegmentIndexInfo) GetIndexID() int64 { + if x != nil { + return x.IndexID + } + return 0 +} + +func (x *SegmentIndexInfo) GetBuildID() int64 { + if x != nil { + return x.BuildID + } + return 0 +} + +func (x *SegmentIndexInfo) GetEnableIndex() bool { + if x != nil { + return x.EnableIndex + } + return false +} + +func (x *SegmentIndexInfo) GetCreateTime() uint64 { + if x != nil { + return x.CreateTime + } + return 0 +} + +// TODO move to proto files of interprocess communication +type CollectionMeta struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ID int64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"` + Schema *schemapb.CollectionSchema `protobuf:"bytes,2,opt,name=schema,proto3" json:"schema,omitempty"` + CreateTime uint64 `protobuf:"varint,3,opt,name=create_time,json=createTime,proto3" json:"create_time,omitempty"` + SegmentIDs []int64 `protobuf:"varint,4,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,omitempty"` + PartitionTags []string `protobuf:"bytes,5,rep,name=partition_tags,json=partitionTags,proto3" json:"partition_tags,omitempty"` + PartitionIDs []int64 `protobuf:"varint,6,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` +} + +func (x *CollectionMeta) Reset() { + *x = CollectionMeta{} + if protoimpl.UnsafeEnabled { + mi := &file_etcd_meta_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CollectionMeta) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CollectionMeta) ProtoMessage() {} + +func (x *CollectionMeta) ProtoReflect() protoreflect.Message { + mi := &file_etcd_meta_proto_msgTypes[7] + 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 CollectionMeta.ProtoReflect.Descriptor instead. +func (*CollectionMeta) Descriptor() ([]byte, []int) { + return file_etcd_meta_proto_rawDescGZIP(), []int{7} +} + +func (x *CollectionMeta) GetID() int64 { + if x != nil { + return x.ID + } + return 0 +} + +func (x *CollectionMeta) GetSchema() *schemapb.CollectionSchema { + if x != nil { + return x.Schema + } + return nil +} + +func (x *CollectionMeta) GetCreateTime() uint64 { + if x != nil { + return x.CreateTime + } + return 0 +} + +func (x *CollectionMeta) GetSegmentIDs() []int64 { + if x != nil { + return x.SegmentIDs + } + return nil +} + +func (x *CollectionMeta) GetPartitionTags() []string { + if x != nil { + return x.PartitionTags + } + return nil +} + +func (x *CollectionMeta) GetPartitionIDs() []int64 { + if x != nil { + return x.PartitionIDs + } + return nil +} + +type CredentialInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Username string `protobuf:"bytes,1,opt,name=username,proto3" json:"username,omitempty"` + // encrypted by bcrypt (for higher security level) + EncryptedPassword string `protobuf:"bytes,2,opt,name=encrypted_password,json=encryptedPassword,proto3" json:"encrypted_password,omitempty"` + Tenant string `protobuf:"bytes,3,opt,name=tenant,proto3" json:"tenant,omitempty"` + IsSuper bool `protobuf:"varint,4,opt,name=is_super,json=isSuper,proto3" json:"is_super,omitempty"` + // encrypted by sha256 (for good performance in cache mapping) + Sha256Password string `protobuf:"bytes,5,opt,name=sha256_password,json=sha256Password,proto3" json:"sha256_password,omitempty"` +} + +func (x *CredentialInfo) Reset() { + *x = CredentialInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_etcd_meta_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CredentialInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CredentialInfo) ProtoMessage() {} + +func (x *CredentialInfo) ProtoReflect() protoreflect.Message { + mi := &file_etcd_meta_proto_msgTypes[8] + 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 CredentialInfo.ProtoReflect.Descriptor instead. +func (*CredentialInfo) Descriptor() ([]byte, []int) { + return file_etcd_meta_proto_rawDescGZIP(), []int{8} +} + +func (x *CredentialInfo) GetUsername() string { + if x != nil { + return x.Username + } + return "" +} + +func (x *CredentialInfo) GetEncryptedPassword() string { + if x != nil { + return x.EncryptedPassword + } + return "" +} + +func (x *CredentialInfo) GetTenant() string { + if x != nil { + return x.Tenant + } + return "" +} + +func (x *CredentialInfo) GetIsSuper() bool { + if x != nil { + return x.IsSuper + } + return false +} + +func (x *CredentialInfo) GetSha256Password() string { + if x != nil { + return x.Sha256Password + } + return "" +} + +var File_etcd_meta_proto protoreflect.FileDescriptor + +var file_etcd_meta_proto_rawDesc = []byte{ + 0x0a, 0x0f, 0x65, 0x74, 0x63, 0x64, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x12, 0x11, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x65, 0x74, 0x63, 0x64, 0x1a, 0x0c, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x1a, 0x0c, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x22, 0xc5, 0x01, 0x0a, 0x09, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1d, + 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x18, 0x0a, + 0x07, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x12, 0x44, 0x0a, 0x0c, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, + 0x52, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x18, 0x0a, + 0x07, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, + 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x22, 0x44, 0x0a, 0x0e, 0x46, 0x69, 0x65, 0x6c, + 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, + 0x6c, 0x65, 0x64, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x6c, + 0x65, 0x64, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x22, 0x90, + 0x06, 0x0a, 0x0e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, + 0x6f, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x49, + 0x44, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x02, 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, 0x1f, 0x0a, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, + 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, + 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x12, 0x26, 0x0a, 0x0e, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x12, 0x46, 0x0a, + 0x0d, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x73, 0x18, 0x06, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x65, 0x74, 0x63, 0x64, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, + 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0c, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, + 0x64, 0x65, 0x78, 0x65, 0x73, 0x12, 0x32, 0x0a, 0x15, 0x76, 0x69, 0x72, 0x74, 0x75, 0x61, 0x6c, + 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x07, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x13, 0x76, 0x69, 0x72, 0x74, 0x75, 0x61, 0x6c, 0x43, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x12, 0x34, 0x0a, 0x16, 0x70, 0x68, 0x79, + 0x73, 0x69, 0x63, 0x61, 0x6c, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x09, 0x52, 0x14, 0x70, 0x68, 0x79, 0x73, 0x69, + 0x63, 0x61, 0x6c, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x12, + 0x40, 0x0a, 0x1c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x73, 0x18, + 0x09, 0x20, 0x03, 0x28, 0x04, 0x52, 0x1a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, + 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x5f, 0x6e, 0x75, 0x6d, 0x18, + 0x0a, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x4e, 0x75, 0x6d, + 0x12, 0x49, 0x0a, 0x0f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x18, 0x0b, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x4b, 0x65, 0x79, 0x44, 0x61, 0x74, 0x61, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0e, 0x73, 0x74, 0x61, + 0x72, 0x74, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x52, 0x0a, 0x11, 0x63, + 0x6f, 0x6e, 0x73, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x63, 0x79, 0x5f, 0x6c, 0x65, 0x76, 0x65, 0x6c, + 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x43, 0x6f, 0x6e, + 0x73, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x63, 0x79, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x10, 0x63, + 0x6f, 0x6e, 0x73, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x63, 0x79, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, + 0x38, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x22, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x65, 0x74, + 0x63, 0x64, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x41, 0x0a, 0x0a, 0x70, 0x72, 0x6f, + 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x18, 0x0e, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, + 0x52, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x12, 0x13, 0x0a, 0x05, + 0x64, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, + 0x64, 0x22, 0xf5, 0x01, 0x0a, 0x0d, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x6e, 0x66, 0x6f, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x24, 0x0a, 0x0d, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x70, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3e, 0x0a, 0x1b, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, + 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, + 0x52, 0x19, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x23, 0x0a, 0x0d, 0x63, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, + 0x12, 0x37, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x65, + 0x74, 0x63, 0x64, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x22, 0xbc, 0x01, 0x0a, 0x09, 0x41, 0x6c, + 0x69, 0x61, 0x73, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1d, 0x0a, 0x0a, 0x61, 0x6c, 0x69, 0x61, 0x73, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x61, 0x6c, 0x69, + 0x61, 0x73, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x63, + 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x04, 0x52, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x33, + 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1d, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x65, 0x74, 0x63, + 0x64, 0x2e, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, + 0x61, 0x74, 0x65, 0x12, 0x13, 0x0a, 0x05, 0x64, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x64, 0x22, 0xed, 0x01, 0x0a, 0x0c, 0x44, 0x61, 0x74, + 0x61, 0x62, 0x61, 0x73, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x65, 0x6e, + 0x61, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x65, + 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, 0x36, 0x0a, 0x05, 0x73, 0x74, + 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x65, 0x74, 0x63, 0x64, 0x2e, 0x44, 0x61, + 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, + 0x74, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x69, + 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x64, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x41, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, + 0x69, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0a, 0x70, 0x72, + 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x22, 0x88, 0x02, 0x0a, 0x10, 0x53, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x22, 0x0a, + 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, + 0x44, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, + 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x12, + 0x21, 0x0a, 0x0c, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, + 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x64, + 0x65, 0x78, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, + 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, + 0x69, 0x6d, 0x65, 0x22, 0xeb, 0x01, 0x0a, 0x0e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x02, 0x49, 0x44, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x18, 0x02, 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, 0x1f, 0x0a, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, + 0x74, 0x69, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x44, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x61, 0x67, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, + 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x67, 0x73, 0x12, 0x22, 0x0a, + 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x06, 0x20, + 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, + 0x73, 0x22, 0xb7, 0x01, 0x0a, 0x0e, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, + 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1a, 0x0a, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, 0x65, + 0x12, 0x2d, 0x0a, 0x12, 0x65, 0x6e, 0x63, 0x72, 0x79, 0x70, 0x74, 0x65, 0x64, 0x5f, 0x70, 0x61, + 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x65, 0x6e, + 0x63, 0x72, 0x79, 0x70, 0x74, 0x65, 0x64, 0x50, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x12, + 0x16, 0x0a, 0x06, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x06, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x12, 0x19, 0x0a, 0x08, 0x69, 0x73, 0x5f, 0x73, 0x75, + 0x70, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x69, 0x73, 0x53, 0x75, 0x70, + 0x65, 0x72, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x68, 0x61, 0x32, 0x35, 0x36, 0x5f, 0x70, 0x61, 0x73, + 0x73, 0x77, 0x6f, 0x72, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x68, 0x61, + 0x32, 0x35, 0x36, 0x50, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x2a, 0x7a, 0x0a, 0x0d, 0x44, + 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x13, 0x0a, 0x0f, + 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, + 0x00, 0x12, 0x13, 0x0a, 0x0f, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x64, 0x10, 0x01, 0x12, 0x14, 0x0a, 0x10, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, + 0x73, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6e, 0x67, 0x10, 0x02, 0x12, 0x14, 0x0a, 0x10, + 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x44, 0x72, 0x6f, 0x70, 0x70, 0x69, 0x6e, 0x67, + 0x10, 0x03, 0x12, 0x13, 0x0a, 0x0f, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x44, 0x72, + 0x6f, 0x70, 0x70, 0x65, 0x64, 0x10, 0x04, 0x2a, 0x6f, 0x0a, 0x0f, 0x43, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x15, 0x0a, 0x11, 0x43, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x10, + 0x00, 0x12, 0x16, 0x0a, 0x12, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, + 0x72, 0x65, 0x61, 0x74, 0x69, 0x6e, 0x67, 0x10, 0x01, 0x12, 0x16, 0x0a, 0x12, 0x43, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x72, 0x6f, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x10, + 0x02, 0x12, 0x15, 0x0a, 0x11, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, + 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x10, 0x03, 0x2a, 0x6a, 0x0a, 0x0e, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x14, 0x0a, 0x10, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x10, 0x00, + 0x12, 0x15, 0x0a, 0x11, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x69, 0x6e, 0x67, 0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x72, 0x6f, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x10, 0x02, 0x12, 0x14, + 0x0a, 0x10, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x72, 0x6f, 0x70, 0x70, + 0x65, 0x64, 0x10, 0x03, 0x2a, 0x56, 0x0a, 0x0a, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x12, 0x10, 0x0a, 0x0c, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x64, 0x10, 0x00, 0x12, 0x11, 0x0a, 0x0d, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x69, 0x6e, 0x67, 0x10, 0x01, 0x12, 0x11, 0x0a, 0x0d, 0x41, 0x6c, 0x69, 0x61, 0x73, + 0x44, 0x72, 0x6f, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x10, 0x02, 0x12, 0x10, 0x0a, 0x0c, 0x41, 0x6c, + 0x69, 0x61, 0x73, 0x44, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x10, 0x03, 0x42, 0x2e, 0x5a, 0x2c, + 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, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x65, 0x74, 0x63, 0x64, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_etcd_meta_proto_rawDescOnce sync.Once + file_etcd_meta_proto_rawDescData = file_etcd_meta_proto_rawDesc +) + +func file_etcd_meta_proto_rawDescGZIP() []byte { + file_etcd_meta_proto_rawDescOnce.Do(func() { + file_etcd_meta_proto_rawDescData = protoimpl.X.CompressGZIP(file_etcd_meta_proto_rawDescData) + }) + return file_etcd_meta_proto_rawDescData +} + +var file_etcd_meta_proto_enumTypes = make([]protoimpl.EnumInfo, 4) +var file_etcd_meta_proto_msgTypes = make([]protoimpl.MessageInfo, 9) +var file_etcd_meta_proto_goTypes = []interface{}{ + (DatabaseState)(0), // 0: milvus.proto.etcd.DatabaseState + (CollectionState)(0), // 1: milvus.proto.etcd.CollectionState + (PartitionState)(0), // 2: milvus.proto.etcd.PartitionState + (AliasState)(0), // 3: milvus.proto.etcd.AliasState + (*IndexInfo)(nil), // 4: milvus.proto.etcd.IndexInfo + (*FieldIndexInfo)(nil), // 5: milvus.proto.etcd.FieldIndexInfo + (*CollectionInfo)(nil), // 6: milvus.proto.etcd.CollectionInfo + (*PartitionInfo)(nil), // 7: milvus.proto.etcd.PartitionInfo + (*AliasInfo)(nil), // 8: milvus.proto.etcd.AliasInfo + (*DatabaseInfo)(nil), // 9: milvus.proto.etcd.DatabaseInfo + (*SegmentIndexInfo)(nil), // 10: milvus.proto.etcd.SegmentIndexInfo + (*CollectionMeta)(nil), // 11: milvus.proto.etcd.CollectionMeta + (*CredentialInfo)(nil), // 12: milvus.proto.etcd.CredentialInfo + (*commonpb.KeyValuePair)(nil), // 13: milvus.proto.common.KeyValuePair + (*schemapb.CollectionSchema)(nil), // 14: milvus.proto.schema.CollectionSchema + (*commonpb.KeyDataPair)(nil), // 15: milvus.proto.common.KeyDataPair + (commonpb.ConsistencyLevel)(0), // 16: milvus.proto.common.ConsistencyLevel +} +var file_etcd_meta_proto_depIdxs = []int32{ + 13, // 0: milvus.proto.etcd.IndexInfo.index_params:type_name -> milvus.proto.common.KeyValuePair + 14, // 1: milvus.proto.etcd.CollectionInfo.schema:type_name -> milvus.proto.schema.CollectionSchema + 5, // 2: milvus.proto.etcd.CollectionInfo.field_indexes:type_name -> milvus.proto.etcd.FieldIndexInfo + 15, // 3: milvus.proto.etcd.CollectionInfo.start_positions:type_name -> milvus.proto.common.KeyDataPair + 16, // 4: milvus.proto.etcd.CollectionInfo.consistency_level:type_name -> milvus.proto.common.ConsistencyLevel + 1, // 5: milvus.proto.etcd.CollectionInfo.state:type_name -> milvus.proto.etcd.CollectionState + 13, // 6: milvus.proto.etcd.CollectionInfo.properties:type_name -> milvus.proto.common.KeyValuePair + 2, // 7: milvus.proto.etcd.PartitionInfo.state:type_name -> milvus.proto.etcd.PartitionState + 3, // 8: milvus.proto.etcd.AliasInfo.state:type_name -> milvus.proto.etcd.AliasState + 0, // 9: milvus.proto.etcd.DatabaseInfo.state:type_name -> milvus.proto.etcd.DatabaseState + 13, // 10: milvus.proto.etcd.DatabaseInfo.properties:type_name -> milvus.proto.common.KeyValuePair + 14, // 11: milvus.proto.etcd.CollectionMeta.schema:type_name -> milvus.proto.schema.CollectionSchema + 12, // [12:12] is the sub-list for method output_type + 12, // [12:12] is the sub-list for method input_type + 12, // [12:12] is the sub-list for extension type_name + 12, // [12:12] is the sub-list for extension extendee + 0, // [0:12] is the sub-list for field type_name +} + +func init() { file_etcd_meta_proto_init() } +func file_etcd_meta_proto_init() { + if File_etcd_meta_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_etcd_meta_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*IndexInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_etcd_meta_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*FieldIndexInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_etcd_meta_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CollectionInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_etcd_meta_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PartitionInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_etcd_meta_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AliasInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_etcd_meta_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DatabaseInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_etcd_meta_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentIndexInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_etcd_meta_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CollectionMeta); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_etcd_meta_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CredentialInfo); 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{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_etcd_meta_proto_rawDesc, + NumEnums: 4, + NumMessages: 9, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_etcd_meta_proto_goTypes, + DependencyIndexes: file_etcd_meta_proto_depIdxs, + EnumInfos: file_etcd_meta_proto_enumTypes, + MessageInfos: file_etcd_meta_proto_msgTypes, + }.Build() + File_etcd_meta_proto = out.File + file_etcd_meta_proto_rawDesc = nil + file_etcd_meta_proto_goTypes = nil + file_etcd_meta_proto_depIdxs = nil +} diff --git a/internal/proto/index_cgo_msg.proto b/pkg/proto/index_cgo_msg.proto similarity index 96% rename from internal/proto/index_cgo_msg.proto rename to pkg/proto/index_cgo_msg.proto index 92e98100f35ec..d2dbbfcbfe427 100644 --- a/internal/proto/index_cgo_msg.proto +++ b/pkg/proto/index_cgo_msg.proto @@ -1,7 +1,7 @@ syntax = "proto3"; package milvus.proto.indexcgo; -option go_package="github.com/milvus-io/milvus/internal/proto/indexcgopb"; +option go_package="github.com/milvus-io/milvus/pkg/proto/indexcgopb"; import "common.proto"; import "schema.proto"; diff --git a/internal/proto/index_coord.proto b/pkg/proto/index_coord.proto similarity index 99% rename from internal/proto/index_coord.proto rename to pkg/proto/index_coord.proto index b736098d46b82..e13ffd46ac1e5 100644 --- a/internal/proto/index_coord.proto +++ b/pkg/proto/index_coord.proto @@ -2,7 +2,7 @@ syntax = "proto3"; package milvus.proto.index; -option go_package = "github.com/milvus-io/milvus/internal/proto/indexpb"; +option go_package = "github.com/milvus-io/milvus/pkg/proto/indexpb"; import "common.proto"; import "internal.proto"; diff --git a/pkg/proto/indexcgopb/index_cgo_msg.pb.go b/pkg/proto/indexcgopb/index_cgo_msg.pb.go new file mode 100644 index 0000000000000..a5b0f622bfd41 --- /dev/null +++ b/pkg/proto/indexcgopb/index_cgo_msg.pb.go @@ -0,0 +1,1193 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.33.0 +// protoc v3.21.4 +// source: index_cgo_msg.proto + +package indexcgopb + +import ( + commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + 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" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type TypeParams struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Params []*commonpb.KeyValuePair `protobuf:"bytes,1,rep,name=params,proto3" json:"params,omitempty"` +} + +func (x *TypeParams) Reset() { + *x = TypeParams{} + if protoimpl.UnsafeEnabled { + mi := &file_index_cgo_msg_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TypeParams) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TypeParams) ProtoMessage() {} + +func (x *TypeParams) ProtoReflect() protoreflect.Message { + mi := &file_index_cgo_msg_proto_msgTypes[0] + 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 TypeParams.ProtoReflect.Descriptor instead. +func (*TypeParams) Descriptor() ([]byte, []int) { + return file_index_cgo_msg_proto_rawDescGZIP(), []int{0} +} + +func (x *TypeParams) GetParams() []*commonpb.KeyValuePair { + if x != nil { + return x.Params + } + return nil +} + +type IndexParams struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Params []*commonpb.KeyValuePair `protobuf:"bytes,1,rep,name=params,proto3" json:"params,omitempty"` +} + +func (x *IndexParams) Reset() { + *x = IndexParams{} + if protoimpl.UnsafeEnabled { + mi := &file_index_cgo_msg_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *IndexParams) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*IndexParams) ProtoMessage() {} + +func (x *IndexParams) ProtoReflect() protoreflect.Message { + mi := &file_index_cgo_msg_proto_msgTypes[1] + 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 IndexParams.ProtoReflect.Descriptor instead. +func (*IndexParams) Descriptor() ([]byte, []int) { + return file_index_cgo_msg_proto_rawDescGZIP(), []int{1} +} + +func (x *IndexParams) GetParams() []*commonpb.KeyValuePair { + if x != nil { + return x.Params + } + return nil +} + +// TypeParams & IndexParams will be replaced by MapParams later +type MapParams struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Params []*commonpb.KeyValuePair `protobuf:"bytes,1,rep,name=params,proto3" json:"params,omitempty"` +} + +func (x *MapParams) Reset() { + *x = MapParams{} + if protoimpl.UnsafeEnabled { + mi := &file_index_cgo_msg_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *MapParams) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MapParams) ProtoMessage() {} + +func (x *MapParams) ProtoReflect() protoreflect.Message { + mi := &file_index_cgo_msg_proto_msgTypes[2] + 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 MapParams.ProtoReflect.Descriptor instead. +func (*MapParams) Descriptor() ([]byte, []int) { + return file_index_cgo_msg_proto_rawDescGZIP(), []int{2} +} + +func (x *MapParams) GetParams() []*commonpb.KeyValuePair { + if x != nil { + return x.Params + } + return nil +} + +type MapParamsV2 struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Params map[string]string `protobuf:"bytes,1,rep,name=params,proto3" json:"params,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *MapParamsV2) Reset() { + *x = MapParamsV2{} + if protoimpl.UnsafeEnabled { + mi := &file_index_cgo_msg_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *MapParamsV2) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MapParamsV2) ProtoMessage() {} + +func (x *MapParamsV2) ProtoReflect() protoreflect.Message { + mi := &file_index_cgo_msg_proto_msgTypes[3] + 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 MapParamsV2.ProtoReflect.Descriptor instead. +func (*MapParamsV2) Descriptor() ([]byte, []int) { + return file_index_cgo_msg_proto_rawDescGZIP(), []int{3} +} + +func (x *MapParamsV2) GetParams() map[string]string { + if x != nil { + return x.Params + } + return nil +} + +type Binary struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` + Value []byte `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` +} + +func (x *Binary) Reset() { + *x = Binary{} + if protoimpl.UnsafeEnabled { + mi := &file_index_cgo_msg_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Binary) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Binary) ProtoMessage() {} + +func (x *Binary) ProtoReflect() protoreflect.Message { + mi := &file_index_cgo_msg_proto_msgTypes[4] + 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 Binary.ProtoReflect.Descriptor instead. +func (*Binary) Descriptor() ([]byte, []int) { + return file_index_cgo_msg_proto_rawDescGZIP(), []int{4} +} + +func (x *Binary) GetKey() string { + if x != nil { + return x.Key + } + return "" +} + +func (x *Binary) GetValue() []byte { + if x != nil { + return x.Value + } + return nil +} + +type BinarySet struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Datas []*Binary `protobuf:"bytes,1,rep,name=datas,proto3" json:"datas,omitempty"` +} + +func (x *BinarySet) Reset() { + *x = BinarySet{} + if protoimpl.UnsafeEnabled { + mi := &file_index_cgo_msg_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BinarySet) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BinarySet) ProtoMessage() {} + +func (x *BinarySet) ProtoReflect() protoreflect.Message { + mi := &file_index_cgo_msg_proto_msgTypes[5] + 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 BinarySet.ProtoReflect.Descriptor instead. +func (*BinarySet) Descriptor() ([]byte, []int) { + return file_index_cgo_msg_proto_rawDescGZIP(), []int{5} +} + +func (x *BinarySet) GetDatas() []*Binary { + if x != nil { + return x.Datas + } + return nil +} + +// Synchronously modify StorageConfig in index_coord.proto file +type StorageConfig struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Address string `protobuf:"bytes,1,opt,name=address,proto3" json:"address,omitempty"` + AccessKeyID string `protobuf:"bytes,2,opt,name=access_keyID,json=accessKeyID,proto3" json:"access_keyID,omitempty"` + SecretAccessKey string `protobuf:"bytes,3,opt,name=secret_access_key,json=secretAccessKey,proto3" json:"secret_access_key,omitempty"` + UseSSL bool `protobuf:"varint,4,opt,name=useSSL,proto3" json:"useSSL,omitempty"` + BucketName string `protobuf:"bytes,5,opt,name=bucket_name,json=bucketName,proto3" json:"bucket_name,omitempty"` + RootPath string `protobuf:"bytes,6,opt,name=root_path,json=rootPath,proto3" json:"root_path,omitempty"` + UseIAM bool `protobuf:"varint,7,opt,name=useIAM,proto3" json:"useIAM,omitempty"` + IAMEndpoint string `protobuf:"bytes,8,opt,name=IAMEndpoint,proto3" json:"IAMEndpoint,omitempty"` + StorageType string `protobuf:"bytes,9,opt,name=storage_type,json=storageType,proto3" json:"storage_type,omitempty"` + UseVirtualHost bool `protobuf:"varint,10,opt,name=use_virtual_host,json=useVirtualHost,proto3" json:"use_virtual_host,omitempty"` + Region string `protobuf:"bytes,11,opt,name=region,proto3" json:"region,omitempty"` + CloudProvider string `protobuf:"bytes,12,opt,name=cloud_provider,json=cloudProvider,proto3" json:"cloud_provider,omitempty"` + RequestTimeoutMs int64 `protobuf:"varint,13,opt,name=request_timeout_ms,json=requestTimeoutMs,proto3" json:"request_timeout_ms,omitempty"` + SslCACert string `protobuf:"bytes,14,opt,name=sslCACert,proto3" json:"sslCACert,omitempty"` + GcpCredentialJSON string `protobuf:"bytes,15,opt,name=GcpCredentialJSON,proto3" json:"GcpCredentialJSON,omitempty"` +} + +func (x *StorageConfig) Reset() { + *x = StorageConfig{} + if protoimpl.UnsafeEnabled { + mi := &file_index_cgo_msg_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StorageConfig) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StorageConfig) ProtoMessage() {} + +func (x *StorageConfig) ProtoReflect() protoreflect.Message { + mi := &file_index_cgo_msg_proto_msgTypes[6] + 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 StorageConfig.ProtoReflect.Descriptor instead. +func (*StorageConfig) Descriptor() ([]byte, []int) { + return file_index_cgo_msg_proto_rawDescGZIP(), []int{6} +} + +func (x *StorageConfig) GetAddress() string { + if x != nil { + return x.Address + } + return "" +} + +func (x *StorageConfig) GetAccessKeyID() string { + if x != nil { + return x.AccessKeyID + } + return "" +} + +func (x *StorageConfig) GetSecretAccessKey() string { + if x != nil { + return x.SecretAccessKey + } + return "" +} + +func (x *StorageConfig) GetUseSSL() bool { + if x != nil { + return x.UseSSL + } + return false +} + +func (x *StorageConfig) GetBucketName() string { + if x != nil { + return x.BucketName + } + return "" +} + +func (x *StorageConfig) GetRootPath() string { + if x != nil { + return x.RootPath + } + return "" +} + +func (x *StorageConfig) GetUseIAM() bool { + if x != nil { + return x.UseIAM + } + return false +} + +func (x *StorageConfig) GetIAMEndpoint() string { + if x != nil { + return x.IAMEndpoint + } + return "" +} + +func (x *StorageConfig) GetStorageType() string { + if x != nil { + return x.StorageType + } + return "" +} + +func (x *StorageConfig) GetUseVirtualHost() bool { + if x != nil { + return x.UseVirtualHost + } + return false +} + +func (x *StorageConfig) GetRegion() string { + if x != nil { + return x.Region + } + return "" +} + +func (x *StorageConfig) GetCloudProvider() string { + if x != nil { + return x.CloudProvider + } + return "" +} + +func (x *StorageConfig) GetRequestTimeoutMs() int64 { + if x != nil { + return x.RequestTimeoutMs + } + return 0 +} + +func (x *StorageConfig) GetSslCACert() string { + if x != nil { + return x.SslCACert + } + return "" +} + +func (x *StorageConfig) GetGcpCredentialJSON() string { + if x != nil { + return x.GcpCredentialJSON + } + return "" +} + +// Synchronously modify OptionalFieldInfo in index_coord.proto file +type OptionalFieldInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + FieldID int64 `protobuf:"varint,1,opt,name=fieldID,proto3" json:"fieldID,omitempty"` + FieldName string `protobuf:"bytes,2,opt,name=field_name,json=fieldName,proto3" json:"field_name,omitempty"` + FieldType int32 `protobuf:"varint,3,opt,name=field_type,json=fieldType,proto3" json:"field_type,omitempty"` + DataPaths []string `protobuf:"bytes,4,rep,name=data_paths,json=dataPaths,proto3" json:"data_paths,omitempty"` +} + +func (x *OptionalFieldInfo) Reset() { + *x = OptionalFieldInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_index_cgo_msg_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *OptionalFieldInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*OptionalFieldInfo) ProtoMessage() {} + +func (x *OptionalFieldInfo) ProtoReflect() protoreflect.Message { + mi := &file_index_cgo_msg_proto_msgTypes[7] + 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 OptionalFieldInfo.ProtoReflect.Descriptor instead. +func (*OptionalFieldInfo) Descriptor() ([]byte, []int) { + return file_index_cgo_msg_proto_rawDescGZIP(), []int{7} +} + +func (x *OptionalFieldInfo) GetFieldID() int64 { + if x != nil { + return x.FieldID + } + return 0 +} + +func (x *OptionalFieldInfo) GetFieldName() string { + if x != nil { + return x.FieldName + } + return "" +} + +func (x *OptionalFieldInfo) GetFieldType() int32 { + if x != nil { + return x.FieldType + } + return 0 +} + +func (x *OptionalFieldInfo) GetDataPaths() []string { + if x != nil { + return x.DataPaths + } + return nil +} + +type BuildIndexInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ClusterID string `protobuf:"bytes,1,opt,name=clusterID,proto3" json:"clusterID,omitempty"` + BuildID int64 `protobuf:"varint,2,opt,name=buildID,proto3" json:"buildID,omitempty"` + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionID int64 `protobuf:"varint,4,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + SegmentID int64 `protobuf:"varint,5,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + IndexVersion int64 `protobuf:"varint,6,opt,name=index_version,json=indexVersion,proto3" json:"index_version,omitempty"` + CurrentIndexVersion int32 `protobuf:"varint,7,opt,name=current_index_version,json=currentIndexVersion,proto3" json:"current_index_version,omitempty"` + NumRows int64 `protobuf:"varint,8,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"` + Dim int64 `protobuf:"varint,9,opt,name=dim,proto3" json:"dim,omitempty"` + IndexFilePrefix string `protobuf:"bytes,10,opt,name=index_file_prefix,json=indexFilePrefix,proto3" json:"index_file_prefix,omitempty"` + InsertFiles []string `protobuf:"bytes,11,rep,name=insert_files,json=insertFiles,proto3" json:"insert_files,omitempty"` + // repeated int64 data_ids = 12; + FieldSchema *schemapb.FieldSchema `protobuf:"bytes,12,opt,name=field_schema,json=fieldSchema,proto3" json:"field_schema,omitempty"` + StorageConfig *StorageConfig `protobuf:"bytes,13,opt,name=storage_config,json=storageConfig,proto3" json:"storage_config,omitempty"` + IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,14,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"` + TypeParams []*commonpb.KeyValuePair `protobuf:"bytes,15,rep,name=type_params,json=typeParams,proto3" json:"type_params,omitempty"` + StorePath string `protobuf:"bytes,16,opt,name=store_path,json=storePath,proto3" json:"store_path,omitempty"` + StoreVersion int64 `protobuf:"varint,17,opt,name=store_version,json=storeVersion,proto3" json:"store_version,omitempty"` + IndexStorePath string `protobuf:"bytes,18,opt,name=index_store_path,json=indexStorePath,proto3" json:"index_store_path,omitempty"` + OptFields []*OptionalFieldInfo `protobuf:"bytes,19,rep,name=opt_fields,json=optFields,proto3" json:"opt_fields,omitempty"` + PartitionKeyIsolation bool `protobuf:"varint,20,opt,name=partition_key_isolation,json=partitionKeyIsolation,proto3" json:"partition_key_isolation,omitempty"` +} + +func (x *BuildIndexInfo) Reset() { + *x = BuildIndexInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_index_cgo_msg_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BuildIndexInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BuildIndexInfo) ProtoMessage() {} + +func (x *BuildIndexInfo) ProtoReflect() protoreflect.Message { + mi := &file_index_cgo_msg_proto_msgTypes[8] + 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 BuildIndexInfo.ProtoReflect.Descriptor instead. +func (*BuildIndexInfo) Descriptor() ([]byte, []int) { + return file_index_cgo_msg_proto_rawDescGZIP(), []int{8} +} + +func (x *BuildIndexInfo) GetClusterID() string { + if x != nil { + return x.ClusterID + } + return "" +} + +func (x *BuildIndexInfo) GetBuildID() int64 { + if x != nil { + return x.BuildID + } + return 0 +} + +func (x *BuildIndexInfo) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *BuildIndexInfo) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *BuildIndexInfo) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *BuildIndexInfo) GetIndexVersion() int64 { + if x != nil { + return x.IndexVersion + } + return 0 +} + +func (x *BuildIndexInfo) GetCurrentIndexVersion() int32 { + if x != nil { + return x.CurrentIndexVersion + } + return 0 +} + +func (x *BuildIndexInfo) GetNumRows() int64 { + if x != nil { + return x.NumRows + } + return 0 +} + +func (x *BuildIndexInfo) GetDim() int64 { + if x != nil { + return x.Dim + } + return 0 +} + +func (x *BuildIndexInfo) GetIndexFilePrefix() string { + if x != nil { + return x.IndexFilePrefix + } + return "" +} + +func (x *BuildIndexInfo) GetInsertFiles() []string { + if x != nil { + return x.InsertFiles + } + return nil +} + +func (x *BuildIndexInfo) GetFieldSchema() *schemapb.FieldSchema { + if x != nil { + return x.FieldSchema + } + return nil +} + +func (x *BuildIndexInfo) GetStorageConfig() *StorageConfig { + if x != nil { + return x.StorageConfig + } + return nil +} + +func (x *BuildIndexInfo) GetIndexParams() []*commonpb.KeyValuePair { + if x != nil { + return x.IndexParams + } + return nil +} + +func (x *BuildIndexInfo) GetTypeParams() []*commonpb.KeyValuePair { + if x != nil { + return x.TypeParams + } + return nil +} + +func (x *BuildIndexInfo) GetStorePath() string { + if x != nil { + return x.StorePath + } + return "" +} + +func (x *BuildIndexInfo) GetStoreVersion() int64 { + if x != nil { + return x.StoreVersion + } + return 0 +} + +func (x *BuildIndexInfo) GetIndexStorePath() string { + if x != nil { + return x.IndexStorePath + } + return "" +} + +func (x *BuildIndexInfo) GetOptFields() []*OptionalFieldInfo { + if x != nil { + return x.OptFields + } + return nil +} + +func (x *BuildIndexInfo) GetPartitionKeyIsolation() bool { + if x != nil { + return x.PartitionKeyIsolation + } + return false +} + +type LoadTextIndexInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + FieldID int64 `protobuf:"varint,1,opt,name=FieldID,proto3" json:"FieldID,omitempty"` + Version int64 `protobuf:"varint,2,opt,name=version,proto3" json:"version,omitempty"` + BuildID int64 `protobuf:"varint,3,opt,name=buildID,proto3" json:"buildID,omitempty"` + Files []string `protobuf:"bytes,4,rep,name=files,proto3" json:"files,omitempty"` + Schema *schemapb.FieldSchema `protobuf:"bytes,5,opt,name=schema,proto3" json:"schema,omitempty"` + CollectionID int64 `protobuf:"varint,6,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionID int64 `protobuf:"varint,7,opt,name=partitionID,proto3" json:"partitionID,omitempty"` +} + +func (x *LoadTextIndexInfo) Reset() { + *x = LoadTextIndexInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_index_cgo_msg_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *LoadTextIndexInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LoadTextIndexInfo) ProtoMessage() {} + +func (x *LoadTextIndexInfo) ProtoReflect() protoreflect.Message { + mi := &file_index_cgo_msg_proto_msgTypes[9] + 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 LoadTextIndexInfo.ProtoReflect.Descriptor instead. +func (*LoadTextIndexInfo) Descriptor() ([]byte, []int) { + return file_index_cgo_msg_proto_rawDescGZIP(), []int{9} +} + +func (x *LoadTextIndexInfo) GetFieldID() int64 { + if x != nil { + return x.FieldID + } + return 0 +} + +func (x *LoadTextIndexInfo) GetVersion() int64 { + if x != nil { + return x.Version + } + return 0 +} + +func (x *LoadTextIndexInfo) GetBuildID() int64 { + if x != nil { + return x.BuildID + } + return 0 +} + +func (x *LoadTextIndexInfo) GetFiles() []string { + if x != nil { + return x.Files + } + return nil +} + +func (x *LoadTextIndexInfo) GetSchema() *schemapb.FieldSchema { + if x != nil { + return x.Schema + } + return nil +} + +func (x *LoadTextIndexInfo) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *LoadTextIndexInfo) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +var File_index_cgo_msg_proto protoreflect.FileDescriptor + +var file_index_cgo_msg_proto_rawDesc = []byte{ + 0x0a, 0x13, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x63, 0x67, 0x6f, 0x5f, 0x6d, 0x73, 0x67, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x15, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x63, 0x67, 0x6f, 0x1a, 0x0c, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x0c, 0x73, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x47, 0x0a, 0x0a, 0x54, 0x79, 0x70, 0x65, + 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x39, 0x0a, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, + 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, + 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, + 0x73, 0x22, 0x48, 0x0a, 0x0b, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, + 0x12, 0x39, 0x0a, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, + 0x61, 0x69, 0x72, 0x52, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x22, 0x46, 0x0a, 0x09, 0x4d, + 0x61, 0x70, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x39, 0x0a, 0x06, 0x70, 0x61, 0x72, 0x61, + 0x6d, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, + 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x06, 0x70, 0x61, 0x72, + 0x61, 0x6d, 0x73, 0x22, 0x90, 0x01, 0x0a, 0x0b, 0x4d, 0x61, 0x70, 0x50, 0x61, 0x72, 0x61, 0x6d, + 0x73, 0x56, 0x32, 0x12, 0x46, 0x0a, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x63, 0x67, 0x6f, 0x2e, 0x4d, 0x61, 0x70, 0x50, + 0x61, 0x72, 0x61, 0x6d, 0x73, 0x56, 0x32, 0x2e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x1a, 0x39, 0x0a, 0x0b, 0x50, + 0x61, 0x72, 0x61, 0x6d, 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, 0x30, 0x0a, 0x06, 0x42, 0x69, 0x6e, 0x61, 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, + 0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0x40, 0x0a, 0x09, 0x42, 0x69, 0x6e, 0x61, + 0x72, 0x79, 0x53, 0x65, 0x74, 0x12, 0x33, 0x0a, 0x05, 0x64, 0x61, 0x74, 0x61, 0x73, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x63, 0x67, 0x6f, 0x2e, 0x42, 0x69, 0x6e, + 0x61, 0x72, 0x79, 0x52, 0x05, 0x64, 0x61, 0x74, 0x61, 0x73, 0x22, 0x8e, 0x04, 0x0a, 0x0d, 0x53, + 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x18, 0x0a, 0x07, + 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, + 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, + 0x5f, 0x6b, 0x65, 0x79, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x61, 0x63, + 0x63, 0x65, 0x73, 0x73, 0x4b, 0x65, 0x79, 0x49, 0x44, 0x12, 0x2a, 0x0a, 0x11, 0x73, 0x65, 0x63, + 0x72, 0x65, 0x74, 0x5f, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, 0x41, 0x63, 0x63, 0x65, + 0x73, 0x73, 0x4b, 0x65, 0x79, 0x12, 0x16, 0x0a, 0x06, 0x75, 0x73, 0x65, 0x53, 0x53, 0x4c, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x75, 0x73, 0x65, 0x53, 0x53, 0x4c, 0x12, 0x1f, 0x0a, + 0x0b, 0x62, 0x75, 0x63, 0x6b, 0x65, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0a, 0x62, 0x75, 0x63, 0x6b, 0x65, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1b, + 0x0a, 0x09, 0x72, 0x6f, 0x6f, 0x74, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x06, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x08, 0x72, 0x6f, 0x6f, 0x74, 0x50, 0x61, 0x74, 0x68, 0x12, 0x16, 0x0a, 0x06, 0x75, + 0x73, 0x65, 0x49, 0x41, 0x4d, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x75, 0x73, 0x65, + 0x49, 0x41, 0x4d, 0x12, 0x20, 0x0a, 0x0b, 0x49, 0x41, 0x4d, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, + 0x6e, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x49, 0x41, 0x4d, 0x45, 0x6e, 0x64, + 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, + 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x74, 0x6f, + 0x72, 0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x28, 0x0a, 0x10, 0x75, 0x73, 0x65, 0x5f, + 0x76, 0x69, 0x72, 0x74, 0x75, 0x61, 0x6c, 0x5f, 0x68, 0x6f, 0x73, 0x74, 0x18, 0x0a, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0e, 0x75, 0x73, 0x65, 0x56, 0x69, 0x72, 0x74, 0x75, 0x61, 0x6c, 0x48, 0x6f, + 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x18, 0x0b, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6c, + 0x6f, 0x75, 0x64, 0x5f, 0x70, 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x18, 0x0c, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0d, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x50, 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, + 0x72, 0x12, 0x2c, 0x0a, 0x12, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x74, 0x69, 0x6d, + 0x65, 0x6f, 0x75, 0x74, 0x5f, 0x6d, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x72, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4d, 0x73, 0x12, + 0x1c, 0x0a, 0x09, 0x73, 0x73, 0x6c, 0x43, 0x41, 0x43, 0x65, 0x72, 0x74, 0x18, 0x0e, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x09, 0x73, 0x73, 0x6c, 0x43, 0x41, 0x43, 0x65, 0x72, 0x74, 0x12, 0x2c, 0x0a, + 0x11, 0x47, 0x63, 0x70, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x4a, 0x53, + 0x4f, 0x4e, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x47, 0x63, 0x70, 0x43, 0x72, 0x65, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x4a, 0x53, 0x4f, 0x4e, 0x22, 0x8a, 0x01, 0x0a, 0x11, + 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x66, + 0x6f, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x66, + 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x09, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x66, 0x69, + 0x65, 0x6c, 0x64, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, + 0x66, 0x69, 0x65, 0x6c, 0x64, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x64, 0x61, 0x74, + 0x61, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x64, + 0x61, 0x74, 0x61, 0x50, 0x61, 0x74, 0x68, 0x73, 0x22, 0x8c, 0x07, 0x0a, 0x0e, 0x42, 0x75, 0x69, + 0x6c, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1c, 0x0a, 0x09, 0x63, + 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, + 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x62, 0x75, 0x69, + 0x6c, 0x64, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x62, 0x75, 0x69, 0x6c, + 0x64, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x32, 0x0a, 0x15, + 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x76, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x05, 0x52, 0x13, 0x63, 0x75, 0x72, + 0x72, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x08, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x10, 0x0a, 0x03, 0x64, + 0x69, 0x6d, 0x18, 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x64, 0x69, 0x6d, 0x12, 0x2a, 0x0a, + 0x11, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x70, 0x72, 0x65, 0x66, + 0x69, 0x78, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x46, + 0x69, 0x6c, 0x65, 0x50, 0x72, 0x65, 0x66, 0x69, 0x78, 0x12, 0x21, 0x0a, 0x0c, 0x69, 0x6e, 0x73, + 0x65, 0x72, 0x74, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x0b, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x0b, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x12, 0x43, 0x0a, 0x0c, + 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x0c, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x53, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x52, 0x0b, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x12, 0x4b, 0x0a, 0x0e, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x63, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x63, 0x67, + 0x6f, 0x2e, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, + 0x0d, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x44, + 0x0a, 0x0c, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x0e, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, + 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x50, 0x61, + 0x72, 0x61, 0x6d, 0x73, 0x12, 0x42, 0x0a, 0x0b, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x70, 0x61, 0x72, + 0x61, 0x6d, 0x73, 0x18, 0x0f, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0a, 0x74, 0x79, + 0x70, 0x65, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x74, 0x6f, 0x72, + 0x65, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x10, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x74, + 0x6f, 0x72, 0x65, 0x50, 0x61, 0x74, 0x68, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x74, 0x6f, 0x72, 0x65, + 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x11, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, + 0x73, 0x74, 0x6f, 0x72, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x28, 0x0a, 0x10, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x5f, 0x70, 0x61, 0x74, 0x68, + 0x18, 0x12, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x6f, + 0x72, 0x65, 0x50, 0x61, 0x74, 0x68, 0x12, 0x47, 0x0a, 0x0a, 0x6f, 0x70, 0x74, 0x5f, 0x66, 0x69, + 0x65, 0x6c, 0x64, 0x73, 0x18, 0x13, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x63, + 0x67, 0x6f, 0x2e, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x46, 0x69, 0x65, 0x6c, 0x64, + 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x6f, 0x70, 0x74, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, + 0x36, 0x0a, 0x17, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6b, 0x65, 0x79, + 0x5f, 0x69, 0x73, 0x6f, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x14, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x15, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x49, 0x73, + 0x6f, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xf7, 0x01, 0x0a, 0x11, 0x4c, 0x6f, 0x61, 0x64, + 0x54, 0x65, 0x78, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x18, 0x0a, + 0x07, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, + 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x14, 0x0a, 0x05, 0x66, + 0x69, 0x6c, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x66, 0x69, 0x6c, 0x65, + 0x73, 0x12, 0x38, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x22, 0x0a, 0x0c, 0x63, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, + 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x07, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x44, 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, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x63, 0x67, 0x6f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_index_cgo_msg_proto_rawDescOnce sync.Once + file_index_cgo_msg_proto_rawDescData = file_index_cgo_msg_proto_rawDesc +) + +func file_index_cgo_msg_proto_rawDescGZIP() []byte { + file_index_cgo_msg_proto_rawDescOnce.Do(func() { + file_index_cgo_msg_proto_rawDescData = protoimpl.X.CompressGZIP(file_index_cgo_msg_proto_rawDescData) + }) + return file_index_cgo_msg_proto_rawDescData +} + +var file_index_cgo_msg_proto_msgTypes = make([]protoimpl.MessageInfo, 11) +var file_index_cgo_msg_proto_goTypes = []interface{}{ + (*TypeParams)(nil), // 0: milvus.proto.indexcgo.TypeParams + (*IndexParams)(nil), // 1: milvus.proto.indexcgo.IndexParams + (*MapParams)(nil), // 2: milvus.proto.indexcgo.MapParams + (*MapParamsV2)(nil), // 3: milvus.proto.indexcgo.MapParamsV2 + (*Binary)(nil), // 4: milvus.proto.indexcgo.Binary + (*BinarySet)(nil), // 5: milvus.proto.indexcgo.BinarySet + (*StorageConfig)(nil), // 6: milvus.proto.indexcgo.StorageConfig + (*OptionalFieldInfo)(nil), // 7: milvus.proto.indexcgo.OptionalFieldInfo + (*BuildIndexInfo)(nil), // 8: milvus.proto.indexcgo.BuildIndexInfo + (*LoadTextIndexInfo)(nil), // 9: milvus.proto.indexcgo.LoadTextIndexInfo + nil, // 10: milvus.proto.indexcgo.MapParamsV2.ParamsEntry + (*commonpb.KeyValuePair)(nil), // 11: milvus.proto.common.KeyValuePair + (*schemapb.FieldSchema)(nil), // 12: milvus.proto.schema.FieldSchema +} +var file_index_cgo_msg_proto_depIdxs = []int32{ + 11, // 0: milvus.proto.indexcgo.TypeParams.params:type_name -> milvus.proto.common.KeyValuePair + 11, // 1: milvus.proto.indexcgo.IndexParams.params:type_name -> milvus.proto.common.KeyValuePair + 11, // 2: milvus.proto.indexcgo.MapParams.params:type_name -> milvus.proto.common.KeyValuePair + 10, // 3: milvus.proto.indexcgo.MapParamsV2.params:type_name -> milvus.proto.indexcgo.MapParamsV2.ParamsEntry + 4, // 4: milvus.proto.indexcgo.BinarySet.datas:type_name -> milvus.proto.indexcgo.Binary + 12, // 5: milvus.proto.indexcgo.BuildIndexInfo.field_schema:type_name -> milvus.proto.schema.FieldSchema + 6, // 6: milvus.proto.indexcgo.BuildIndexInfo.storage_config:type_name -> milvus.proto.indexcgo.StorageConfig + 11, // 7: milvus.proto.indexcgo.BuildIndexInfo.index_params:type_name -> milvus.proto.common.KeyValuePair + 11, // 8: milvus.proto.indexcgo.BuildIndexInfo.type_params:type_name -> milvus.proto.common.KeyValuePair + 7, // 9: milvus.proto.indexcgo.BuildIndexInfo.opt_fields:type_name -> milvus.proto.indexcgo.OptionalFieldInfo + 12, // 10: milvus.proto.indexcgo.LoadTextIndexInfo.schema:type_name -> milvus.proto.schema.FieldSchema + 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_index_cgo_msg_proto_init() } +func file_index_cgo_msg_proto_init() { + if File_index_cgo_msg_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_index_cgo_msg_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*TypeParams); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_cgo_msg_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*IndexParams); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_cgo_msg_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*MapParams); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_cgo_msg_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*MapParamsV2); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_cgo_msg_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Binary); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_cgo_msg_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BinarySet); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_cgo_msg_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*StorageConfig); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_cgo_msg_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*OptionalFieldInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_cgo_msg_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BuildIndexInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_cgo_msg_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*LoadTextIndexInfo); 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{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_index_cgo_msg_proto_rawDesc, + NumEnums: 0, + NumMessages: 11, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_index_cgo_msg_proto_goTypes, + DependencyIndexes: file_index_cgo_msg_proto_depIdxs, + MessageInfos: file_index_cgo_msg_proto_msgTypes, + }.Build() + File_index_cgo_msg_proto = out.File + file_index_cgo_msg_proto_rawDesc = nil + file_index_cgo_msg_proto_goTypes = nil + file_index_cgo_msg_proto_depIdxs = nil +} diff --git a/pkg/proto/indexpb/index_coord.pb.go b/pkg/proto/indexpb/index_coord.pb.go new file mode 100644 index 0000000000000..7a206f4a8423e --- /dev/null +++ b/pkg/proto/indexpb/index_coord.pb.go @@ -0,0 +1,3882 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.33.0 +// protoc v3.21.4 +// source: index_coord.proto + +package indexpb + +import ( + commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + schemapb "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type JobType int32 + +const ( + JobType_JobTypeNone JobType = 0 + JobType_JobTypeIndexJob JobType = 1 + JobType_JobTypeAnalyzeJob JobType = 2 + JobType_JobTypeStatsJob JobType = 3 +) + +// Enum value maps for JobType. +var ( + JobType_name = map[int32]string{ + 0: "JobTypeNone", + 1: "JobTypeIndexJob", + 2: "JobTypeAnalyzeJob", + 3: "JobTypeStatsJob", + } + JobType_value = map[string]int32{ + "JobTypeNone": 0, + "JobTypeIndexJob": 1, + "JobTypeAnalyzeJob": 2, + "JobTypeStatsJob": 3, + } +) + +func (x JobType) Enum() *JobType { + p := new(JobType) + *p = x + return p +} + +func (x JobType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (JobType) Descriptor() protoreflect.EnumDescriptor { + return file_index_coord_proto_enumTypes[0].Descriptor() +} + +func (JobType) Type() protoreflect.EnumType { + return &file_index_coord_proto_enumTypes[0] +} + +func (x JobType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use JobType.Descriptor instead. +func (JobType) EnumDescriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{0} +} + +type JobState int32 + +const ( + JobState_JobStateNone JobState = 0 + JobState_JobStateInit JobState = 1 + JobState_JobStateInProgress JobState = 2 + JobState_JobStateFinished JobState = 3 + JobState_JobStateFailed JobState = 4 + JobState_JobStateRetry JobState = 5 +) + +// Enum value maps for JobState. +var ( + JobState_name = map[int32]string{ + 0: "JobStateNone", + 1: "JobStateInit", + 2: "JobStateInProgress", + 3: "JobStateFinished", + 4: "JobStateFailed", + 5: "JobStateRetry", + } + JobState_value = map[string]int32{ + "JobStateNone": 0, + "JobStateInit": 1, + "JobStateInProgress": 2, + "JobStateFinished": 3, + "JobStateFailed": 4, + "JobStateRetry": 5, + } +) + +func (x JobState) Enum() *JobState { + p := new(JobState) + *p = x + return p +} + +func (x JobState) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (JobState) Descriptor() protoreflect.EnumDescriptor { + return file_index_coord_proto_enumTypes[1].Descriptor() +} + +func (JobState) Type() protoreflect.EnumType { + return &file_index_coord_proto_enumTypes[1] +} + +func (x JobState) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use JobState.Descriptor instead. +func (JobState) EnumDescriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{1} +} + +type StatsSubJob int32 + +const ( + StatsSubJob_None StatsSubJob = 0 + StatsSubJob_Sort StatsSubJob = 1 + StatsSubJob_TextIndexJob StatsSubJob = 2 + StatsSubJob_BM25Job StatsSubJob = 3 +) + +// Enum value maps for StatsSubJob. +var ( + StatsSubJob_name = map[int32]string{ + 0: "None", + 1: "Sort", + 2: "TextIndexJob", + 3: "BM25Job", + } + StatsSubJob_value = map[string]int32{ + "None": 0, + "Sort": 1, + "TextIndexJob": 2, + "BM25Job": 3, + } +) + +func (x StatsSubJob) Enum() *StatsSubJob { + p := new(StatsSubJob) + *p = x + return p +} + +func (x StatsSubJob) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (StatsSubJob) Descriptor() protoreflect.EnumDescriptor { + return file_index_coord_proto_enumTypes[2].Descriptor() +} + +func (StatsSubJob) Type() protoreflect.EnumType { + return &file_index_coord_proto_enumTypes[2] +} + +func (x StatsSubJob) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use StatsSubJob.Descriptor instead. +func (StatsSubJob) EnumDescriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{2} +} + +type IndexInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + FieldID int64 `protobuf:"varint,2,opt,name=fieldID,proto3" json:"fieldID,omitempty"` + IndexName string `protobuf:"bytes,3,opt,name=index_name,json=indexName,proto3" json:"index_name,omitempty"` + IndexID int64 `protobuf:"varint,4,opt,name=indexID,proto3" json:"indexID,omitempty"` + TypeParams []*commonpb.KeyValuePair `protobuf:"bytes,5,rep,name=type_params,json=typeParams,proto3" json:"type_params,omitempty"` + IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,6,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"` + // index build progress + // The real-time statistics may not be expected due to the existence of the compaction mechanism. + IndexedRows int64 `protobuf:"varint,7,opt,name=indexed_rows,json=indexedRows,proto3" json:"indexed_rows,omitempty"` + TotalRows int64 `protobuf:"varint,8,opt,name=total_rows,json=totalRows,proto3" json:"total_rows,omitempty"` + // index state + State commonpb.IndexState `protobuf:"varint,9,opt,name=state,proto3,enum=milvus.proto.common.IndexState" json:"state,omitempty"` + IndexStateFailReason string `protobuf:"bytes,10,opt,name=index_state_fail_reason,json=indexStateFailReason,proto3" json:"index_state_fail_reason,omitempty"` + IsAutoIndex bool `protobuf:"varint,11,opt,name=is_auto_index,json=isAutoIndex,proto3" json:"is_auto_index,omitempty"` + UserIndexParams []*commonpb.KeyValuePair `protobuf:"bytes,12,rep,name=user_index_params,json=userIndexParams,proto3" json:"user_index_params,omitempty"` + PendingIndexRows int64 `protobuf:"varint,13,opt,name=pending_index_rows,json=pendingIndexRows,proto3" json:"pending_index_rows,omitempty"` +} + +func (x *IndexInfo) Reset() { + *x = IndexInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *IndexInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*IndexInfo) ProtoMessage() {} + +func (x *IndexInfo) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[0] + 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 IndexInfo.ProtoReflect.Descriptor instead. +func (*IndexInfo) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{0} +} + +func (x *IndexInfo) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *IndexInfo) GetFieldID() int64 { + if x != nil { + return x.FieldID + } + return 0 +} + +func (x *IndexInfo) GetIndexName() string { + if x != nil { + return x.IndexName + } + return "" +} + +func (x *IndexInfo) GetIndexID() int64 { + if x != nil { + return x.IndexID + } + return 0 +} + +func (x *IndexInfo) GetTypeParams() []*commonpb.KeyValuePair { + if x != nil { + return x.TypeParams + } + return nil +} + +func (x *IndexInfo) GetIndexParams() []*commonpb.KeyValuePair { + if x != nil { + return x.IndexParams + } + return nil +} + +func (x *IndexInfo) GetIndexedRows() int64 { + if x != nil { + return x.IndexedRows + } + return 0 +} + +func (x *IndexInfo) GetTotalRows() int64 { + if x != nil { + return x.TotalRows + } + return 0 +} + +func (x *IndexInfo) GetState() commonpb.IndexState { + if x != nil { + return x.State + } + return commonpb.IndexState(0) +} + +func (x *IndexInfo) GetIndexStateFailReason() string { + if x != nil { + return x.IndexStateFailReason + } + return "" +} + +func (x *IndexInfo) GetIsAutoIndex() bool { + if x != nil { + return x.IsAutoIndex + } + return false +} + +func (x *IndexInfo) GetUserIndexParams() []*commonpb.KeyValuePair { + if x != nil { + return x.UserIndexParams + } + return nil +} + +func (x *IndexInfo) GetPendingIndexRows() int64 { + if x != nil { + return x.PendingIndexRows + } + return 0 +} + +type FieldIndex struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + IndexInfo *IndexInfo `protobuf:"bytes,1,opt,name=index_info,json=indexInfo,proto3" json:"index_info,omitempty"` + Deleted bool `protobuf:"varint,2,opt,name=deleted,proto3" json:"deleted,omitempty"` + CreateTime uint64 `protobuf:"varint,3,opt,name=create_time,json=createTime,proto3" json:"create_time,omitempty"` +} + +func (x *FieldIndex) Reset() { + *x = FieldIndex{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *FieldIndex) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FieldIndex) ProtoMessage() {} + +func (x *FieldIndex) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[1] + 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 FieldIndex.ProtoReflect.Descriptor instead. +func (*FieldIndex) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{1} +} + +func (x *FieldIndex) GetIndexInfo() *IndexInfo { + if x != nil { + return x.IndexInfo + } + return nil +} + +func (x *FieldIndex) GetDeleted() bool { + if x != nil { + return x.Deleted + } + return false +} + +func (x *FieldIndex) GetCreateTime() uint64 { + if x != nil { + return x.CreateTime + } + return 0 +} + +type SegmentIndex struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionID int64 `protobuf:"varint,2,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + SegmentID int64 `protobuf:"varint,3,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + NumRows int64 `protobuf:"varint,4,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"` + IndexID int64 `protobuf:"varint,5,opt,name=indexID,proto3" json:"indexID,omitempty"` + BuildID int64 `protobuf:"varint,6,opt,name=buildID,proto3" json:"buildID,omitempty"` + NodeID int64 `protobuf:"varint,7,opt,name=nodeID,proto3" json:"nodeID,omitempty"` + IndexVersion int64 `protobuf:"varint,8,opt,name=index_version,json=indexVersion,proto3" json:"index_version,omitempty"` + State commonpb.IndexState `protobuf:"varint,9,opt,name=state,proto3,enum=milvus.proto.common.IndexState" json:"state,omitempty"` + FailReason string `protobuf:"bytes,10,opt,name=fail_reason,json=failReason,proto3" json:"fail_reason,omitempty"` + IndexFileKeys []string `protobuf:"bytes,11,rep,name=index_file_keys,json=indexFileKeys,proto3" json:"index_file_keys,omitempty"` + Deleted bool `protobuf:"varint,12,opt,name=deleted,proto3" json:"deleted,omitempty"` + CreateTime uint64 `protobuf:"varint,13,opt,name=create_time,json=createTime,proto3" json:"create_time,omitempty"` + SerializeSize uint64 `protobuf:"varint,14,opt,name=serialize_size,json=serializeSize,proto3" json:"serialize_size,omitempty"` + WriteHandoff bool `protobuf:"varint,15,opt,name=write_handoff,json=writeHandoff,proto3" json:"write_handoff,omitempty"` + CurrentIndexVersion int32 `protobuf:"varint,16,opt,name=current_index_version,json=currentIndexVersion,proto3" json:"current_index_version,omitempty"` + IndexStoreVersion int64 `protobuf:"varint,17,opt,name=index_store_version,json=indexStoreVersion,proto3" json:"index_store_version,omitempty"` + FinishedTime uint64 `protobuf:"varint,18,opt,name=finished_time,json=finishedTime,proto3" json:"finished_time,omitempty"` +} + +func (x *SegmentIndex) Reset() { + *x = SegmentIndex{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentIndex) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentIndex) ProtoMessage() {} + +func (x *SegmentIndex) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[2] + 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 SegmentIndex.ProtoReflect.Descriptor instead. +func (*SegmentIndex) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{2} +} + +func (x *SegmentIndex) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *SegmentIndex) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *SegmentIndex) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *SegmentIndex) GetNumRows() int64 { + if x != nil { + return x.NumRows + } + return 0 +} + +func (x *SegmentIndex) GetIndexID() int64 { + if x != nil { + return x.IndexID + } + return 0 +} + +func (x *SegmentIndex) GetBuildID() int64 { + if x != nil { + return x.BuildID + } + return 0 +} + +func (x *SegmentIndex) GetNodeID() int64 { + if x != nil { + return x.NodeID + } + return 0 +} + +func (x *SegmentIndex) GetIndexVersion() int64 { + if x != nil { + return x.IndexVersion + } + return 0 +} + +func (x *SegmentIndex) GetState() commonpb.IndexState { + if x != nil { + return x.State + } + return commonpb.IndexState(0) +} + +func (x *SegmentIndex) GetFailReason() string { + if x != nil { + return x.FailReason + } + return "" +} + +func (x *SegmentIndex) GetIndexFileKeys() []string { + if x != nil { + return x.IndexFileKeys + } + return nil +} + +func (x *SegmentIndex) GetDeleted() bool { + if x != nil { + return x.Deleted + } + return false +} + +func (x *SegmentIndex) GetCreateTime() uint64 { + if x != nil { + return x.CreateTime + } + return 0 +} + +func (x *SegmentIndex) GetSerializeSize() uint64 { + if x != nil { + return x.SerializeSize + } + return 0 +} + +func (x *SegmentIndex) GetWriteHandoff() bool { + if x != nil { + return x.WriteHandoff + } + return false +} + +func (x *SegmentIndex) GetCurrentIndexVersion() int32 { + if x != nil { + return x.CurrentIndexVersion + } + return 0 +} + +func (x *SegmentIndex) GetIndexStoreVersion() int64 { + if x != nil { + return x.IndexStoreVersion + } + return 0 +} + +func (x *SegmentIndex) GetFinishedTime() uint64 { + if x != nil { + return x.FinishedTime + } + return 0 +} + +type RegisterNodeRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + Address *commonpb.Address `protobuf:"bytes,2,opt,name=address,proto3" json:"address,omitempty"` + NodeID int64 `protobuf:"varint,3,opt,name=nodeID,proto3" json:"nodeID,omitempty"` +} + +func (x *RegisterNodeRequest) Reset() { + *x = RegisterNodeRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RegisterNodeRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RegisterNodeRequest) ProtoMessage() {} + +func (x *RegisterNodeRequest) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[3] + 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 RegisterNodeRequest.ProtoReflect.Descriptor instead. +func (*RegisterNodeRequest) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{3} +} + +func (x *RegisterNodeRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *RegisterNodeRequest) GetAddress() *commonpb.Address { + if x != nil { + return x.Address + } + return nil +} + +func (x *RegisterNodeRequest) GetNodeID() int64 { + if x != nil { + return x.NodeID + } + return 0 +} + +type RegisterNodeResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + InitParams *internalpb.InitParams `protobuf:"bytes,2,opt,name=init_params,json=initParams,proto3" json:"init_params,omitempty"` +} + +func (x *RegisterNodeResponse) Reset() { + *x = RegisterNodeResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RegisterNodeResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RegisterNodeResponse) ProtoMessage() {} + +func (x *RegisterNodeResponse) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[4] + 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 RegisterNodeResponse.ProtoReflect.Descriptor instead. +func (*RegisterNodeResponse) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{4} +} + +func (x *RegisterNodeResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *RegisterNodeResponse) GetInitParams() *internalpb.InitParams { + if x != nil { + return x.InitParams + } + return nil +} + +type GetIndexStateRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + IndexName string `protobuf:"bytes,2,opt,name=index_name,json=indexName,proto3" json:"index_name,omitempty"` +} + +func (x *GetIndexStateRequest) Reset() { + *x = GetIndexStateRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetIndexStateRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetIndexStateRequest) ProtoMessage() {} + +func (x *GetIndexStateRequest) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[5] + 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 GetIndexStateRequest.ProtoReflect.Descriptor instead. +func (*GetIndexStateRequest) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{5} +} + +func (x *GetIndexStateRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *GetIndexStateRequest) GetIndexName() string { + if x != nil { + return x.IndexName + } + return "" +} + +type GetIndexStateResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + State commonpb.IndexState `protobuf:"varint,2,opt,name=state,proto3,enum=milvus.proto.common.IndexState" json:"state,omitempty"` + FailReason string `protobuf:"bytes,3,opt,name=fail_reason,json=failReason,proto3" json:"fail_reason,omitempty"` +} + +func (x *GetIndexStateResponse) Reset() { + *x = GetIndexStateResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetIndexStateResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetIndexStateResponse) ProtoMessage() {} + +func (x *GetIndexStateResponse) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[6] + 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 GetIndexStateResponse.ProtoReflect.Descriptor instead. +func (*GetIndexStateResponse) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{6} +} + +func (x *GetIndexStateResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *GetIndexStateResponse) GetState() commonpb.IndexState { + if x != nil { + return x.State + } + return commonpb.IndexState(0) +} + +func (x *GetIndexStateResponse) GetFailReason() string { + if x != nil { + return x.FailReason + } + return "" +} + +type GetSegmentIndexStateRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + IndexName string `protobuf:"bytes,2,opt,name=index_name,json=indexName,proto3" json:"index_name,omitempty"` + SegmentIDs []int64 `protobuf:"varint,3,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,omitempty"` +} + +func (x *GetSegmentIndexStateRequest) Reset() { + *x = GetSegmentIndexStateRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetSegmentIndexStateRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetSegmentIndexStateRequest) ProtoMessage() {} + +func (x *GetSegmentIndexStateRequest) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[7] + 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 GetSegmentIndexStateRequest.ProtoReflect.Descriptor instead. +func (*GetSegmentIndexStateRequest) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{7} +} + +func (x *GetSegmentIndexStateRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *GetSegmentIndexStateRequest) GetIndexName() string { + if x != nil { + return x.IndexName + } + return "" +} + +func (x *GetSegmentIndexStateRequest) GetSegmentIDs() []int64 { + if x != nil { + return x.SegmentIDs + } + return nil +} + +type SegmentIndexState struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SegmentID int64 `protobuf:"varint,1,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + State commonpb.IndexState `protobuf:"varint,2,opt,name=state,proto3,enum=milvus.proto.common.IndexState" json:"state,omitempty"` + FailReason string `protobuf:"bytes,3,opt,name=fail_reason,json=failReason,proto3" json:"fail_reason,omitempty"` + IndexName string `protobuf:"bytes,4,opt,name=index_name,json=indexName,proto3" json:"index_name,omitempty"` +} + +func (x *SegmentIndexState) Reset() { + *x = SegmentIndexState{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentIndexState) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentIndexState) ProtoMessage() {} + +func (x *SegmentIndexState) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[8] + 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 SegmentIndexState.ProtoReflect.Descriptor instead. +func (*SegmentIndexState) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{8} +} + +func (x *SegmentIndexState) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *SegmentIndexState) GetState() commonpb.IndexState { + if x != nil { + return x.State + } + return commonpb.IndexState(0) +} + +func (x *SegmentIndexState) GetFailReason() string { + if x != nil { + return x.FailReason + } + return "" +} + +func (x *SegmentIndexState) GetIndexName() string { + if x != nil { + return x.IndexName + } + return "" +} + +type GetSegmentIndexStateResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + States []*SegmentIndexState `protobuf:"bytes,2,rep,name=states,proto3" json:"states,omitempty"` +} + +func (x *GetSegmentIndexStateResponse) Reset() { + *x = GetSegmentIndexStateResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetSegmentIndexStateResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetSegmentIndexStateResponse) ProtoMessage() {} + +func (x *GetSegmentIndexStateResponse) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[9] + 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 GetSegmentIndexStateResponse.ProtoReflect.Descriptor instead. +func (*GetSegmentIndexStateResponse) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{9} +} + +func (x *GetSegmentIndexStateResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *GetSegmentIndexStateResponse) GetStates() []*SegmentIndexState { + if x != nil { + return x.States + } + return nil +} + +type CreateIndexRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + FieldID int64 `protobuf:"varint,2,opt,name=fieldID,proto3" json:"fieldID,omitempty"` + IndexName string `protobuf:"bytes,3,opt,name=index_name,json=indexName,proto3" json:"index_name,omitempty"` + TypeParams []*commonpb.KeyValuePair `protobuf:"bytes,4,rep,name=type_params,json=typeParams,proto3" json:"type_params,omitempty"` + IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,5,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"` + Timestamp uint64 `protobuf:"varint,6,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + IsAutoIndex bool `protobuf:"varint,7,opt,name=is_auto_index,json=isAutoIndex,proto3" json:"is_auto_index,omitempty"` + UserIndexParams []*commonpb.KeyValuePair `protobuf:"bytes,8,rep,name=user_index_params,json=userIndexParams,proto3" json:"user_index_params,omitempty"` + UserAutoindexMetricTypeSpecified bool `protobuf:"varint,9,opt,name=user_autoindex_metric_type_specified,json=userAutoindexMetricTypeSpecified,proto3" json:"user_autoindex_metric_type_specified,omitempty"` +} + +func (x *CreateIndexRequest) Reset() { + *x = CreateIndexRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateIndexRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateIndexRequest) ProtoMessage() {} + +func (x *CreateIndexRequest) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[10] + 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 CreateIndexRequest.ProtoReflect.Descriptor instead. +func (*CreateIndexRequest) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{10} +} + +func (x *CreateIndexRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *CreateIndexRequest) GetFieldID() int64 { + if x != nil { + return x.FieldID + } + return 0 +} + +func (x *CreateIndexRequest) GetIndexName() string { + if x != nil { + return x.IndexName + } + return "" +} + +func (x *CreateIndexRequest) GetTypeParams() []*commonpb.KeyValuePair { + if x != nil { + return x.TypeParams + } + return nil +} + +func (x *CreateIndexRequest) GetIndexParams() []*commonpb.KeyValuePair { + if x != nil { + return x.IndexParams + } + return nil +} + +func (x *CreateIndexRequest) GetTimestamp() uint64 { + if x != nil { + return x.Timestamp + } + return 0 +} + +func (x *CreateIndexRequest) GetIsAutoIndex() bool { + if x != nil { + return x.IsAutoIndex + } + return false +} + +func (x *CreateIndexRequest) GetUserIndexParams() []*commonpb.KeyValuePair { + if x != nil { + return x.UserIndexParams + } + return nil +} + +func (x *CreateIndexRequest) GetUserAutoindexMetricTypeSpecified() bool { + if x != nil { + return x.UserAutoindexMetricTypeSpecified + } + return false +} + +type AlterIndexRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + IndexName string `protobuf:"bytes,2,opt,name=index_name,json=indexName,proto3" json:"index_name,omitempty"` + Params []*commonpb.KeyValuePair `protobuf:"bytes,3,rep,name=params,proto3" json:"params,omitempty"` + DeleteKeys []string `protobuf:"bytes,4,rep,name=delete_keys,json=deleteKeys,proto3" json:"delete_keys,omitempty"` +} + +func (x *AlterIndexRequest) Reset() { + *x = AlterIndexRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AlterIndexRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AlterIndexRequest) ProtoMessage() {} + +func (x *AlterIndexRequest) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[11] + 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 AlterIndexRequest.ProtoReflect.Descriptor instead. +func (*AlterIndexRequest) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{11} +} + +func (x *AlterIndexRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *AlterIndexRequest) GetIndexName() string { + if x != nil { + return x.IndexName + } + return "" +} + +func (x *AlterIndexRequest) GetParams() []*commonpb.KeyValuePair { + if x != nil { + return x.Params + } + return nil +} + +func (x *AlterIndexRequest) GetDeleteKeys() []string { + if x != nil { + return x.DeleteKeys + } + return nil +} + +type GetIndexInfoRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + SegmentIDs []int64 `protobuf:"varint,2,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,omitempty"` + IndexName string `protobuf:"bytes,3,opt,name=index_name,json=indexName,proto3" json:"index_name,omitempty"` +} + +func (x *GetIndexInfoRequest) Reset() { + *x = GetIndexInfoRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetIndexInfoRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetIndexInfoRequest) ProtoMessage() {} + +func (x *GetIndexInfoRequest) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[12] + 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 GetIndexInfoRequest.ProtoReflect.Descriptor instead. +func (*GetIndexInfoRequest) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{12} +} + +func (x *GetIndexInfoRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *GetIndexInfoRequest) GetSegmentIDs() []int64 { + if x != nil { + return x.SegmentIDs + } + return nil +} + +func (x *GetIndexInfoRequest) GetIndexName() string { + if x != nil { + return x.IndexName + } + return "" +} + +type IndexFilePathInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SegmentID int64 `protobuf:"varint,1,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + FieldID int64 `protobuf:"varint,2,opt,name=fieldID,proto3" json:"fieldID,omitempty"` + IndexID int64 `protobuf:"varint,3,opt,name=indexID,proto3" json:"indexID,omitempty"` + BuildID int64 `protobuf:"varint,4,opt,name=buildID,proto3" json:"buildID,omitempty"` + IndexName string `protobuf:"bytes,5,opt,name=index_name,json=indexName,proto3" json:"index_name,omitempty"` + IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,6,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"` + IndexFilePaths []string `protobuf:"bytes,7,rep,name=index_file_paths,json=indexFilePaths,proto3" json:"index_file_paths,omitempty"` + SerializedSize uint64 `protobuf:"varint,8,opt,name=serialized_size,json=serializedSize,proto3" json:"serialized_size,omitempty"` + IndexVersion int64 `protobuf:"varint,9,opt,name=index_version,json=indexVersion,proto3" json:"index_version,omitempty"` + NumRows int64 `protobuf:"varint,10,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"` + CurrentIndexVersion int32 `protobuf:"varint,11,opt,name=current_index_version,json=currentIndexVersion,proto3" json:"current_index_version,omitempty"` +} + +func (x *IndexFilePathInfo) Reset() { + *x = IndexFilePathInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *IndexFilePathInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*IndexFilePathInfo) ProtoMessage() {} + +func (x *IndexFilePathInfo) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[13] + 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 IndexFilePathInfo.ProtoReflect.Descriptor instead. +func (*IndexFilePathInfo) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{13} +} + +func (x *IndexFilePathInfo) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *IndexFilePathInfo) GetFieldID() int64 { + if x != nil { + return x.FieldID + } + return 0 +} + +func (x *IndexFilePathInfo) GetIndexID() int64 { + if x != nil { + return x.IndexID + } + return 0 +} + +func (x *IndexFilePathInfo) GetBuildID() int64 { + if x != nil { + return x.BuildID + } + return 0 +} + +func (x *IndexFilePathInfo) GetIndexName() string { + if x != nil { + return x.IndexName + } + return "" +} + +func (x *IndexFilePathInfo) GetIndexParams() []*commonpb.KeyValuePair { + if x != nil { + return x.IndexParams + } + return nil +} + +func (x *IndexFilePathInfo) GetIndexFilePaths() []string { + if x != nil { + return x.IndexFilePaths + } + return nil +} + +func (x *IndexFilePathInfo) GetSerializedSize() uint64 { + if x != nil { + return x.SerializedSize + } + return 0 +} + +func (x *IndexFilePathInfo) GetIndexVersion() int64 { + if x != nil { + return x.IndexVersion + } + return 0 +} + +func (x *IndexFilePathInfo) GetNumRows() int64 { + if x != nil { + return x.NumRows + } + return 0 +} + +func (x *IndexFilePathInfo) GetCurrentIndexVersion() int32 { + if x != nil { + return x.CurrentIndexVersion + } + return 0 +} + +type SegmentInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + SegmentID int64 `protobuf:"varint,2,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + EnableIndex bool `protobuf:"varint,3,opt,name=enable_index,json=enableIndex,proto3" json:"enable_index,omitempty"` + IndexInfos []*IndexFilePathInfo `protobuf:"bytes,4,rep,name=index_infos,json=indexInfos,proto3" json:"index_infos,omitempty"` +} + +func (x *SegmentInfo) Reset() { + *x = SegmentInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentInfo) ProtoMessage() {} + +func (x *SegmentInfo) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[14] + 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 SegmentInfo.ProtoReflect.Descriptor instead. +func (*SegmentInfo) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{14} +} + +func (x *SegmentInfo) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *SegmentInfo) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *SegmentInfo) GetEnableIndex() bool { + if x != nil { + return x.EnableIndex + } + return false +} + +func (x *SegmentInfo) GetIndexInfos() []*IndexFilePathInfo { + if x != nil { + return x.IndexInfos + } + return nil +} + +type GetIndexInfoResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + SegmentInfo map[int64]*SegmentInfo `protobuf:"bytes,2,rep,name=segment_info,json=segmentInfo,proto3" json:"segment_info,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *GetIndexInfoResponse) Reset() { + *x = GetIndexInfoResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetIndexInfoResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetIndexInfoResponse) ProtoMessage() {} + +func (x *GetIndexInfoResponse) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[15] + 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 GetIndexInfoResponse.ProtoReflect.Descriptor instead. +func (*GetIndexInfoResponse) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{15} +} + +func (x *GetIndexInfoResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *GetIndexInfoResponse) GetSegmentInfo() map[int64]*SegmentInfo { + if x != nil { + return x.SegmentInfo + } + return nil +} + +type DropIndexRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionIDs []int64 `protobuf:"varint,2,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` + IndexName string `protobuf:"bytes,3,opt,name=index_name,json=indexName,proto3" json:"index_name,omitempty"` + DropAll bool `protobuf:"varint,4,opt,name=drop_all,json=dropAll,proto3" json:"drop_all,omitempty"` +} + +func (x *DropIndexRequest) Reset() { + *x = DropIndexRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DropIndexRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DropIndexRequest) ProtoMessage() {} + +func (x *DropIndexRequest) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[16] + 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 DropIndexRequest.ProtoReflect.Descriptor instead. +func (*DropIndexRequest) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{16} +} + +func (x *DropIndexRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *DropIndexRequest) GetPartitionIDs() []int64 { + if x != nil { + return x.PartitionIDs + } + return nil +} + +func (x *DropIndexRequest) GetIndexName() string { + if x != nil { + return x.IndexName + } + return "" +} + +func (x *DropIndexRequest) GetDropAll() bool { + if x != nil { + return x.DropAll + } + return false +} + +type DescribeIndexRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + IndexName string `protobuf:"bytes,2,opt,name=index_name,json=indexName,proto3" json:"index_name,omitempty"` + Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` +} + +func (x *DescribeIndexRequest) Reset() { + *x = DescribeIndexRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DescribeIndexRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DescribeIndexRequest) ProtoMessage() {} + +func (x *DescribeIndexRequest) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[17] + 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 DescribeIndexRequest.ProtoReflect.Descriptor instead. +func (*DescribeIndexRequest) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{17} +} + +func (x *DescribeIndexRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *DescribeIndexRequest) GetIndexName() string { + if x != nil { + return x.IndexName + } + return "" +} + +func (x *DescribeIndexRequest) GetTimestamp() uint64 { + if x != nil { + return x.Timestamp + } + return 0 +} + +type DescribeIndexResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + IndexInfos []*IndexInfo `protobuf:"bytes,2,rep,name=index_infos,json=indexInfos,proto3" json:"index_infos,omitempty"` +} + +func (x *DescribeIndexResponse) Reset() { + *x = DescribeIndexResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DescribeIndexResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DescribeIndexResponse) ProtoMessage() {} + +func (x *DescribeIndexResponse) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[18] + 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 DescribeIndexResponse.ProtoReflect.Descriptor instead. +func (*DescribeIndexResponse) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{18} +} + +func (x *DescribeIndexResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *DescribeIndexResponse) GetIndexInfos() []*IndexInfo { + if x != nil { + return x.IndexInfos + } + return nil +} + +type GetIndexBuildProgressRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + IndexName string `protobuf:"bytes,2,opt,name=index_name,json=indexName,proto3" json:"index_name,omitempty"` +} + +func (x *GetIndexBuildProgressRequest) Reset() { + *x = GetIndexBuildProgressRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[19] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetIndexBuildProgressRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetIndexBuildProgressRequest) ProtoMessage() {} + +func (x *GetIndexBuildProgressRequest) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[19] + 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 GetIndexBuildProgressRequest.ProtoReflect.Descriptor instead. +func (*GetIndexBuildProgressRequest) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{19} +} + +func (x *GetIndexBuildProgressRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *GetIndexBuildProgressRequest) GetIndexName() string { + if x != nil { + return x.IndexName + } + return "" +} + +type GetIndexBuildProgressResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + IndexedRows int64 `protobuf:"varint,2,opt,name=indexed_rows,json=indexedRows,proto3" json:"indexed_rows,omitempty"` + TotalRows int64 `protobuf:"varint,3,opt,name=total_rows,json=totalRows,proto3" json:"total_rows,omitempty"` + PendingIndexRows int64 `protobuf:"varint,4,opt,name=pending_index_rows,json=pendingIndexRows,proto3" json:"pending_index_rows,omitempty"` +} + +func (x *GetIndexBuildProgressResponse) Reset() { + *x = GetIndexBuildProgressResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[20] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetIndexBuildProgressResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetIndexBuildProgressResponse) ProtoMessage() {} + +func (x *GetIndexBuildProgressResponse) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[20] + 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 GetIndexBuildProgressResponse.ProtoReflect.Descriptor instead. +func (*GetIndexBuildProgressResponse) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{20} +} + +func (x *GetIndexBuildProgressResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *GetIndexBuildProgressResponse) GetIndexedRows() int64 { + if x != nil { + return x.IndexedRows + } + return 0 +} + +func (x *GetIndexBuildProgressResponse) GetTotalRows() int64 { + if x != nil { + return x.TotalRows + } + return 0 +} + +func (x *GetIndexBuildProgressResponse) GetPendingIndexRows() int64 { + if x != nil { + return x.PendingIndexRows + } + return 0 +} + +// Synchronously modify StorageConfig in index_cgo_msg.proto/clustering.proto file +type StorageConfig struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Address string `protobuf:"bytes,1,opt,name=address,proto3" json:"address,omitempty"` + AccessKeyID string `protobuf:"bytes,2,opt,name=access_keyID,json=accessKeyID,proto3" json:"access_keyID,omitempty"` + SecretAccessKey string `protobuf:"bytes,3,opt,name=secret_access_key,json=secretAccessKey,proto3" json:"secret_access_key,omitempty"` + UseSSL bool `protobuf:"varint,4,opt,name=useSSL,proto3" json:"useSSL,omitempty"` + BucketName string `protobuf:"bytes,5,opt,name=bucket_name,json=bucketName,proto3" json:"bucket_name,omitempty"` + RootPath string `protobuf:"bytes,6,opt,name=root_path,json=rootPath,proto3" json:"root_path,omitempty"` + UseIAM bool `protobuf:"varint,7,opt,name=useIAM,proto3" json:"useIAM,omitempty"` + IAMEndpoint string `protobuf:"bytes,8,opt,name=IAMEndpoint,proto3" json:"IAMEndpoint,omitempty"` + StorageType string `protobuf:"bytes,9,opt,name=storage_type,json=storageType,proto3" json:"storage_type,omitempty"` + UseVirtualHost bool `protobuf:"varint,10,opt,name=use_virtual_host,json=useVirtualHost,proto3" json:"use_virtual_host,omitempty"` + Region string `protobuf:"bytes,11,opt,name=region,proto3" json:"region,omitempty"` + CloudProvider string `protobuf:"bytes,12,opt,name=cloud_provider,json=cloudProvider,proto3" json:"cloud_provider,omitempty"` + RequestTimeoutMs int64 `protobuf:"varint,13,opt,name=request_timeout_ms,json=requestTimeoutMs,proto3" json:"request_timeout_ms,omitempty"` + SslCACert string `protobuf:"bytes,14,opt,name=sslCACert,proto3" json:"sslCACert,omitempty"` + GcpCredentialJSON string `protobuf:"bytes,15,opt,name=GcpCredentialJSON,proto3" json:"GcpCredentialJSON,omitempty"` +} + +func (x *StorageConfig) Reset() { + *x = StorageConfig{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[21] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StorageConfig) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StorageConfig) ProtoMessage() {} + +func (x *StorageConfig) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[21] + 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 StorageConfig.ProtoReflect.Descriptor instead. +func (*StorageConfig) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{21} +} + +func (x *StorageConfig) GetAddress() string { + if x != nil { + return x.Address + } + return "" +} + +func (x *StorageConfig) GetAccessKeyID() string { + if x != nil { + return x.AccessKeyID + } + return "" +} + +func (x *StorageConfig) GetSecretAccessKey() string { + if x != nil { + return x.SecretAccessKey + } + return "" +} + +func (x *StorageConfig) GetUseSSL() bool { + if x != nil { + return x.UseSSL + } + return false +} + +func (x *StorageConfig) GetBucketName() string { + if x != nil { + return x.BucketName + } + return "" +} + +func (x *StorageConfig) GetRootPath() string { + if x != nil { + return x.RootPath + } + return "" +} + +func (x *StorageConfig) GetUseIAM() bool { + if x != nil { + return x.UseIAM + } + return false +} + +func (x *StorageConfig) GetIAMEndpoint() string { + if x != nil { + return x.IAMEndpoint + } + return "" +} + +func (x *StorageConfig) GetStorageType() string { + if x != nil { + return x.StorageType + } + return "" +} + +func (x *StorageConfig) GetUseVirtualHost() bool { + if x != nil { + return x.UseVirtualHost + } + return false +} + +func (x *StorageConfig) GetRegion() string { + if x != nil { + return x.Region + } + return "" +} + +func (x *StorageConfig) GetCloudProvider() string { + if x != nil { + return x.CloudProvider + } + return "" +} + +func (x *StorageConfig) GetRequestTimeoutMs() int64 { + if x != nil { + return x.RequestTimeoutMs + } + return 0 +} + +func (x *StorageConfig) GetSslCACert() string { + if x != nil { + return x.SslCACert + } + return "" +} + +func (x *StorageConfig) GetGcpCredentialJSON() string { + if x != nil { + return x.GcpCredentialJSON + } + return "" +} + +// Synchronously modify OptionalFieldInfo in index_cgo_msg.proto file +type OptionalFieldInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + FieldID int64 `protobuf:"varint,1,opt,name=fieldID,proto3" json:"fieldID,omitempty"` + FieldName string `protobuf:"bytes,2,opt,name=field_name,json=fieldName,proto3" json:"field_name,omitempty"` + FieldType int32 `protobuf:"varint,3,opt,name=field_type,json=fieldType,proto3" json:"field_type,omitempty"` + DataPaths []string `protobuf:"bytes,4,rep,name=data_paths,json=dataPaths,proto3" json:"data_paths,omitempty"` + DataIds []int64 `protobuf:"varint,5,rep,packed,name=data_ids,json=dataIds,proto3" json:"data_ids,omitempty"` +} + +func (x *OptionalFieldInfo) Reset() { + *x = OptionalFieldInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[22] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *OptionalFieldInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*OptionalFieldInfo) ProtoMessage() {} + +func (x *OptionalFieldInfo) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[22] + 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 OptionalFieldInfo.ProtoReflect.Descriptor instead. +func (*OptionalFieldInfo) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{22} +} + +func (x *OptionalFieldInfo) GetFieldID() int64 { + if x != nil { + return x.FieldID + } + return 0 +} + +func (x *OptionalFieldInfo) GetFieldName() string { + if x != nil { + return x.FieldName + } + return "" +} + +func (x *OptionalFieldInfo) GetFieldType() int32 { + if x != nil { + return x.FieldType + } + return 0 +} + +func (x *OptionalFieldInfo) GetDataPaths() []string { + if x != nil { + return x.DataPaths + } + return nil +} + +func (x *OptionalFieldInfo) GetDataIds() []int64 { + if x != nil { + return x.DataIds + } + return nil +} + +type JobInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + NumRows int64 `protobuf:"varint,1,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"` + Dim int64 `protobuf:"varint,2,opt,name=dim,proto3" json:"dim,omitempty"` + StartTime int64 `protobuf:"varint,3,opt,name=start_time,json=startTime,proto3" json:"start_time,omitempty"` + EndTime int64 `protobuf:"varint,4,opt,name=end_time,json=endTime,proto3" json:"end_time,omitempty"` + IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,5,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"` + PodID int64 `protobuf:"varint,6,opt,name=podID,proto3" json:"podID,omitempty"` +} + +func (x *JobInfo) Reset() { + *x = JobInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[23] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *JobInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*JobInfo) ProtoMessage() {} + +func (x *JobInfo) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[23] + 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 JobInfo.ProtoReflect.Descriptor instead. +func (*JobInfo) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{23} +} + +func (x *JobInfo) GetNumRows() int64 { + if x != nil { + return x.NumRows + } + return 0 +} + +func (x *JobInfo) GetDim() int64 { + if x != nil { + return x.Dim + } + return 0 +} + +func (x *JobInfo) GetStartTime() int64 { + if x != nil { + return x.StartTime + } + return 0 +} + +func (x *JobInfo) GetEndTime() int64 { + if x != nil { + return x.EndTime + } + return 0 +} + +func (x *JobInfo) GetIndexParams() []*commonpb.KeyValuePair { + if x != nil { + return x.IndexParams + } + return nil +} + +func (x *JobInfo) GetPodID() int64 { + if x != nil { + return x.PodID + } + return 0 +} + +type GetIndexStatisticsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + IndexName string `protobuf:"bytes,2,opt,name=index_name,json=indexName,proto3" json:"index_name,omitempty"` +} + +func (x *GetIndexStatisticsRequest) Reset() { + *x = GetIndexStatisticsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[24] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetIndexStatisticsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetIndexStatisticsRequest) ProtoMessage() {} + +func (x *GetIndexStatisticsRequest) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[24] + 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 GetIndexStatisticsRequest.ProtoReflect.Descriptor instead. +func (*GetIndexStatisticsRequest) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{24} +} + +func (x *GetIndexStatisticsRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *GetIndexStatisticsRequest) GetIndexName() string { + if x != nil { + return x.IndexName + } + return "" +} + +type GetIndexStatisticsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + IndexInfos []*IndexInfo `protobuf:"bytes,2,rep,name=index_infos,json=indexInfos,proto3" json:"index_infos,omitempty"` +} + +func (x *GetIndexStatisticsResponse) Reset() { + *x = GetIndexStatisticsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[25] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetIndexStatisticsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetIndexStatisticsResponse) ProtoMessage() {} + +func (x *GetIndexStatisticsResponse) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[25] + 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 GetIndexStatisticsResponse.ProtoReflect.Descriptor instead. +func (*GetIndexStatisticsResponse) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{25} +} + +func (x *GetIndexStatisticsResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *GetIndexStatisticsResponse) GetIndexInfos() []*IndexInfo { + if x != nil { + return x.IndexInfos + } + return nil +} + +type ListIndexesRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` +} + +func (x *ListIndexesRequest) Reset() { + *x = ListIndexesRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[26] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListIndexesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListIndexesRequest) ProtoMessage() {} + +func (x *ListIndexesRequest) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[26] + 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 ListIndexesRequest.ProtoReflect.Descriptor instead. +func (*ListIndexesRequest) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{26} +} + +func (x *ListIndexesRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +type ListIndexesResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + IndexInfos []*IndexInfo `protobuf:"bytes,2,rep,name=index_infos,json=indexInfos,proto3" json:"index_infos,omitempty"` +} + +func (x *ListIndexesResponse) Reset() { + *x = ListIndexesResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[27] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListIndexesResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListIndexesResponse) ProtoMessage() {} + +func (x *ListIndexesResponse) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[27] + 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 ListIndexesResponse.ProtoReflect.Descriptor instead. +func (*ListIndexesResponse) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{27} +} + +func (x *ListIndexesResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *ListIndexesResponse) GetIndexInfos() []*IndexInfo { + if x != nil { + return x.IndexInfos + } + return nil +} + +type AnalyzeTask struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionID int64 `protobuf:"varint,2,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + FieldID int64 `protobuf:"varint,3,opt,name=fieldID,proto3" json:"fieldID,omitempty"` + FieldName string `protobuf:"bytes,4,opt,name=field_name,json=fieldName,proto3" json:"field_name,omitempty"` + FieldType schemapb.DataType `protobuf:"varint,5,opt,name=field_type,json=fieldType,proto3,enum=milvus.proto.schema.DataType" json:"field_type,omitempty"` + TaskID int64 `protobuf:"varint,6,opt,name=taskID,proto3" json:"taskID,omitempty"` + Version int64 `protobuf:"varint,7,opt,name=version,proto3" json:"version,omitempty"` + SegmentIDs []int64 `protobuf:"varint,8,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,omitempty"` + NodeID int64 `protobuf:"varint,9,opt,name=nodeID,proto3" json:"nodeID,omitempty"` + State JobState `protobuf:"varint,10,opt,name=state,proto3,enum=milvus.proto.index.JobState" json:"state,omitempty"` + FailReason string `protobuf:"bytes,11,opt,name=fail_reason,json=failReason,proto3" json:"fail_reason,omitempty"` + Dim int64 `protobuf:"varint,12,opt,name=dim,proto3" json:"dim,omitempty"` + CentroidsFile string `protobuf:"bytes,13,opt,name=centroids_file,json=centroidsFile,proto3" json:"centroids_file,omitempty"` +} + +func (x *AnalyzeTask) Reset() { + *x = AnalyzeTask{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[28] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AnalyzeTask) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AnalyzeTask) ProtoMessage() {} + +func (x *AnalyzeTask) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_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 AnalyzeTask.ProtoReflect.Descriptor instead. +func (*AnalyzeTask) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{28} +} + +func (x *AnalyzeTask) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *AnalyzeTask) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *AnalyzeTask) GetFieldID() int64 { + if x != nil { + return x.FieldID + } + return 0 +} + +func (x *AnalyzeTask) GetFieldName() string { + if x != nil { + return x.FieldName + } + return "" +} + +func (x *AnalyzeTask) GetFieldType() schemapb.DataType { + if x != nil { + return x.FieldType + } + return schemapb.DataType(0) +} + +func (x *AnalyzeTask) GetTaskID() int64 { + if x != nil { + return x.TaskID + } + return 0 +} + +func (x *AnalyzeTask) GetVersion() int64 { + if x != nil { + return x.Version + } + return 0 +} + +func (x *AnalyzeTask) GetSegmentIDs() []int64 { + if x != nil { + return x.SegmentIDs + } + return nil +} + +func (x *AnalyzeTask) GetNodeID() int64 { + if x != nil { + return x.NodeID + } + return 0 +} + +func (x *AnalyzeTask) GetState() JobState { + if x != nil { + return x.State + } + return JobState_JobStateNone +} + +func (x *AnalyzeTask) GetFailReason() string { + if x != nil { + return x.FailReason + } + return "" +} + +func (x *AnalyzeTask) GetDim() int64 { + if x != nil { + return x.Dim + } + return 0 +} + +func (x *AnalyzeTask) GetCentroidsFile() string { + if x != nil { + return x.CentroidsFile + } + return "" +} + +type SegmentStats struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ID int64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"` + NumRows int64 `protobuf:"varint,2,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"` + LogIDs []int64 `protobuf:"varint,3,rep,packed,name=logIDs,proto3" json:"logIDs,omitempty"` +} + +func (x *SegmentStats) Reset() { + *x = SegmentStats{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[29] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentStats) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentStats) ProtoMessage() {} + +func (x *SegmentStats) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_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 SegmentStats.ProtoReflect.Descriptor instead. +func (*SegmentStats) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{29} +} + +func (x *SegmentStats) GetID() int64 { + if x != nil { + return x.ID + } + return 0 +} + +func (x *SegmentStats) GetNumRows() int64 { + if x != nil { + return x.NumRows + } + return 0 +} + +func (x *SegmentStats) GetLogIDs() []int64 { + if x != nil { + return x.LogIDs + } + return nil +} + +type FieldLogPath struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + FieldID int64 `protobuf:"varint,1,opt,name=fieldID,proto3" json:"fieldID,omitempty"` + FilePaths []string `protobuf:"bytes,2,rep,name=file_paths,json=filePaths,proto3" json:"file_paths,omitempty"` +} + +func (x *FieldLogPath) Reset() { + *x = FieldLogPath{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[30] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *FieldLogPath) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FieldLogPath) ProtoMessage() {} + +func (x *FieldLogPath) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[30] + 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 FieldLogPath.ProtoReflect.Descriptor instead. +func (*FieldLogPath) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{30} +} + +func (x *FieldLogPath) GetFieldID() int64 { + if x != nil { + return x.FieldID + } + return 0 +} + +func (x *FieldLogPath) GetFilePaths() []string { + if x != nil { + return x.FilePaths + } + return nil +} + +type StatsTask struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionID int64 `protobuf:"varint,2,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + SegmentID int64 `protobuf:"varint,3,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + InsertChannel string `protobuf:"bytes,4,opt,name=insert_channel,json=insertChannel,proto3" json:"insert_channel,omitempty"` + TaskID int64 `protobuf:"varint,5,opt,name=taskID,proto3" json:"taskID,omitempty"` + Version int64 `protobuf:"varint,6,opt,name=version,proto3" json:"version,omitempty"` + NodeID int64 `protobuf:"varint,7,opt,name=nodeID,proto3" json:"nodeID,omitempty"` + State JobState `protobuf:"varint,8,opt,name=state,proto3,enum=milvus.proto.index.JobState" json:"state,omitempty"` + FailReason string `protobuf:"bytes,9,opt,name=fail_reason,json=failReason,proto3" json:"fail_reason,omitempty"` + TargetSegmentID int64 `protobuf:"varint,10,opt,name=target_segmentID,json=targetSegmentID,proto3" json:"target_segmentID,omitempty"` + SubJobType StatsSubJob `protobuf:"varint,11,opt,name=subJobType,proto3,enum=milvus.proto.index.StatsSubJob" json:"subJobType,omitempty"` + CanRecycle bool `protobuf:"varint,12,opt,name=canRecycle,proto3" json:"canRecycle,omitempty"` +} + +func (x *StatsTask) Reset() { + *x = StatsTask{} + if protoimpl.UnsafeEnabled { + mi := &file_index_coord_proto_msgTypes[31] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StatsTask) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StatsTask) ProtoMessage() {} + +func (x *StatsTask) ProtoReflect() protoreflect.Message { + mi := &file_index_coord_proto_msgTypes[31] + 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 StatsTask.ProtoReflect.Descriptor instead. +func (*StatsTask) Descriptor() ([]byte, []int) { + return file_index_coord_proto_rawDescGZIP(), []int{31} +} + +func (x *StatsTask) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *StatsTask) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *StatsTask) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *StatsTask) GetInsertChannel() string { + if x != nil { + return x.InsertChannel + } + return "" +} + +func (x *StatsTask) GetTaskID() int64 { + if x != nil { + return x.TaskID + } + return 0 +} + +func (x *StatsTask) GetVersion() int64 { + if x != nil { + return x.Version + } + return 0 +} + +func (x *StatsTask) GetNodeID() int64 { + if x != nil { + return x.NodeID + } + return 0 +} + +func (x *StatsTask) GetState() JobState { + if x != nil { + return x.State + } + return JobState_JobStateNone +} + +func (x *StatsTask) GetFailReason() string { + if x != nil { + return x.FailReason + } + return "" +} + +func (x *StatsTask) GetTargetSegmentID() int64 { + if x != nil { + return x.TargetSegmentID + } + return 0 +} + +func (x *StatsTask) GetSubJobType() StatsSubJob { + if x != nil { + return x.SubJobType + } + return StatsSubJob_None +} + +func (x *StatsTask) GetCanRecycle() bool { + if x != nil { + return x.CanRecycle + } + return false +} + +var File_index_coord_proto protoreflect.FileDescriptor + +var file_index_coord_proto_rawDesc = []byte{ + 0x0a, 0x11, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x12, 0x12, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x1a, 0x0c, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x0e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x0c, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x1a, 0x0c, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x22, 0xdd, 0x04, 0x0a, 0x09, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x12, + 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x1d, 0x0a, + 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x12, 0x42, 0x0a, 0x0b, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x70, + 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0a, + 0x74, 0x79, 0x70, 0x65, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x44, 0x0a, 0x0c, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, + 0x61, 0x69, 0x72, 0x52, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, + 0x12, 0x21, 0x0a, 0x0c, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x64, 0x5f, 0x72, 0x6f, 0x77, 0x73, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x64, 0x52, + 0x6f, 0x77, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x72, 0x6f, 0x77, + 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x52, 0x6f, + 0x77, 0x73, 0x12, 0x35, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, + 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x35, 0x0a, 0x17, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72, 0x65, + 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x14, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x46, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x61, 0x73, 0x6f, 0x6e, + 0x12, 0x22, 0x0a, 0x0d, 0x69, 0x73, 0x5f, 0x61, 0x75, 0x74, 0x6f, 0x5f, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x69, 0x73, 0x41, 0x75, 0x74, 0x6f, 0x49, + 0x6e, 0x64, 0x65, 0x78, 0x12, 0x4d, 0x0a, 0x11, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x0c, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, + 0x69, 0x72, 0x52, 0x0f, 0x75, 0x73, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x50, 0x61, 0x72, + 0x61, 0x6d, 0x73, 0x12, 0x2c, 0x0a, 0x12, 0x70, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x5f, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x10, 0x70, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x6f, 0x77, + 0x73, 0x22, 0x85, 0x01, 0x0a, 0x0a, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, + 0x12, 0x3c, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, + 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x18, + 0x0a, 0x07, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x07, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x63, + 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x22, 0x8e, 0x05, 0x0a, 0x0c, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, + 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, + 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x19, + 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x49, 0x44, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x06, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x16, 0x0a, + 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, + 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x76, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x35, 0x0a, 0x05, 0x73, 0x74, + 0x61, 0x74, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, + 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, + 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x61, 0x73, + 0x6f, 0x6e, 0x12, 0x26, 0x0a, 0x0f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x66, 0x69, 0x6c, 0x65, + 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x0b, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x46, 0x69, 0x6c, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x64, 0x65, + 0x6c, 0x65, 0x74, 0x65, 0x64, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x64, 0x65, 0x6c, + 0x65, 0x74, 0x65, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x74, + 0x69, 0x6d, 0x65, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, + 0x7a, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x73, + 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x23, 0x0a, 0x0d, + 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x68, 0x61, 0x6e, 0x64, 0x6f, 0x66, 0x66, 0x18, 0x0f, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x0c, 0x77, 0x72, 0x69, 0x74, 0x65, 0x48, 0x61, 0x6e, 0x64, 0x6f, 0x66, + 0x66, 0x12, 0x32, 0x0a, 0x15, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x10, 0x20, 0x01, 0x28, 0x05, + 0x52, 0x13, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x56, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x2e, 0x0a, 0x13, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x73, + 0x74, 0x6f, 0x72, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x11, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x11, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x6f, 0x72, 0x65, 0x56, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0d, 0x66, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, + 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x12, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0c, 0x66, 0x69, + 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x22, 0x97, 0x01, 0x0a, 0x13, 0x52, + 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, 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, 0x36, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, + 0x02, 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, 0x41, 0x64, 0x64, 0x72, + 0x65, 0x73, 0x73, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x16, 0x0a, 0x06, + 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, + 0x64, 0x65, 0x49, 0x44, 0x22, 0x8f, 0x01, 0x0a, 0x14, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, + 0x72, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, + 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x12, 0x42, 0x0a, 0x0b, 0x69, 0x6e, 0x69, 0x74, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, + 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, + 0x49, 0x6e, 0x69, 0x74, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x52, 0x0a, 0x69, 0x6e, 0x69, 0x74, + 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x22, 0x59, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, + 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x22, + 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, + 0x65, 0x22, 0xa4, 0x01, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x12, 0x35, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, 0x5f, + 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, + 0x69, 0x6c, 0x52, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x22, 0x80, 0x01, 0x0a, 0x1b, 0x47, 0x65, 0x74, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, + 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x73, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, + 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x22, 0xa8, 0x01, 0x0a, 0x11, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, + 0x35, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, + 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72, + 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, 0x69, + 0x6c, 0x52, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x92, 0x01, 0x0a, 0x1c, 0x47, 0x65, 0x74, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x3d, 0x0a, 0x06, + 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x22, 0xdc, 0x03, 0x0a, 0x12, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, + 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, + 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x12, + 0x42, 0x0a, 0x0b, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x04, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, + 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0a, 0x74, 0x79, 0x70, 0x65, 0x50, 0x61, 0x72, + 0x61, 0x6d, 0x73, 0x12, 0x44, 0x0a, 0x0c, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x70, 0x61, 0x72, + 0x61, 0x6d, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0b, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x74, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x22, 0x0a, 0x0d, 0x69, 0x73, 0x5f, 0x61, 0x75, + 0x74, 0x6f, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, + 0x69, 0x73, 0x41, 0x75, 0x74, 0x6f, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x4d, 0x0a, 0x11, 0x75, + 0x73, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, + 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, + 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0f, 0x75, 0x73, 0x65, 0x72, 0x49, + 0x6e, 0x64, 0x65, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x4e, 0x0a, 0x24, 0x75, 0x73, + 0x65, 0x72, 0x5f, 0x61, 0x75, 0x74, 0x6f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6d, 0x65, 0x74, + 0x72, 0x69, 0x63, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x69, 0x66, 0x69, + 0x65, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x08, 0x52, 0x20, 0x75, 0x73, 0x65, 0x72, 0x41, 0x75, + 0x74, 0x6f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x54, 0x79, 0x70, + 0x65, 0x53, 0x70, 0x65, 0x63, 0x69, 0x66, 0x69, 0x65, 0x64, 0x22, 0xb2, 0x01, 0x0a, 0x11, 0x41, + 0x6c, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, + 0x61, 0x6d, 0x65, 0x12, 0x39, 0x0a, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x03, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, + 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x1f, + 0x0a, 0x0b, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x04, 0x20, + 0x03, 0x28, 0x09, 0x52, 0x0a, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x22, + 0x78, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, + 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xab, 0x03, 0x0a, 0x11, 0x49, 0x6e, + 0x64, 0x65, 0x78, 0x46, 0x69, 0x6c, 0x65, 0x50, 0x61, 0x74, 0x68, 0x49, 0x6e, 0x66, 0x6f, 0x12, + 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x18, 0x0a, + 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, + 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, + 0x44, 0x12, 0x18, 0x0a, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x44, 0x0a, 0x0c, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, + 0x61, 0x69, 0x72, 0x52, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, + 0x12, 0x28, 0x0a, 0x10, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x70, + 0x61, 0x74, 0x68, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x46, 0x69, 0x6c, 0x65, 0x50, 0x61, 0x74, 0x68, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x65, + 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x08, 0x20, + 0x01, 0x28, 0x04, 0x52, 0x0e, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x53, + 0x69, 0x7a, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x76, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, + 0x72, 0x6f, 0x77, 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, + 0x6f, 0x77, 0x73, 0x12, 0x32, 0x0a, 0x15, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x0b, 0x20, 0x01, + 0x28, 0x05, 0x52, 0x13, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, + 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0xba, 0x01, 0x0a, 0x0b, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x73, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, + 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x21, 0x0a, 0x0c, 0x65, 0x6e, 0x61, + 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x0b, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x46, 0x0a, 0x0b, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x46, 0x69, 0x6c, 0x65, + 0x50, 0x61, 0x74, 0x68, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, + 0x6e, 0x66, 0x6f, 0x73, 0x22, 0x8a, 0x02, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, + 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, + 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x12, 0x5c, 0x0a, 0x0c, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x6e, + 0x66, 0x6f, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, + 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x0b, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, + 0x1a, 0x5f, 0x0a, 0x10, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x35, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x22, 0x94, 0x01, 0x0a, 0x10, 0x44, 0x72, 0x6f, 0x70, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x70, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, + 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x12, 0x1d, + 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x19, 0x0a, + 0x08, 0x64, 0x72, 0x6f, 0x70, 0x5f, 0x61, 0x6c, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x07, 0x64, 0x72, 0x6f, 0x70, 0x41, 0x6c, 0x6c, 0x22, 0x77, 0x0a, 0x14, 0x44, 0x65, 0x73, 0x63, + 0x72, 0x69, 0x62, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, + 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x22, 0x8c, 0x01, 0x0a, 0x15, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x49, 0x6e, + 0x64, 0x65, 0x78, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x12, 0x3e, 0x0a, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, + 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, + 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x73, + 0x22, 0x61, 0x0a, 0x1c, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, + 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, + 0x61, 0x6d, 0x65, 0x22, 0xc4, 0x01, 0x0a, 0x1d, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, + 0x42, 0x75, 0x69, 0x6c, 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x65, 0x64, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x64, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x1d, 0x0a, + 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x09, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x2c, 0x0a, 0x12, + 0x70, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x72, 0x6f, + 0x77, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x70, 0x65, 0x6e, 0x64, 0x69, 0x6e, + 0x67, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x6f, 0x77, 0x73, 0x22, 0x8e, 0x04, 0x0a, 0x0d, 0x53, + 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x18, 0x0a, 0x07, + 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, + 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, + 0x5f, 0x6b, 0x65, 0x79, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x61, 0x63, + 0x63, 0x65, 0x73, 0x73, 0x4b, 0x65, 0x79, 0x49, 0x44, 0x12, 0x2a, 0x0a, 0x11, 0x73, 0x65, 0x63, + 0x72, 0x65, 0x74, 0x5f, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, 0x41, 0x63, 0x63, 0x65, + 0x73, 0x73, 0x4b, 0x65, 0x79, 0x12, 0x16, 0x0a, 0x06, 0x75, 0x73, 0x65, 0x53, 0x53, 0x4c, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x75, 0x73, 0x65, 0x53, 0x53, 0x4c, 0x12, 0x1f, 0x0a, + 0x0b, 0x62, 0x75, 0x63, 0x6b, 0x65, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0a, 0x62, 0x75, 0x63, 0x6b, 0x65, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1b, + 0x0a, 0x09, 0x72, 0x6f, 0x6f, 0x74, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x06, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x08, 0x72, 0x6f, 0x6f, 0x74, 0x50, 0x61, 0x74, 0x68, 0x12, 0x16, 0x0a, 0x06, 0x75, + 0x73, 0x65, 0x49, 0x41, 0x4d, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x75, 0x73, 0x65, + 0x49, 0x41, 0x4d, 0x12, 0x20, 0x0a, 0x0b, 0x49, 0x41, 0x4d, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, + 0x6e, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x49, 0x41, 0x4d, 0x45, 0x6e, 0x64, + 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, + 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x74, 0x6f, + 0x72, 0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x28, 0x0a, 0x10, 0x75, 0x73, 0x65, 0x5f, + 0x76, 0x69, 0x72, 0x74, 0x75, 0x61, 0x6c, 0x5f, 0x68, 0x6f, 0x73, 0x74, 0x18, 0x0a, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0e, 0x75, 0x73, 0x65, 0x56, 0x69, 0x72, 0x74, 0x75, 0x61, 0x6c, 0x48, 0x6f, + 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x18, 0x0b, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6c, + 0x6f, 0x75, 0x64, 0x5f, 0x70, 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x18, 0x0c, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0d, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x50, 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, + 0x72, 0x12, 0x2c, 0x0a, 0x12, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x74, 0x69, 0x6d, + 0x65, 0x6f, 0x75, 0x74, 0x5f, 0x6d, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x72, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4d, 0x73, 0x12, + 0x1c, 0x0a, 0x09, 0x73, 0x73, 0x6c, 0x43, 0x41, 0x43, 0x65, 0x72, 0x74, 0x18, 0x0e, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x09, 0x73, 0x73, 0x6c, 0x43, 0x41, 0x43, 0x65, 0x72, 0x74, 0x12, 0x2c, 0x0a, + 0x11, 0x47, 0x63, 0x70, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x4a, 0x53, + 0x4f, 0x4e, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x47, 0x63, 0x70, 0x43, 0x72, 0x65, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x4a, 0x53, 0x4f, 0x4e, 0x22, 0xa5, 0x01, 0x0a, 0x11, + 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x66, + 0x6f, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x66, + 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x09, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x66, 0x69, + 0x65, 0x6c, 0x64, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, + 0x66, 0x69, 0x65, 0x6c, 0x64, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x64, 0x61, 0x74, + 0x61, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x64, + 0x61, 0x74, 0x61, 0x50, 0x61, 0x74, 0x68, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x64, 0x61, 0x74, 0x61, + 0x5f, 0x69, 0x64, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x03, 0x52, 0x07, 0x64, 0x61, 0x74, 0x61, + 0x49, 0x64, 0x73, 0x22, 0xcc, 0x01, 0x0a, 0x07, 0x4a, 0x6f, 0x62, 0x49, 0x6e, 0x66, 0x6f, 0x12, + 0x19, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x10, 0x0a, 0x03, 0x64, 0x69, + 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x64, 0x69, 0x6d, 0x12, 0x1d, 0x0a, 0x0a, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x65, + 0x6e, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x65, + 0x6e, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x44, 0x0a, 0x0c, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, + 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, + 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x14, 0x0a, 0x05, + 0x70, 0x6f, 0x64, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x70, 0x6f, 0x64, + 0x49, 0x44, 0x22, 0x5e, 0x0a, 0x19, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, + 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, + 0x6d, 0x65, 0x22, 0x91, 0x01, 0x0a, 0x1a, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, + 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, + 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x3e, 0x0a, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, + 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x22, 0x38, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6e, + 0x64, 0x65, 0x78, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x22, 0x0a, 0x0c, + 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, + 0x22, 0x8a, 0x01, 0x0a, 0x13, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x3e, 0x0a, + 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x02, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, + 0x6f, 0x52, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x22, 0xc2, 0x03, + 0x0a, 0x0b, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x22, 0x0a, + 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x1d, 0x0a, + 0x0a, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x09, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3c, 0x0a, 0x0a, + 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x52, + 0x09, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x54, 0x79, 0x70, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, + 0x73, 0x6b, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, + 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, + 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x03, + 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, 0x16, 0x0a, 0x06, + 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, + 0x64, 0x65, 0x49, 0x44, 0x12, 0x32, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x0a, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, + 0x5f, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, + 0x61, 0x69, 0x6c, 0x52, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x64, 0x69, 0x6d, + 0x18, 0x0c, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x64, 0x69, 0x6d, 0x12, 0x25, 0x0a, 0x0e, 0x63, + 0x65, 0x6e, 0x74, 0x72, 0x6f, 0x69, 0x64, 0x73, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x18, 0x0d, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x65, 0x6e, 0x74, 0x72, 0x6f, 0x69, 0x64, 0x73, 0x46, 0x69, + 0x6c, 0x65, 0x22, 0x51, 0x0a, 0x0c, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, + 0x74, 0x73, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, + 0x49, 0x44, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x16, 0x0a, + 0x06, 0x6c, 0x6f, 0x67, 0x49, 0x44, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x06, 0x6c, + 0x6f, 0x67, 0x49, 0x44, 0x73, 0x22, 0x47, 0x0a, 0x0c, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x4c, 0x6f, + 0x67, 0x50, 0x61, 0x74, 0x68, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, + 0x1d, 0x0a, 0x0a, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x73, 0x18, 0x02, 0x20, + 0x03, 0x28, 0x09, 0x52, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x50, 0x61, 0x74, 0x68, 0x73, 0x22, 0xc1, + 0x03, 0x0a, 0x09, 0x53, 0x74, 0x61, 0x74, 0x73, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x22, 0x0a, 0x0c, + 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, + 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, + 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, + 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, + 0x44, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, + 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, + 0x65, 0x49, 0x44, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, + 0x44, 0x12, 0x32, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, + 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72, 0x65, + 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, 0x69, 0x6c, + 0x52, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, + 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x0f, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, + 0x44, 0x12, 0x3f, 0x0a, 0x0a, 0x73, 0x75, 0x62, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x18, + 0x0b, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x73, + 0x53, 0x75, 0x62, 0x4a, 0x6f, 0x62, 0x52, 0x0a, 0x73, 0x75, 0x62, 0x4a, 0x6f, 0x62, 0x54, 0x79, + 0x70, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x61, 0x6e, 0x52, 0x65, 0x63, 0x79, 0x63, 0x6c, 0x65, + 0x18, 0x0c, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x63, 0x61, 0x6e, 0x52, 0x65, 0x63, 0x79, 0x63, + 0x6c, 0x65, 0x2a, 0x5b, 0x0a, 0x07, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0f, 0x0a, + 0x0b, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x13, + 0x0a, 0x0f, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, + 0x62, 0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x41, 0x6e, + 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x4a, 0x6f, 0x62, 0x10, 0x02, 0x12, 0x13, 0x0a, 0x0f, 0x4a, 0x6f, + 0x62, 0x54, 0x79, 0x70, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4a, 0x6f, 0x62, 0x10, 0x03, 0x2a, + 0x83, 0x01, 0x0a, 0x08, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x10, 0x0a, 0x0c, + 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x10, + 0x0a, 0x0c, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x69, 0x74, 0x10, 0x01, + 0x12, 0x16, 0x0a, 0x12, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x50, 0x72, + 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x10, 0x02, 0x12, 0x14, 0x0a, 0x10, 0x4a, 0x6f, 0x62, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x46, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x10, 0x03, 0x12, 0x12, + 0x0a, 0x0e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, + 0x10, 0x04, 0x12, 0x11, 0x0a, 0x0d, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, + 0x74, 0x72, 0x79, 0x10, 0x05, 0x2a, 0x40, 0x0a, 0x0b, 0x53, 0x74, 0x61, 0x74, 0x73, 0x53, 0x75, + 0x62, 0x4a, 0x6f, 0x62, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x08, + 0x0a, 0x04, 0x53, 0x6f, 0x72, 0x74, 0x10, 0x01, 0x12, 0x10, 0x0a, 0x0c, 0x54, 0x65, 0x78, 0x74, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62, 0x10, 0x02, 0x12, 0x0b, 0x0a, 0x07, 0x42, 0x4d, + 0x32, 0x35, 0x4a, 0x6f, 0x62, 0x10, 0x03, 0x32, 0xd5, 0x0b, 0x0a, 0x0a, 0x49, 0x6e, 0x64, 0x65, + 0x78, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x12, 0x6c, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, + 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x2e, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x73, 0x22, 0x00, 0x12, 0x71, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, + 0x73, 0x74, 0x69, 0x63, 0x73, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x32, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, + 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, + 0x63, 0x73, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x54, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x52, 0x0a, + 0x0a, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x25, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x2e, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, + 0x00, 0x12, 0x66, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7b, 0x0a, 0x14, 0x47, 0x65, 0x74, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x64, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, + 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, + 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x50, 0x0a, 0x09, + 0x44, 0x72, 0x6f, 0x70, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, + 0x72, 0x6f, 0x70, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x66, + 0x0a, 0x0d, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, + 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x49, 0x6e, 0x64, + 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, + 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x75, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, + 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, 0x2d, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, + 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, + 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7e, 0x0a, + 0x15, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x50, 0x72, + 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, + 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, + 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, + 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 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, 0x43, + 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, + 0x65, 0x61, 0x6c, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, + 0x2f, 0x5a, 0x2d, 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, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x70, 0x62, + 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_index_coord_proto_rawDescOnce sync.Once + file_index_coord_proto_rawDescData = file_index_coord_proto_rawDesc +) + +func file_index_coord_proto_rawDescGZIP() []byte { + file_index_coord_proto_rawDescOnce.Do(func() { + file_index_coord_proto_rawDescData = protoimpl.X.CompressGZIP(file_index_coord_proto_rawDescData) + }) + return file_index_coord_proto_rawDescData +} + +var file_index_coord_proto_enumTypes = make([]protoimpl.EnumInfo, 3) +var file_index_coord_proto_msgTypes = make([]protoimpl.MessageInfo, 33) +var file_index_coord_proto_goTypes = []interface{}{ + (JobType)(0), // 0: milvus.proto.index.JobType + (JobState)(0), // 1: milvus.proto.index.JobState + (StatsSubJob)(0), // 2: milvus.proto.index.StatsSubJob + (*IndexInfo)(nil), // 3: milvus.proto.index.IndexInfo + (*FieldIndex)(nil), // 4: milvus.proto.index.FieldIndex + (*SegmentIndex)(nil), // 5: milvus.proto.index.SegmentIndex + (*RegisterNodeRequest)(nil), // 6: milvus.proto.index.RegisterNodeRequest + (*RegisterNodeResponse)(nil), // 7: milvus.proto.index.RegisterNodeResponse + (*GetIndexStateRequest)(nil), // 8: milvus.proto.index.GetIndexStateRequest + (*GetIndexStateResponse)(nil), // 9: milvus.proto.index.GetIndexStateResponse + (*GetSegmentIndexStateRequest)(nil), // 10: milvus.proto.index.GetSegmentIndexStateRequest + (*SegmentIndexState)(nil), // 11: milvus.proto.index.SegmentIndexState + (*GetSegmentIndexStateResponse)(nil), // 12: milvus.proto.index.GetSegmentIndexStateResponse + (*CreateIndexRequest)(nil), // 13: milvus.proto.index.CreateIndexRequest + (*AlterIndexRequest)(nil), // 14: milvus.proto.index.AlterIndexRequest + (*GetIndexInfoRequest)(nil), // 15: milvus.proto.index.GetIndexInfoRequest + (*IndexFilePathInfo)(nil), // 16: milvus.proto.index.IndexFilePathInfo + (*SegmentInfo)(nil), // 17: milvus.proto.index.SegmentInfo + (*GetIndexInfoResponse)(nil), // 18: milvus.proto.index.GetIndexInfoResponse + (*DropIndexRequest)(nil), // 19: milvus.proto.index.DropIndexRequest + (*DescribeIndexRequest)(nil), // 20: milvus.proto.index.DescribeIndexRequest + (*DescribeIndexResponse)(nil), // 21: milvus.proto.index.DescribeIndexResponse + (*GetIndexBuildProgressRequest)(nil), // 22: milvus.proto.index.GetIndexBuildProgressRequest + (*GetIndexBuildProgressResponse)(nil), // 23: milvus.proto.index.GetIndexBuildProgressResponse + (*StorageConfig)(nil), // 24: milvus.proto.index.StorageConfig + (*OptionalFieldInfo)(nil), // 25: milvus.proto.index.OptionalFieldInfo + (*JobInfo)(nil), // 26: milvus.proto.index.JobInfo + (*GetIndexStatisticsRequest)(nil), // 27: milvus.proto.index.GetIndexStatisticsRequest + (*GetIndexStatisticsResponse)(nil), // 28: milvus.proto.index.GetIndexStatisticsResponse + (*ListIndexesRequest)(nil), // 29: milvus.proto.index.ListIndexesRequest + (*ListIndexesResponse)(nil), // 30: milvus.proto.index.ListIndexesResponse + (*AnalyzeTask)(nil), // 31: milvus.proto.index.AnalyzeTask + (*SegmentStats)(nil), // 32: milvus.proto.index.SegmentStats + (*FieldLogPath)(nil), // 33: milvus.proto.index.FieldLogPath + (*StatsTask)(nil), // 34: milvus.proto.index.StatsTask + nil, // 35: milvus.proto.index.GetIndexInfoResponse.SegmentInfoEntry + (*commonpb.KeyValuePair)(nil), // 36: milvus.proto.common.KeyValuePair + (commonpb.IndexState)(0), // 37: milvus.proto.common.IndexState + (*commonpb.MsgBase)(nil), // 38: milvus.proto.common.MsgBase + (*commonpb.Address)(nil), // 39: milvus.proto.common.Address + (*commonpb.Status)(nil), // 40: milvus.proto.common.Status + (*internalpb.InitParams)(nil), // 41: milvus.proto.internal.InitParams + (schemapb.DataType)(0), // 42: milvus.proto.schema.DataType + (*milvuspb.GetComponentStatesRequest)(nil), // 43: milvus.proto.milvus.GetComponentStatesRequest + (*internalpb.GetStatisticsChannelRequest)(nil), // 44: milvus.proto.internal.GetStatisticsChannelRequest + (*internalpb.ShowConfigurationsRequest)(nil), // 45: milvus.proto.internal.ShowConfigurationsRequest + (*milvuspb.GetMetricsRequest)(nil), // 46: milvus.proto.milvus.GetMetricsRequest + (*milvuspb.CheckHealthRequest)(nil), // 47: milvus.proto.milvus.CheckHealthRequest + (*milvuspb.ComponentStates)(nil), // 48: milvus.proto.milvus.ComponentStates + (*milvuspb.StringResponse)(nil), // 49: milvus.proto.milvus.StringResponse + (*internalpb.ShowConfigurationsResponse)(nil), // 50: milvus.proto.internal.ShowConfigurationsResponse + (*milvuspb.GetMetricsResponse)(nil), // 51: milvus.proto.milvus.GetMetricsResponse + (*milvuspb.CheckHealthResponse)(nil), // 52: milvus.proto.milvus.CheckHealthResponse +} +var file_index_coord_proto_depIdxs = []int32{ + 36, // 0: milvus.proto.index.IndexInfo.type_params:type_name -> milvus.proto.common.KeyValuePair + 36, // 1: milvus.proto.index.IndexInfo.index_params:type_name -> milvus.proto.common.KeyValuePair + 37, // 2: milvus.proto.index.IndexInfo.state:type_name -> milvus.proto.common.IndexState + 36, // 3: milvus.proto.index.IndexInfo.user_index_params:type_name -> milvus.proto.common.KeyValuePair + 3, // 4: milvus.proto.index.FieldIndex.index_info:type_name -> milvus.proto.index.IndexInfo + 37, // 5: milvus.proto.index.SegmentIndex.state:type_name -> milvus.proto.common.IndexState + 38, // 6: milvus.proto.index.RegisterNodeRequest.base:type_name -> milvus.proto.common.MsgBase + 39, // 7: milvus.proto.index.RegisterNodeRequest.address:type_name -> milvus.proto.common.Address + 40, // 8: milvus.proto.index.RegisterNodeResponse.status:type_name -> milvus.proto.common.Status + 41, // 9: milvus.proto.index.RegisterNodeResponse.init_params:type_name -> milvus.proto.internal.InitParams + 40, // 10: milvus.proto.index.GetIndexStateResponse.status:type_name -> milvus.proto.common.Status + 37, // 11: milvus.proto.index.GetIndexStateResponse.state:type_name -> milvus.proto.common.IndexState + 37, // 12: milvus.proto.index.SegmentIndexState.state:type_name -> milvus.proto.common.IndexState + 40, // 13: milvus.proto.index.GetSegmentIndexStateResponse.status:type_name -> milvus.proto.common.Status + 11, // 14: milvus.proto.index.GetSegmentIndexStateResponse.states:type_name -> milvus.proto.index.SegmentIndexState + 36, // 15: milvus.proto.index.CreateIndexRequest.type_params:type_name -> milvus.proto.common.KeyValuePair + 36, // 16: milvus.proto.index.CreateIndexRequest.index_params:type_name -> milvus.proto.common.KeyValuePair + 36, // 17: milvus.proto.index.CreateIndexRequest.user_index_params:type_name -> milvus.proto.common.KeyValuePair + 36, // 18: milvus.proto.index.AlterIndexRequest.params:type_name -> milvus.proto.common.KeyValuePair + 36, // 19: milvus.proto.index.IndexFilePathInfo.index_params:type_name -> milvus.proto.common.KeyValuePair + 16, // 20: milvus.proto.index.SegmentInfo.index_infos:type_name -> milvus.proto.index.IndexFilePathInfo + 40, // 21: milvus.proto.index.GetIndexInfoResponse.status:type_name -> milvus.proto.common.Status + 35, // 22: milvus.proto.index.GetIndexInfoResponse.segment_info:type_name -> milvus.proto.index.GetIndexInfoResponse.SegmentInfoEntry + 40, // 23: milvus.proto.index.DescribeIndexResponse.status:type_name -> milvus.proto.common.Status + 3, // 24: milvus.proto.index.DescribeIndexResponse.index_infos:type_name -> milvus.proto.index.IndexInfo + 40, // 25: milvus.proto.index.GetIndexBuildProgressResponse.status:type_name -> milvus.proto.common.Status + 36, // 26: milvus.proto.index.JobInfo.index_params:type_name -> milvus.proto.common.KeyValuePair + 40, // 27: milvus.proto.index.GetIndexStatisticsResponse.status:type_name -> milvus.proto.common.Status + 3, // 28: milvus.proto.index.GetIndexStatisticsResponse.index_infos:type_name -> milvus.proto.index.IndexInfo + 40, // 29: milvus.proto.index.ListIndexesResponse.status:type_name -> milvus.proto.common.Status + 3, // 30: milvus.proto.index.ListIndexesResponse.index_infos:type_name -> milvus.proto.index.IndexInfo + 42, // 31: milvus.proto.index.AnalyzeTask.field_type:type_name -> milvus.proto.schema.DataType + 1, // 32: milvus.proto.index.AnalyzeTask.state:type_name -> milvus.proto.index.JobState + 1, // 33: milvus.proto.index.StatsTask.state:type_name -> milvus.proto.index.JobState + 2, // 34: milvus.proto.index.StatsTask.subJobType:type_name -> milvus.proto.index.StatsSubJob + 17, // 35: milvus.proto.index.GetIndexInfoResponse.SegmentInfoEntry.value:type_name -> milvus.proto.index.SegmentInfo + 43, // 36: milvus.proto.index.IndexCoord.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest + 44, // 37: milvus.proto.index.IndexCoord.GetStatisticsChannel:input_type -> milvus.proto.internal.GetStatisticsChannelRequest + 13, // 38: milvus.proto.index.IndexCoord.CreateIndex:input_type -> milvus.proto.index.CreateIndexRequest + 14, // 39: milvus.proto.index.IndexCoord.AlterIndex:input_type -> milvus.proto.index.AlterIndexRequest + 8, // 40: milvus.proto.index.IndexCoord.GetIndexState:input_type -> milvus.proto.index.GetIndexStateRequest + 10, // 41: milvus.proto.index.IndexCoord.GetSegmentIndexState:input_type -> milvus.proto.index.GetSegmentIndexStateRequest + 15, // 42: milvus.proto.index.IndexCoord.GetIndexInfos:input_type -> milvus.proto.index.GetIndexInfoRequest + 19, // 43: milvus.proto.index.IndexCoord.DropIndex:input_type -> milvus.proto.index.DropIndexRequest + 20, // 44: milvus.proto.index.IndexCoord.DescribeIndex:input_type -> milvus.proto.index.DescribeIndexRequest + 27, // 45: milvus.proto.index.IndexCoord.GetIndexStatistics:input_type -> milvus.proto.index.GetIndexStatisticsRequest + 22, // 46: milvus.proto.index.IndexCoord.GetIndexBuildProgress:input_type -> milvus.proto.index.GetIndexBuildProgressRequest + 45, // 47: milvus.proto.index.IndexCoord.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest + 46, // 48: milvus.proto.index.IndexCoord.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest + 47, // 49: milvus.proto.index.IndexCoord.CheckHealth:input_type -> milvus.proto.milvus.CheckHealthRequest + 48, // 50: milvus.proto.index.IndexCoord.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates + 49, // 51: milvus.proto.index.IndexCoord.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse + 40, // 52: milvus.proto.index.IndexCoord.CreateIndex:output_type -> milvus.proto.common.Status + 40, // 53: milvus.proto.index.IndexCoord.AlterIndex:output_type -> milvus.proto.common.Status + 9, // 54: milvus.proto.index.IndexCoord.GetIndexState:output_type -> milvus.proto.index.GetIndexStateResponse + 12, // 55: milvus.proto.index.IndexCoord.GetSegmentIndexState:output_type -> milvus.proto.index.GetSegmentIndexStateResponse + 18, // 56: milvus.proto.index.IndexCoord.GetIndexInfos:output_type -> milvus.proto.index.GetIndexInfoResponse + 40, // 57: milvus.proto.index.IndexCoord.DropIndex:output_type -> milvus.proto.common.Status + 21, // 58: milvus.proto.index.IndexCoord.DescribeIndex:output_type -> milvus.proto.index.DescribeIndexResponse + 28, // 59: milvus.proto.index.IndexCoord.GetIndexStatistics:output_type -> milvus.proto.index.GetIndexStatisticsResponse + 23, // 60: milvus.proto.index.IndexCoord.GetIndexBuildProgress:output_type -> milvus.proto.index.GetIndexBuildProgressResponse + 50, // 61: milvus.proto.index.IndexCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse + 51, // 62: milvus.proto.index.IndexCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse + 52, // 63: milvus.proto.index.IndexCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse + 50, // [50:64] is the sub-list for method output_type + 36, // [36:50] is the sub-list for method input_type + 36, // [36:36] is the sub-list for extension type_name + 36, // [36:36] is the sub-list for extension extendee + 0, // [0:36] is the sub-list for field type_name +} + +func init() { file_index_coord_proto_init() } +func file_index_coord_proto_init() { + if File_index_coord_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_index_coord_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*IndexInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*FieldIndex); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentIndex); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RegisterNodeRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RegisterNodeResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetIndexStateRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetIndexStateResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetSegmentIndexStateRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentIndexState); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetSegmentIndexStateResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CreateIndexRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AlterIndexRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetIndexInfoRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*IndexFilePathInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetIndexInfoResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DropIndexRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DescribeIndexRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DescribeIndexResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetIndexBuildProgressRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetIndexBuildProgressResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*StorageConfig); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*OptionalFieldInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*JobInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetIndexStatisticsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetIndexStatisticsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListIndexesRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListIndexesResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AnalyzeTask); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentStats); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*FieldLogPath); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_index_coord_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*StatsTask); 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{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_index_coord_proto_rawDesc, + NumEnums: 3, + NumMessages: 33, + NumExtensions: 0, + NumServices: 1, + }, + GoTypes: file_index_coord_proto_goTypes, + DependencyIndexes: file_index_coord_proto_depIdxs, + EnumInfos: file_index_coord_proto_enumTypes, + MessageInfos: file_index_coord_proto_msgTypes, + }.Build() + File_index_coord_proto = out.File + file_index_coord_proto_rawDesc = nil + file_index_coord_proto_goTypes = nil + file_index_coord_proto_depIdxs = nil +} diff --git a/pkg/proto/indexpb/index_coord_grpc.pb.go b/pkg/proto/indexpb/index_coord_grpc.pb.go new file mode 100644 index 0000000000000..8c162ad32974f --- /dev/null +++ b/pkg/proto/indexpb/index_coord_grpc.pb.go @@ -0,0 +1,597 @@ +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. +// versions: +// - protoc-gen-go-grpc v1.3.0 +// - protoc v3.21.4 +// source: index_coord.proto + +package indexpb + +import ( + context "context" + commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" +) + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +// Requires gRPC-Go v1.32.0 or later. +const _ = grpc.SupportPackageIsVersion7 + +const ( + IndexCoord_GetComponentStates_FullMethodName = "/milvus.proto.index.IndexCoord/GetComponentStates" + IndexCoord_GetStatisticsChannel_FullMethodName = "/milvus.proto.index.IndexCoord/GetStatisticsChannel" + IndexCoord_CreateIndex_FullMethodName = "/milvus.proto.index.IndexCoord/CreateIndex" + IndexCoord_AlterIndex_FullMethodName = "/milvus.proto.index.IndexCoord/AlterIndex" + IndexCoord_GetIndexState_FullMethodName = "/milvus.proto.index.IndexCoord/GetIndexState" + IndexCoord_GetSegmentIndexState_FullMethodName = "/milvus.proto.index.IndexCoord/GetSegmentIndexState" + IndexCoord_GetIndexInfos_FullMethodName = "/milvus.proto.index.IndexCoord/GetIndexInfos" + IndexCoord_DropIndex_FullMethodName = "/milvus.proto.index.IndexCoord/DropIndex" + IndexCoord_DescribeIndex_FullMethodName = "/milvus.proto.index.IndexCoord/DescribeIndex" + IndexCoord_GetIndexStatistics_FullMethodName = "/milvus.proto.index.IndexCoord/GetIndexStatistics" + IndexCoord_GetIndexBuildProgress_FullMethodName = "/milvus.proto.index.IndexCoord/GetIndexBuildProgress" + IndexCoord_ShowConfigurations_FullMethodName = "/milvus.proto.index.IndexCoord/ShowConfigurations" + IndexCoord_GetMetrics_FullMethodName = "/milvus.proto.index.IndexCoord/GetMetrics" + IndexCoord_CheckHealth_FullMethodName = "/milvus.proto.index.IndexCoord/CheckHealth" +) + +// IndexCoordClient is the client API for IndexCoord service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +type IndexCoordClient interface { + GetComponentStates(ctx context.Context, in *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) + GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) + CreateIndex(ctx context.Context, in *CreateIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + AlterIndex(ctx context.Context, in *AlterIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + // Deprecated: use DescribeIndex instead + GetIndexState(ctx context.Context, in *GetIndexStateRequest, opts ...grpc.CallOption) (*GetIndexStateResponse, error) + GetSegmentIndexState(ctx context.Context, in *GetSegmentIndexStateRequest, opts ...grpc.CallOption) (*GetSegmentIndexStateResponse, error) + GetIndexInfos(ctx context.Context, in *GetIndexInfoRequest, opts ...grpc.CallOption) (*GetIndexInfoResponse, error) + DropIndex(ctx context.Context, in *DropIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + DescribeIndex(ctx context.Context, in *DescribeIndexRequest, opts ...grpc.CallOption) (*DescribeIndexResponse, error) + GetIndexStatistics(ctx context.Context, in *GetIndexStatisticsRequest, opts ...grpc.CallOption) (*GetIndexStatisticsResponse, error) + // Deprecated: use DescribeIndex instead + GetIndexBuildProgress(ctx context.Context, in *GetIndexBuildProgressRequest, opts ...grpc.CallOption) (*GetIndexBuildProgressResponse, error) + ShowConfigurations(ctx context.Context, in *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error) + // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy + GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) + CheckHealth(ctx context.Context, in *milvuspb.CheckHealthRequest, opts ...grpc.CallOption) (*milvuspb.CheckHealthResponse, error) +} + +type indexCoordClient struct { + cc grpc.ClientConnInterface +} + +func NewIndexCoordClient(cc grpc.ClientConnInterface) IndexCoordClient { + return &indexCoordClient{cc} +} + +func (c *indexCoordClient) GetComponentStates(ctx context.Context, in *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) { + out := new(milvuspb.ComponentStates) + err := c.cc.Invoke(ctx, IndexCoord_GetComponentStates_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *indexCoordClient) GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) { + out := new(milvuspb.StringResponse) + err := c.cc.Invoke(ctx, IndexCoord_GetStatisticsChannel_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *indexCoordClient) CreateIndex(ctx context.Context, in *CreateIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, IndexCoord_CreateIndex_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *indexCoordClient) AlterIndex(ctx context.Context, in *AlterIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, IndexCoord_AlterIndex_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *indexCoordClient) GetIndexState(ctx context.Context, in *GetIndexStateRequest, opts ...grpc.CallOption) (*GetIndexStateResponse, error) { + out := new(GetIndexStateResponse) + err := c.cc.Invoke(ctx, IndexCoord_GetIndexState_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *indexCoordClient) GetSegmentIndexState(ctx context.Context, in *GetSegmentIndexStateRequest, opts ...grpc.CallOption) (*GetSegmentIndexStateResponse, error) { + out := new(GetSegmentIndexStateResponse) + err := c.cc.Invoke(ctx, IndexCoord_GetSegmentIndexState_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *indexCoordClient) GetIndexInfos(ctx context.Context, in *GetIndexInfoRequest, opts ...grpc.CallOption) (*GetIndexInfoResponse, error) { + out := new(GetIndexInfoResponse) + err := c.cc.Invoke(ctx, IndexCoord_GetIndexInfos_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *indexCoordClient) DropIndex(ctx context.Context, in *DropIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, IndexCoord_DropIndex_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *indexCoordClient) DescribeIndex(ctx context.Context, in *DescribeIndexRequest, opts ...grpc.CallOption) (*DescribeIndexResponse, error) { + out := new(DescribeIndexResponse) + err := c.cc.Invoke(ctx, IndexCoord_DescribeIndex_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *indexCoordClient) GetIndexStatistics(ctx context.Context, in *GetIndexStatisticsRequest, opts ...grpc.CallOption) (*GetIndexStatisticsResponse, error) { + out := new(GetIndexStatisticsResponse) + err := c.cc.Invoke(ctx, IndexCoord_GetIndexStatistics_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *indexCoordClient) GetIndexBuildProgress(ctx context.Context, in *GetIndexBuildProgressRequest, opts ...grpc.CallOption) (*GetIndexBuildProgressResponse, error) { + out := new(GetIndexBuildProgressResponse) + err := c.cc.Invoke(ctx, IndexCoord_GetIndexBuildProgress_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *indexCoordClient) ShowConfigurations(ctx context.Context, in *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error) { + out := new(internalpb.ShowConfigurationsResponse) + err := c.cc.Invoke(ctx, IndexCoord_ShowConfigurations_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *indexCoordClient) GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) { + out := new(milvuspb.GetMetricsResponse) + err := c.cc.Invoke(ctx, IndexCoord_GetMetrics_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *indexCoordClient) CheckHealth(ctx context.Context, in *milvuspb.CheckHealthRequest, opts ...grpc.CallOption) (*milvuspb.CheckHealthResponse, error) { + out := new(milvuspb.CheckHealthResponse) + err := c.cc.Invoke(ctx, IndexCoord_CheckHealth_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// IndexCoordServer is the server API for IndexCoord service. +// All implementations should embed UnimplementedIndexCoordServer +// for forward compatibility +type IndexCoordServer interface { + GetComponentStates(context.Context, *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) + GetStatisticsChannel(context.Context, *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) + CreateIndex(context.Context, *CreateIndexRequest) (*commonpb.Status, error) + AlterIndex(context.Context, *AlterIndexRequest) (*commonpb.Status, error) + // Deprecated: use DescribeIndex instead + GetIndexState(context.Context, *GetIndexStateRequest) (*GetIndexStateResponse, error) + GetSegmentIndexState(context.Context, *GetSegmentIndexStateRequest) (*GetSegmentIndexStateResponse, error) + GetIndexInfos(context.Context, *GetIndexInfoRequest) (*GetIndexInfoResponse, error) + DropIndex(context.Context, *DropIndexRequest) (*commonpb.Status, error) + DescribeIndex(context.Context, *DescribeIndexRequest) (*DescribeIndexResponse, error) + GetIndexStatistics(context.Context, *GetIndexStatisticsRequest) (*GetIndexStatisticsResponse, error) + // Deprecated: use DescribeIndex instead + GetIndexBuildProgress(context.Context, *GetIndexBuildProgressRequest) (*GetIndexBuildProgressResponse, error) + ShowConfigurations(context.Context, *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) + // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy + GetMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) + CheckHealth(context.Context, *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error) +} + +// UnimplementedIndexCoordServer should be embedded to have forward compatible implementations. +type UnimplementedIndexCoordServer struct { +} + +func (UnimplementedIndexCoordServer) GetComponentStates(context.Context, *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetComponentStates not implemented") +} +func (UnimplementedIndexCoordServer) GetStatisticsChannel(context.Context, *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetStatisticsChannel not implemented") +} +func (UnimplementedIndexCoordServer) CreateIndex(context.Context, *CreateIndexRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method CreateIndex not implemented") +} +func (UnimplementedIndexCoordServer) AlterIndex(context.Context, *AlterIndexRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method AlterIndex not implemented") +} +func (UnimplementedIndexCoordServer) GetIndexState(context.Context, *GetIndexStateRequest) (*GetIndexStateResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetIndexState not implemented") +} +func (UnimplementedIndexCoordServer) GetSegmentIndexState(context.Context, *GetSegmentIndexStateRequest) (*GetSegmentIndexStateResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetSegmentIndexState not implemented") +} +func (UnimplementedIndexCoordServer) GetIndexInfos(context.Context, *GetIndexInfoRequest) (*GetIndexInfoResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetIndexInfos not implemented") +} +func (UnimplementedIndexCoordServer) DropIndex(context.Context, *DropIndexRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method DropIndex not implemented") +} +func (UnimplementedIndexCoordServer) DescribeIndex(context.Context, *DescribeIndexRequest) (*DescribeIndexResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DescribeIndex not implemented") +} +func (UnimplementedIndexCoordServer) GetIndexStatistics(context.Context, *GetIndexStatisticsRequest) (*GetIndexStatisticsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetIndexStatistics not implemented") +} +func (UnimplementedIndexCoordServer) GetIndexBuildProgress(context.Context, *GetIndexBuildProgressRequest) (*GetIndexBuildProgressResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetIndexBuildProgress not implemented") +} +func (UnimplementedIndexCoordServer) ShowConfigurations(context.Context, *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ShowConfigurations not implemented") +} +func (UnimplementedIndexCoordServer) GetMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetMetrics not implemented") +} +func (UnimplementedIndexCoordServer) CheckHealth(context.Context, *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method CheckHealth not implemented") +} + +// UnsafeIndexCoordServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to IndexCoordServer will +// result in compilation errors. +type UnsafeIndexCoordServer interface { + mustEmbedUnimplementedIndexCoordServer() +} + +func RegisterIndexCoordServer(s grpc.ServiceRegistrar, srv IndexCoordServer) { + s.RegisterService(&IndexCoord_ServiceDesc, srv) +} + +func _IndexCoord_GetComponentStates_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.GetComponentStatesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IndexCoordServer).GetComponentStates(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: IndexCoord_GetComponentStates_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IndexCoordServer).GetComponentStates(ctx, req.(*milvuspb.GetComponentStatesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _IndexCoord_GetStatisticsChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.GetStatisticsChannelRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IndexCoordServer).GetStatisticsChannel(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: IndexCoord_GetStatisticsChannel_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IndexCoordServer).GetStatisticsChannel(ctx, req.(*internalpb.GetStatisticsChannelRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _IndexCoord_CreateIndex_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CreateIndexRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IndexCoordServer).CreateIndex(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: IndexCoord_CreateIndex_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IndexCoordServer).CreateIndex(ctx, req.(*CreateIndexRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _IndexCoord_AlterIndex_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(AlterIndexRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IndexCoordServer).AlterIndex(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: IndexCoord_AlterIndex_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IndexCoordServer).AlterIndex(ctx, req.(*AlterIndexRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _IndexCoord_GetIndexState_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetIndexStateRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IndexCoordServer).GetIndexState(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: IndexCoord_GetIndexState_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IndexCoordServer).GetIndexState(ctx, req.(*GetIndexStateRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _IndexCoord_GetSegmentIndexState_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetSegmentIndexStateRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IndexCoordServer).GetSegmentIndexState(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: IndexCoord_GetSegmentIndexState_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IndexCoordServer).GetSegmentIndexState(ctx, req.(*GetSegmentIndexStateRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _IndexCoord_GetIndexInfos_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetIndexInfoRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IndexCoordServer).GetIndexInfos(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: IndexCoord_GetIndexInfos_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IndexCoordServer).GetIndexInfos(ctx, req.(*GetIndexInfoRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _IndexCoord_DropIndex_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DropIndexRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IndexCoordServer).DropIndex(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: IndexCoord_DropIndex_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IndexCoordServer).DropIndex(ctx, req.(*DropIndexRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _IndexCoord_DescribeIndex_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DescribeIndexRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IndexCoordServer).DescribeIndex(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: IndexCoord_DescribeIndex_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IndexCoordServer).DescribeIndex(ctx, req.(*DescribeIndexRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _IndexCoord_GetIndexStatistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetIndexStatisticsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IndexCoordServer).GetIndexStatistics(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: IndexCoord_GetIndexStatistics_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IndexCoordServer).GetIndexStatistics(ctx, req.(*GetIndexStatisticsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _IndexCoord_GetIndexBuildProgress_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetIndexBuildProgressRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IndexCoordServer).GetIndexBuildProgress(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: IndexCoord_GetIndexBuildProgress_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IndexCoordServer).GetIndexBuildProgress(ctx, req.(*GetIndexBuildProgressRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _IndexCoord_ShowConfigurations_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.ShowConfigurationsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IndexCoordServer).ShowConfigurations(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: IndexCoord_ShowConfigurations_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IndexCoordServer).ShowConfigurations(ctx, req.(*internalpb.ShowConfigurationsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _IndexCoord_GetMetrics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.GetMetricsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IndexCoordServer).GetMetrics(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: IndexCoord_GetMetrics_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IndexCoordServer).GetMetrics(ctx, req.(*milvuspb.GetMetricsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _IndexCoord_CheckHealth_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.CheckHealthRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IndexCoordServer).CheckHealth(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: IndexCoord_CheckHealth_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IndexCoordServer).CheckHealth(ctx, req.(*milvuspb.CheckHealthRequest)) + } + return interceptor(ctx, in, info, handler) +} + +// IndexCoord_ServiceDesc is the grpc.ServiceDesc for IndexCoord service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var IndexCoord_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "milvus.proto.index.IndexCoord", + HandlerType: (*IndexCoordServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "GetComponentStates", + Handler: _IndexCoord_GetComponentStates_Handler, + }, + { + MethodName: "GetStatisticsChannel", + Handler: _IndexCoord_GetStatisticsChannel_Handler, + }, + { + MethodName: "CreateIndex", + Handler: _IndexCoord_CreateIndex_Handler, + }, + { + MethodName: "AlterIndex", + Handler: _IndexCoord_AlterIndex_Handler, + }, + { + MethodName: "GetIndexState", + Handler: _IndexCoord_GetIndexState_Handler, + }, + { + MethodName: "GetSegmentIndexState", + Handler: _IndexCoord_GetSegmentIndexState_Handler, + }, + { + MethodName: "GetIndexInfos", + Handler: _IndexCoord_GetIndexInfos_Handler, + }, + { + MethodName: "DropIndex", + Handler: _IndexCoord_DropIndex_Handler, + }, + { + MethodName: "DescribeIndex", + Handler: _IndexCoord_DescribeIndex_Handler, + }, + { + MethodName: "GetIndexStatistics", + Handler: _IndexCoord_GetIndexStatistics_Handler, + }, + { + MethodName: "GetIndexBuildProgress", + Handler: _IndexCoord_GetIndexBuildProgress_Handler, + }, + { + MethodName: "ShowConfigurations", + Handler: _IndexCoord_ShowConfigurations_Handler, + }, + { + MethodName: "GetMetrics", + Handler: _IndexCoord_GetMetrics_Handler, + }, + { + MethodName: "CheckHealth", + Handler: _IndexCoord_CheckHealth_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "index_coord.proto", +} diff --git a/internal/proto/internal.proto b/pkg/proto/internal.proto similarity index 99% rename from internal/proto/internal.proto rename to pkg/proto/internal.proto index f61d4f7acbab8..8fac366937c37 100644 --- a/internal/proto/internal.proto +++ b/pkg/proto/internal.proto @@ -1,6 +1,6 @@ syntax = "proto3"; package milvus.proto.internal; -option go_package = "github.com/milvus-io/milvus/internal/proto/internalpb"; +option go_package = "github.com/milvus-io/milvus/pkg/proto/internalpb"; import "common.proto"; import "schema.proto"; diff --git a/pkg/proto/internalpb/internal.pb.go b/pkg/proto/internalpb/internal.pb.go new file mode 100644 index 0000000000000..c53a40294b787 --- /dev/null +++ b/pkg/proto/internalpb/internal.pb.go @@ -0,0 +1,4767 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.33.0 +// protoc v3.21.4 +// source: internal.proto + +package internalpb + +import ( + commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + 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" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type RateScope int32 + +const ( + RateScope_Cluster RateScope = 0 + RateScope_Database RateScope = 1 + RateScope_Collection RateScope = 2 + RateScope_Partition RateScope = 3 +) + +// Enum value maps for RateScope. +var ( + RateScope_name = map[int32]string{ + 0: "Cluster", + 1: "Database", + 2: "Collection", + 3: "Partition", + } + RateScope_value = map[string]int32{ + "Cluster": 0, + "Database": 1, + "Collection": 2, + "Partition": 3, + } +) + +func (x RateScope) Enum() *RateScope { + p := new(RateScope) + *p = x + return p +} + +func (x RateScope) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (RateScope) Descriptor() protoreflect.EnumDescriptor { + return file_internal_proto_enumTypes[0].Descriptor() +} + +func (RateScope) Type() protoreflect.EnumType { + return &file_internal_proto_enumTypes[0] +} + +func (x RateScope) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use RateScope.Descriptor instead. +func (RateScope) EnumDescriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{0} +} + +type RateType int32 + +const ( + RateType_DDLCollection RateType = 0 + RateType_DDLPartition RateType = 1 + RateType_DDLIndex RateType = 2 + RateType_DDLFlush RateType = 3 + RateType_DDLCompaction RateType = 4 + RateType_DMLInsert RateType = 5 + RateType_DMLDelete RateType = 6 + RateType_DMLBulkLoad RateType = 7 + RateType_DQLSearch RateType = 8 + RateType_DQLQuery RateType = 9 + RateType_DMLUpsert RateType = 10 +) + +// Enum value maps for RateType. +var ( + RateType_name = map[int32]string{ + 0: "DDLCollection", + 1: "DDLPartition", + 2: "DDLIndex", + 3: "DDLFlush", + 4: "DDLCompaction", + 5: "DMLInsert", + 6: "DMLDelete", + 7: "DMLBulkLoad", + 8: "DQLSearch", + 9: "DQLQuery", + 10: "DMLUpsert", + } + RateType_value = map[string]int32{ + "DDLCollection": 0, + "DDLPartition": 1, + "DDLIndex": 2, + "DDLFlush": 3, + "DDLCompaction": 4, + "DMLInsert": 5, + "DMLDelete": 6, + "DMLBulkLoad": 7, + "DQLSearch": 8, + "DQLQuery": 9, + "DMLUpsert": 10, + } +) + +func (x RateType) Enum() *RateType { + p := new(RateType) + *p = x + return p +} + +func (x RateType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (RateType) Descriptor() protoreflect.EnumDescriptor { + return file_internal_proto_enumTypes[1].Descriptor() +} + +func (RateType) Type() protoreflect.EnumType { + return &file_internal_proto_enumTypes[1] +} + +func (x RateType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use RateType.Descriptor instead. +func (RateType) EnumDescriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{1} +} + +type ImportJobState int32 + +const ( + ImportJobState_None ImportJobState = 0 + ImportJobState_Pending ImportJobState = 1 + ImportJobState_PreImporting ImportJobState = 2 + ImportJobState_Importing ImportJobState = 3 + ImportJobState_Failed ImportJobState = 4 + ImportJobState_Completed ImportJobState = 5 + ImportJobState_IndexBuilding ImportJobState = 6 + ImportJobState_Stats ImportJobState = 7 +) + +// Enum value maps for ImportJobState. +var ( + ImportJobState_name = map[int32]string{ + 0: "None", + 1: "Pending", + 2: "PreImporting", + 3: "Importing", + 4: "Failed", + 5: "Completed", + 6: "IndexBuilding", + 7: "Stats", + } + ImportJobState_value = map[string]int32{ + "None": 0, + "Pending": 1, + "PreImporting": 2, + "Importing": 3, + "Failed": 4, + "Completed": 5, + "IndexBuilding": 6, + "Stats": 7, + } +) + +func (x ImportJobState) Enum() *ImportJobState { + p := new(ImportJobState) + *p = x + return p +} + +func (x ImportJobState) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (ImportJobState) Descriptor() protoreflect.EnumDescriptor { + return file_internal_proto_enumTypes[2].Descriptor() +} + +func (ImportJobState) Type() protoreflect.EnumType { + return &file_internal_proto_enumTypes[2] +} + +func (x ImportJobState) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ImportJobState.Descriptor instead. +func (ImportJobState) EnumDescriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{2} +} + +type GetTimeTickChannelRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *GetTimeTickChannelRequest) Reset() { + *x = GetTimeTickChannelRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetTimeTickChannelRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetTimeTickChannelRequest) ProtoMessage() {} + +func (x *GetTimeTickChannelRequest) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[0] + 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 GetTimeTickChannelRequest.ProtoReflect.Descriptor instead. +func (*GetTimeTickChannelRequest) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{0} +} + +type GetStatisticsChannelRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *GetStatisticsChannelRequest) Reset() { + *x = GetStatisticsChannelRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetStatisticsChannelRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetStatisticsChannelRequest) ProtoMessage() {} + +func (x *GetStatisticsChannelRequest) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[1] + 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 GetStatisticsChannelRequest.ProtoReflect.Descriptor instead. +func (*GetStatisticsChannelRequest) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{1} +} + +type GetDdChannelRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *GetDdChannelRequest) Reset() { + *x = GetDdChannelRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetDdChannelRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetDdChannelRequest) ProtoMessage() {} + +func (x *GetDdChannelRequest) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[2] + 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 GetDdChannelRequest.ProtoReflect.Descriptor instead. +func (*GetDdChannelRequest) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{2} +} + +type NodeInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Address *commonpb.Address `protobuf:"bytes,1,opt,name=address,proto3" json:"address,omitempty"` + Role string `protobuf:"bytes,2,opt,name=role,proto3" json:"role,omitempty"` +} + +func (x *NodeInfo) Reset() { + *x = NodeInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NodeInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NodeInfo) ProtoMessage() {} + +func (x *NodeInfo) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[3] + 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 NodeInfo.ProtoReflect.Descriptor instead. +func (*NodeInfo) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{3} +} + +func (x *NodeInfo) GetAddress() *commonpb.Address { + if x != nil { + return x.Address + } + return nil +} + +func (x *NodeInfo) GetRole() string { + if x != nil { + return x.Role + } + return "" +} + +type InitParams struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + NodeID int64 `protobuf:"varint,1,opt,name=nodeID,proto3" json:"nodeID,omitempty"` + StartParams []*commonpb.KeyValuePair `protobuf:"bytes,2,rep,name=start_params,json=startParams,proto3" json:"start_params,omitempty"` +} + +func (x *InitParams) Reset() { + *x = InitParams{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *InitParams) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*InitParams) ProtoMessage() {} + +func (x *InitParams) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[4] + 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 InitParams.ProtoReflect.Descriptor instead. +func (*InitParams) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{4} +} + +func (x *InitParams) GetNodeID() int64 { + if x != nil { + return x.NodeID + } + return 0 +} + +func (x *InitParams) GetStartParams() []*commonpb.KeyValuePair { + if x != nil { + return x.StartParams + } + return nil +} + +type StringList struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Values []string `protobuf:"bytes,1,rep,name=values,proto3" json:"values,omitempty"` + Status *commonpb.Status `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` +} + +func (x *StringList) Reset() { + *x = StringList{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StringList) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StringList) ProtoMessage() {} + +func (x *StringList) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[5] + 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 StringList.ProtoReflect.Descriptor instead. +func (*StringList) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{5} +} + +func (x *StringList) GetValues() []string { + if x != nil { + return x.Values + } + return nil +} + +func (x *StringList) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +type GetStatisticsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + // Not useful for now + DbID int64 `protobuf:"varint,2,opt,name=dbID,proto3" json:"dbID,omitempty"` + // The collection you want get statistics + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + // The partitions you want get statistics + PartitionIDs []int64 `protobuf:"varint,4,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` + // timestamp of the statistics + TravelTimestamp uint64 `protobuf:"varint,5,opt,name=travel_timestamp,json=travelTimestamp,proto3" json:"travel_timestamp,omitempty"` + GuaranteeTimestamp uint64 `protobuf:"varint,6,opt,name=guarantee_timestamp,json=guaranteeTimestamp,proto3" json:"guarantee_timestamp,omitempty"` + TimeoutTimestamp uint64 `protobuf:"varint,7,opt,name=timeout_timestamp,json=timeoutTimestamp,proto3" json:"timeout_timestamp,omitempty"` +} + +func (x *GetStatisticsRequest) Reset() { + *x = GetStatisticsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetStatisticsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetStatisticsRequest) ProtoMessage() {} + +func (x *GetStatisticsRequest) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[6] + 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 GetStatisticsRequest.ProtoReflect.Descriptor instead. +func (*GetStatisticsRequest) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{6} +} + +func (x *GetStatisticsRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *GetStatisticsRequest) GetDbID() int64 { + if x != nil { + return x.DbID + } + return 0 +} + +func (x *GetStatisticsRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *GetStatisticsRequest) GetPartitionIDs() []int64 { + if x != nil { + return x.PartitionIDs + } + return nil +} + +func (x *GetStatisticsRequest) GetTravelTimestamp() uint64 { + if x != nil { + return x.TravelTimestamp + } + return 0 +} + +func (x *GetStatisticsRequest) GetGuaranteeTimestamp() uint64 { + if x != nil { + return x.GuaranteeTimestamp + } + return 0 +} + +func (x *GetStatisticsRequest) GetTimeoutTimestamp() uint64 { + if x != nil { + return x.TimeoutTimestamp + } + return 0 +} + +type GetStatisticsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + // Contain error_code and reason + Status *commonpb.Status `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` + // Collection statistics data. Contain pairs like {"row_count": "1"} + Stats []*commonpb.KeyValuePair `protobuf:"bytes,3,rep,name=stats,proto3" json:"stats,omitempty"` +} + +func (x *GetStatisticsResponse) Reset() { + *x = GetStatisticsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetStatisticsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetStatisticsResponse) ProtoMessage() {} + +func (x *GetStatisticsResponse) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[7] + 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 GetStatisticsResponse.ProtoReflect.Descriptor instead. +func (*GetStatisticsResponse) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{7} +} + +func (x *GetStatisticsResponse) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *GetStatisticsResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *GetStatisticsResponse) GetStats() []*commonpb.KeyValuePair { + if x != nil { + return x.Stats + } + return nil +} + +type CreateAliasRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + DbName string `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` + CollectionName string `protobuf:"bytes,3,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` + Alias string `protobuf:"bytes,4,opt,name=alias,proto3" json:"alias,omitempty"` +} + +func (x *CreateAliasRequest) Reset() { + *x = CreateAliasRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateAliasRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateAliasRequest) ProtoMessage() {} + +func (x *CreateAliasRequest) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[8] + 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 CreateAliasRequest.ProtoReflect.Descriptor instead. +func (*CreateAliasRequest) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{8} +} + +func (x *CreateAliasRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *CreateAliasRequest) GetDbName() string { + if x != nil { + return x.DbName + } + return "" +} + +func (x *CreateAliasRequest) GetCollectionName() string { + if x != nil { + return x.CollectionName + } + return "" +} + +func (x *CreateAliasRequest) GetAlias() string { + if x != nil { + return x.Alias + } + return "" +} + +type DropAliasRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + DbName string `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` + Alias string `protobuf:"bytes,3,opt,name=alias,proto3" json:"alias,omitempty"` +} + +func (x *DropAliasRequest) Reset() { + *x = DropAliasRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DropAliasRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DropAliasRequest) ProtoMessage() {} + +func (x *DropAliasRequest) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[9] + 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 DropAliasRequest.ProtoReflect.Descriptor instead. +func (*DropAliasRequest) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{9} +} + +func (x *DropAliasRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *DropAliasRequest) GetDbName() string { + if x != nil { + return x.DbName + } + return "" +} + +func (x *DropAliasRequest) GetAlias() string { + if x != nil { + return x.Alias + } + return "" +} + +type AlterAliasRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + DbName string `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` + CollectionName string `protobuf:"bytes,3,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` + Alias string `protobuf:"bytes,4,opt,name=alias,proto3" json:"alias,omitempty"` +} + +func (x *AlterAliasRequest) Reset() { + *x = AlterAliasRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AlterAliasRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AlterAliasRequest) ProtoMessage() {} + +func (x *AlterAliasRequest) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[10] + 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 AlterAliasRequest.ProtoReflect.Descriptor instead. +func (*AlterAliasRequest) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{10} +} + +func (x *AlterAliasRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *AlterAliasRequest) GetDbName() string { + if x != nil { + return x.DbName + } + return "" +} + +func (x *AlterAliasRequest) GetCollectionName() string { + if x != nil { + return x.CollectionName + } + return "" +} + +func (x *AlterAliasRequest) GetAlias() string { + if x != nil { + return x.Alias + } + return "" +} + +type CreateIndexRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + DbName string `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` + CollectionName string `protobuf:"bytes,3,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` + FieldName string `protobuf:"bytes,4,opt,name=field_name,json=fieldName,proto3" json:"field_name,omitempty"` + DbID int64 `protobuf:"varint,5,opt,name=dbID,proto3" json:"dbID,omitempty"` + CollectionID int64 `protobuf:"varint,6,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + FieldID int64 `protobuf:"varint,7,opt,name=fieldID,proto3" json:"fieldID,omitempty"` + ExtraParams []*commonpb.KeyValuePair `protobuf:"bytes,8,rep,name=extra_params,json=extraParams,proto3" json:"extra_params,omitempty"` +} + +func (x *CreateIndexRequest) Reset() { + *x = CreateIndexRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateIndexRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateIndexRequest) ProtoMessage() {} + +func (x *CreateIndexRequest) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[11] + 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 CreateIndexRequest.ProtoReflect.Descriptor instead. +func (*CreateIndexRequest) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{11} +} + +func (x *CreateIndexRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *CreateIndexRequest) GetDbName() string { + if x != nil { + return x.DbName + } + return "" +} + +func (x *CreateIndexRequest) GetCollectionName() string { + if x != nil { + return x.CollectionName + } + return "" +} + +func (x *CreateIndexRequest) GetFieldName() string { + if x != nil { + return x.FieldName + } + return "" +} + +func (x *CreateIndexRequest) GetDbID() int64 { + if x != nil { + return x.DbID + } + return 0 +} + +func (x *CreateIndexRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *CreateIndexRequest) GetFieldID() int64 { + if x != nil { + return x.FieldID + } + return 0 +} + +func (x *CreateIndexRequest) GetExtraParams() []*commonpb.KeyValuePair { + if x != nil { + return x.ExtraParams + } + return nil +} + +type SubSearchRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Dsl string `protobuf:"bytes,1,opt,name=dsl,proto3" json:"dsl,omitempty"` + // serialized `PlaceholderGroup` + PlaceholderGroup []byte `protobuf:"bytes,2,opt,name=placeholder_group,json=placeholderGroup,proto3" json:"placeholder_group,omitempty"` + DslType commonpb.DslType `protobuf:"varint,3,opt,name=dsl_type,json=dslType,proto3,enum=milvus.proto.common.DslType" json:"dsl_type,omitempty"` + SerializedExprPlan []byte `protobuf:"bytes,4,opt,name=serialized_expr_plan,json=serializedExprPlan,proto3" json:"serialized_expr_plan,omitempty"` + Nq int64 `protobuf:"varint,5,opt,name=nq,proto3" json:"nq,omitempty"` + PartitionIDs []int64 `protobuf:"varint,6,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` + Topk int64 `protobuf:"varint,7,opt,name=topk,proto3" json:"topk,omitempty"` + Offset int64 `protobuf:"varint,8,opt,name=offset,proto3" json:"offset,omitempty"` + MetricType string `protobuf:"bytes,9,opt,name=metricType,proto3" json:"metricType,omitempty"` + GroupByFieldId int64 `protobuf:"varint,10,opt,name=group_by_field_id,json=groupByFieldId,proto3" json:"group_by_field_id,omitempty"` + GroupSize int64 `protobuf:"varint,11,opt,name=group_size,json=groupSize,proto3" json:"group_size,omitempty"` + FieldId int64 `protobuf:"varint,12,opt,name=field_id,json=fieldId,proto3" json:"field_id,omitempty"` +} + +func (x *SubSearchRequest) Reset() { + *x = SubSearchRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SubSearchRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SubSearchRequest) ProtoMessage() {} + +func (x *SubSearchRequest) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[12] + 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 SubSearchRequest.ProtoReflect.Descriptor instead. +func (*SubSearchRequest) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{12} +} + +func (x *SubSearchRequest) GetDsl() string { + if x != nil { + return x.Dsl + } + return "" +} + +func (x *SubSearchRequest) GetPlaceholderGroup() []byte { + if x != nil { + return x.PlaceholderGroup + } + return nil +} + +func (x *SubSearchRequest) GetDslType() commonpb.DslType { + if x != nil { + return x.DslType + } + return commonpb.DslType(0) +} + +func (x *SubSearchRequest) GetSerializedExprPlan() []byte { + if x != nil { + return x.SerializedExprPlan + } + return nil +} + +func (x *SubSearchRequest) GetNq() int64 { + if x != nil { + return x.Nq + } + return 0 +} + +func (x *SubSearchRequest) GetPartitionIDs() []int64 { + if x != nil { + return x.PartitionIDs + } + return nil +} + +func (x *SubSearchRequest) GetTopk() int64 { + if x != nil { + return x.Topk + } + return 0 +} + +func (x *SubSearchRequest) GetOffset() int64 { + if x != nil { + return x.Offset + } + return 0 +} + +func (x *SubSearchRequest) GetMetricType() string { + if x != nil { + return x.MetricType + } + return "" +} + +func (x *SubSearchRequest) GetGroupByFieldId() int64 { + if x != nil { + return x.GroupByFieldId + } + return 0 +} + +func (x *SubSearchRequest) GetGroupSize() int64 { + if x != nil { + return x.GroupSize + } + return 0 +} + +func (x *SubSearchRequest) GetFieldId() int64 { + if x != nil { + return x.FieldId + } + return 0 +} + +type SearchRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + ReqID int64 `protobuf:"varint,2,opt,name=reqID,proto3" json:"reqID,omitempty"` + DbID int64 `protobuf:"varint,3,opt,name=dbID,proto3" json:"dbID,omitempty"` + CollectionID int64 `protobuf:"varint,4,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionIDs []int64 `protobuf:"varint,5,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` + Dsl string `protobuf:"bytes,6,opt,name=dsl,proto3" json:"dsl,omitempty"` + // serialized `PlaceholderGroup` + PlaceholderGroup []byte `protobuf:"bytes,7,opt,name=placeholder_group,json=placeholderGroup,proto3" json:"placeholder_group,omitempty"` + DslType commonpb.DslType `protobuf:"varint,8,opt,name=dsl_type,json=dslType,proto3,enum=milvus.proto.common.DslType" json:"dsl_type,omitempty"` + SerializedExprPlan []byte `protobuf:"bytes,9,opt,name=serialized_expr_plan,json=serializedExprPlan,proto3" json:"serialized_expr_plan,omitempty"` + OutputFieldsId []int64 `protobuf:"varint,10,rep,packed,name=output_fields_id,json=outputFieldsId,proto3" json:"output_fields_id,omitempty"` + MvccTimestamp uint64 `protobuf:"varint,11,opt,name=mvcc_timestamp,json=mvccTimestamp,proto3" json:"mvcc_timestamp,omitempty"` + GuaranteeTimestamp uint64 `protobuf:"varint,12,opt,name=guarantee_timestamp,json=guaranteeTimestamp,proto3" json:"guarantee_timestamp,omitempty"` + TimeoutTimestamp uint64 `protobuf:"varint,13,opt,name=timeout_timestamp,json=timeoutTimestamp,proto3" json:"timeout_timestamp,omitempty"` + Nq int64 `protobuf:"varint,14,opt,name=nq,proto3" json:"nq,omitempty"` + Topk int64 `protobuf:"varint,15,opt,name=topk,proto3" json:"topk,omitempty"` + MetricType string `protobuf:"bytes,16,opt,name=metricType,proto3" json:"metricType,omitempty"` + IgnoreGrowing bool `protobuf:"varint,17,opt,name=ignoreGrowing,proto3" json:"ignoreGrowing,omitempty"` // Optional + Username string `protobuf:"bytes,18,opt,name=username,proto3" json:"username,omitempty"` + SubReqs []*SubSearchRequest `protobuf:"bytes,19,rep,name=sub_reqs,json=subReqs,proto3" json:"sub_reqs,omitempty"` + IsAdvanced bool `protobuf:"varint,20,opt,name=is_advanced,json=isAdvanced,proto3" json:"is_advanced,omitempty"` + Offset int64 `protobuf:"varint,21,opt,name=offset,proto3" json:"offset,omitempty"` + ConsistencyLevel commonpb.ConsistencyLevel `protobuf:"varint,22,opt,name=consistency_level,json=consistencyLevel,proto3,enum=milvus.proto.common.ConsistencyLevel" json:"consistency_level,omitempty"` + GroupByFieldId int64 `protobuf:"varint,23,opt,name=group_by_field_id,json=groupByFieldId,proto3" json:"group_by_field_id,omitempty"` + GroupSize int64 `protobuf:"varint,24,opt,name=group_size,json=groupSize,proto3" json:"group_size,omitempty"` + FieldId int64 `protobuf:"varint,25,opt,name=field_id,json=fieldId,proto3" json:"field_id,omitempty"` + IsTopkReduce bool `protobuf:"varint,26,opt,name=is_topk_reduce,json=isTopkReduce,proto3" json:"is_topk_reduce,omitempty"` + IsRecallEvaluation bool `protobuf:"varint,27,opt,name=is_recall_evaluation,json=isRecallEvaluation,proto3" json:"is_recall_evaluation,omitempty"` +} + +func (x *SearchRequest) Reset() { + *x = SearchRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SearchRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SearchRequest) ProtoMessage() {} + +func (x *SearchRequest) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[13] + 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 SearchRequest.ProtoReflect.Descriptor instead. +func (*SearchRequest) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{13} +} + +func (x *SearchRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *SearchRequest) GetReqID() int64 { + if x != nil { + return x.ReqID + } + return 0 +} + +func (x *SearchRequest) GetDbID() int64 { + if x != nil { + return x.DbID + } + return 0 +} + +func (x *SearchRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *SearchRequest) GetPartitionIDs() []int64 { + if x != nil { + return x.PartitionIDs + } + return nil +} + +func (x *SearchRequest) GetDsl() string { + if x != nil { + return x.Dsl + } + return "" +} + +func (x *SearchRequest) GetPlaceholderGroup() []byte { + if x != nil { + return x.PlaceholderGroup + } + return nil +} + +func (x *SearchRequest) GetDslType() commonpb.DslType { + if x != nil { + return x.DslType + } + return commonpb.DslType(0) +} + +func (x *SearchRequest) GetSerializedExprPlan() []byte { + if x != nil { + return x.SerializedExprPlan + } + return nil +} + +func (x *SearchRequest) GetOutputFieldsId() []int64 { + if x != nil { + return x.OutputFieldsId + } + return nil +} + +func (x *SearchRequest) GetMvccTimestamp() uint64 { + if x != nil { + return x.MvccTimestamp + } + return 0 +} + +func (x *SearchRequest) GetGuaranteeTimestamp() uint64 { + if x != nil { + return x.GuaranteeTimestamp + } + return 0 +} + +func (x *SearchRequest) GetTimeoutTimestamp() uint64 { + if x != nil { + return x.TimeoutTimestamp + } + return 0 +} + +func (x *SearchRequest) GetNq() int64 { + if x != nil { + return x.Nq + } + return 0 +} + +func (x *SearchRequest) GetTopk() int64 { + if x != nil { + return x.Topk + } + return 0 +} + +func (x *SearchRequest) GetMetricType() string { + if x != nil { + return x.MetricType + } + return "" +} + +func (x *SearchRequest) GetIgnoreGrowing() bool { + if x != nil { + return x.IgnoreGrowing + } + return false +} + +func (x *SearchRequest) GetUsername() string { + if x != nil { + return x.Username + } + return "" +} + +func (x *SearchRequest) GetSubReqs() []*SubSearchRequest { + if x != nil { + return x.SubReqs + } + return nil +} + +func (x *SearchRequest) GetIsAdvanced() bool { + if x != nil { + return x.IsAdvanced + } + return false +} + +func (x *SearchRequest) GetOffset() int64 { + if x != nil { + return x.Offset + } + return 0 +} + +func (x *SearchRequest) GetConsistencyLevel() commonpb.ConsistencyLevel { + if x != nil { + return x.ConsistencyLevel + } + return commonpb.ConsistencyLevel(0) +} + +func (x *SearchRequest) GetGroupByFieldId() int64 { + if x != nil { + return x.GroupByFieldId + } + return 0 +} + +func (x *SearchRequest) GetGroupSize() int64 { + if x != nil { + return x.GroupSize + } + return 0 +} + +func (x *SearchRequest) GetFieldId() int64 { + if x != nil { + return x.FieldId + } + return 0 +} + +func (x *SearchRequest) GetIsTopkReduce() bool { + if x != nil { + return x.IsTopkReduce + } + return false +} + +func (x *SearchRequest) GetIsRecallEvaluation() bool { + if x != nil { + return x.IsRecallEvaluation + } + return false +} + +type SubSearchResults struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + MetricType string `protobuf:"bytes,1,opt,name=metric_type,json=metricType,proto3" json:"metric_type,omitempty"` + NumQueries int64 `protobuf:"varint,2,opt,name=num_queries,json=numQueries,proto3" json:"num_queries,omitempty"` + TopK int64 `protobuf:"varint,3,opt,name=top_k,json=topK,proto3" json:"top_k,omitempty"` + // schema.SearchResultsData inside + SlicedBlob []byte `protobuf:"bytes,4,opt,name=sliced_blob,json=slicedBlob,proto3" json:"sliced_blob,omitempty"` + SlicedNumCount int64 `protobuf:"varint,5,opt,name=sliced_num_count,json=slicedNumCount,proto3" json:"sliced_num_count,omitempty"` + SlicedOffset int64 `protobuf:"varint,6,opt,name=sliced_offset,json=slicedOffset,proto3" json:"sliced_offset,omitempty"` + // to indicate it belongs to which sub request + ReqIndex int64 `protobuf:"varint,7,opt,name=req_index,json=reqIndex,proto3" json:"req_index,omitempty"` +} + +func (x *SubSearchResults) Reset() { + *x = SubSearchResults{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SubSearchResults) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SubSearchResults) ProtoMessage() {} + +func (x *SubSearchResults) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[14] + 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 SubSearchResults.ProtoReflect.Descriptor instead. +func (*SubSearchResults) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{14} +} + +func (x *SubSearchResults) GetMetricType() string { + if x != nil { + return x.MetricType + } + return "" +} + +func (x *SubSearchResults) GetNumQueries() int64 { + if x != nil { + return x.NumQueries + } + return 0 +} + +func (x *SubSearchResults) GetTopK() int64 { + if x != nil { + return x.TopK + } + return 0 +} + +func (x *SubSearchResults) GetSlicedBlob() []byte { + if x != nil { + return x.SlicedBlob + } + return nil +} + +func (x *SubSearchResults) GetSlicedNumCount() int64 { + if x != nil { + return x.SlicedNumCount + } + return 0 +} + +func (x *SubSearchResults) GetSlicedOffset() int64 { + if x != nil { + return x.SlicedOffset + } + return 0 +} + +func (x *SubSearchResults) GetReqIndex() int64 { + if x != nil { + return x.ReqIndex + } + return 0 +} + +type SearchResults struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + Status *commonpb.Status `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` + ReqID int64 `protobuf:"varint,3,opt,name=reqID,proto3" json:"reqID,omitempty"` + MetricType string `protobuf:"bytes,4,opt,name=metric_type,json=metricType,proto3" json:"metric_type,omitempty"` + NumQueries int64 `protobuf:"varint,5,opt,name=num_queries,json=numQueries,proto3" json:"num_queries,omitempty"` + TopK int64 `protobuf:"varint,6,opt,name=top_k,json=topK,proto3" json:"top_k,omitempty"` + SealedSegmentIDsSearched []int64 `protobuf:"varint,7,rep,packed,name=sealed_segmentIDs_searched,json=sealedSegmentIDsSearched,proto3" json:"sealed_segmentIDs_searched,omitempty"` + ChannelIDsSearched []string `protobuf:"bytes,8,rep,name=channelIDs_searched,json=channelIDsSearched,proto3" json:"channelIDs_searched,omitempty"` + GlobalSealedSegmentIDs []int64 `protobuf:"varint,9,rep,packed,name=global_sealed_segmentIDs,json=globalSealedSegmentIDs,proto3" json:"global_sealed_segmentIDs,omitempty"` + // schema.SearchResultsData inside + SlicedBlob []byte `protobuf:"bytes,10,opt,name=sliced_blob,json=slicedBlob,proto3" json:"sliced_blob,omitempty"` + SlicedNumCount int64 `protobuf:"varint,11,opt,name=sliced_num_count,json=slicedNumCount,proto3" json:"sliced_num_count,omitempty"` + SlicedOffset int64 `protobuf:"varint,12,opt,name=sliced_offset,json=slicedOffset,proto3" json:"sliced_offset,omitempty"` + // search request cost + CostAggregation *CostAggregation `protobuf:"bytes,13,opt,name=costAggregation,proto3" json:"costAggregation,omitempty"` + ChannelsMvcc map[string]uint64 `protobuf:"bytes,14,rep,name=channels_mvcc,json=channelsMvcc,proto3" json:"channels_mvcc,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + SubResults []*SubSearchResults `protobuf:"bytes,15,rep,name=sub_results,json=subResults,proto3" json:"sub_results,omitempty"` + IsAdvanced bool `protobuf:"varint,16,opt,name=is_advanced,json=isAdvanced,proto3" json:"is_advanced,omitempty"` + AllSearchCount int64 `protobuf:"varint,17,opt,name=all_search_count,json=allSearchCount,proto3" json:"all_search_count,omitempty"` + IsTopkReduce bool `protobuf:"varint,18,opt,name=is_topk_reduce,json=isTopkReduce,proto3" json:"is_topk_reduce,omitempty"` + IsRecallEvaluation bool `protobuf:"varint,19,opt,name=is_recall_evaluation,json=isRecallEvaluation,proto3" json:"is_recall_evaluation,omitempty"` +} + +func (x *SearchResults) Reset() { + *x = SearchResults{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SearchResults) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SearchResults) ProtoMessage() {} + +func (x *SearchResults) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[15] + 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 SearchResults.ProtoReflect.Descriptor instead. +func (*SearchResults) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{15} +} + +func (x *SearchResults) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *SearchResults) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *SearchResults) GetReqID() int64 { + if x != nil { + return x.ReqID + } + return 0 +} + +func (x *SearchResults) GetMetricType() string { + if x != nil { + return x.MetricType + } + return "" +} + +func (x *SearchResults) GetNumQueries() int64 { + if x != nil { + return x.NumQueries + } + return 0 +} + +func (x *SearchResults) GetTopK() int64 { + if x != nil { + return x.TopK + } + return 0 +} + +func (x *SearchResults) GetSealedSegmentIDsSearched() []int64 { + if x != nil { + return x.SealedSegmentIDsSearched + } + return nil +} + +func (x *SearchResults) GetChannelIDsSearched() []string { + if x != nil { + return x.ChannelIDsSearched + } + return nil +} + +func (x *SearchResults) GetGlobalSealedSegmentIDs() []int64 { + if x != nil { + return x.GlobalSealedSegmentIDs + } + return nil +} + +func (x *SearchResults) GetSlicedBlob() []byte { + if x != nil { + return x.SlicedBlob + } + return nil +} + +func (x *SearchResults) GetSlicedNumCount() int64 { + if x != nil { + return x.SlicedNumCount + } + return 0 +} + +func (x *SearchResults) GetSlicedOffset() int64 { + if x != nil { + return x.SlicedOffset + } + return 0 +} + +func (x *SearchResults) GetCostAggregation() *CostAggregation { + if x != nil { + return x.CostAggregation + } + return nil +} + +func (x *SearchResults) GetChannelsMvcc() map[string]uint64 { + if x != nil { + return x.ChannelsMvcc + } + return nil +} + +func (x *SearchResults) GetSubResults() []*SubSearchResults { + if x != nil { + return x.SubResults + } + return nil +} + +func (x *SearchResults) GetIsAdvanced() bool { + if x != nil { + return x.IsAdvanced + } + return false +} + +func (x *SearchResults) GetAllSearchCount() int64 { + if x != nil { + return x.AllSearchCount + } + return 0 +} + +func (x *SearchResults) GetIsTopkReduce() bool { + if x != nil { + return x.IsTopkReduce + } + return false +} + +func (x *SearchResults) GetIsRecallEvaluation() bool { + if x != nil { + return x.IsRecallEvaluation + } + return false +} + +type CostAggregation struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ResponseTime int64 `protobuf:"varint,1,opt,name=responseTime,proto3" json:"responseTime,omitempty"` + ServiceTime int64 `protobuf:"varint,2,opt,name=serviceTime,proto3" json:"serviceTime,omitempty"` + TotalNQ int64 `protobuf:"varint,3,opt,name=totalNQ,proto3" json:"totalNQ,omitempty"` + TotalRelatedDataSize int64 `protobuf:"varint,4,opt,name=totalRelatedDataSize,proto3" json:"totalRelatedDataSize,omitempty"` +} + +func (x *CostAggregation) Reset() { + *x = CostAggregation{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CostAggregation) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CostAggregation) ProtoMessage() {} + +func (x *CostAggregation) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[16] + 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 CostAggregation.ProtoReflect.Descriptor instead. +func (*CostAggregation) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{16} +} + +func (x *CostAggregation) GetResponseTime() int64 { + if x != nil { + return x.ResponseTime + } + return 0 +} + +func (x *CostAggregation) GetServiceTime() int64 { + if x != nil { + return x.ServiceTime + } + return 0 +} + +func (x *CostAggregation) GetTotalNQ() int64 { + if x != nil { + return x.TotalNQ + } + return 0 +} + +func (x *CostAggregation) GetTotalRelatedDataSize() int64 { + if x != nil { + return x.TotalRelatedDataSize + } + return 0 +} + +type RetrieveRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + ReqID int64 `protobuf:"varint,2,opt,name=reqID,proto3" json:"reqID,omitempty"` + DbID int64 `protobuf:"varint,3,opt,name=dbID,proto3" json:"dbID,omitempty"` + CollectionID int64 `protobuf:"varint,4,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionIDs []int64 `protobuf:"varint,5,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` + SerializedExprPlan []byte `protobuf:"bytes,6,opt,name=serialized_expr_plan,json=serializedExprPlan,proto3" json:"serialized_expr_plan,omitempty"` + OutputFieldsId []int64 `protobuf:"varint,7,rep,packed,name=output_fields_id,json=outputFieldsId,proto3" json:"output_fields_id,omitempty"` + MvccTimestamp uint64 `protobuf:"varint,8,opt,name=mvcc_timestamp,json=mvccTimestamp,proto3" json:"mvcc_timestamp,omitempty"` + GuaranteeTimestamp uint64 `protobuf:"varint,9,opt,name=guarantee_timestamp,json=guaranteeTimestamp,proto3" json:"guarantee_timestamp,omitempty"` + TimeoutTimestamp uint64 `protobuf:"varint,10,opt,name=timeout_timestamp,json=timeoutTimestamp,proto3" json:"timeout_timestamp,omitempty"` + Limit int64 `protobuf:"varint,11,opt,name=limit,proto3" json:"limit,omitempty"` // Optional + IgnoreGrowing bool `protobuf:"varint,12,opt,name=ignoreGrowing,proto3" json:"ignoreGrowing,omitempty"` + IsCount bool `protobuf:"varint,13,opt,name=is_count,json=isCount,proto3" json:"is_count,omitempty"` + IterationExtensionReduceRate int64 `protobuf:"varint,14,opt,name=iteration_extension_reduce_rate,json=iterationExtensionReduceRate,proto3" json:"iteration_extension_reduce_rate,omitempty"` + Username string `protobuf:"bytes,15,opt,name=username,proto3" json:"username,omitempty"` + ReduceStopForBest bool `protobuf:"varint,16,opt,name=reduce_stop_for_best,json=reduceStopForBest,proto3" json:"reduce_stop_for_best,omitempty"` //deprecated + ReduceType int32 `protobuf:"varint,17,opt,name=reduce_type,json=reduceType,proto3" json:"reduce_type,omitempty"` +} + +func (x *RetrieveRequest) Reset() { + *x = RetrieveRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RetrieveRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RetrieveRequest) ProtoMessage() {} + +func (x *RetrieveRequest) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[17] + 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 RetrieveRequest.ProtoReflect.Descriptor instead. +func (*RetrieveRequest) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{17} +} + +func (x *RetrieveRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *RetrieveRequest) GetReqID() int64 { + if x != nil { + return x.ReqID + } + return 0 +} + +func (x *RetrieveRequest) GetDbID() int64 { + if x != nil { + return x.DbID + } + return 0 +} + +func (x *RetrieveRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *RetrieveRequest) GetPartitionIDs() []int64 { + if x != nil { + return x.PartitionIDs + } + return nil +} + +func (x *RetrieveRequest) GetSerializedExprPlan() []byte { + if x != nil { + return x.SerializedExprPlan + } + return nil +} + +func (x *RetrieveRequest) GetOutputFieldsId() []int64 { + if x != nil { + return x.OutputFieldsId + } + return nil +} + +func (x *RetrieveRequest) GetMvccTimestamp() uint64 { + if x != nil { + return x.MvccTimestamp + } + return 0 +} + +func (x *RetrieveRequest) GetGuaranteeTimestamp() uint64 { + if x != nil { + return x.GuaranteeTimestamp + } + return 0 +} + +func (x *RetrieveRequest) GetTimeoutTimestamp() uint64 { + if x != nil { + return x.TimeoutTimestamp + } + return 0 +} + +func (x *RetrieveRequest) GetLimit() int64 { + if x != nil { + return x.Limit + } + return 0 +} + +func (x *RetrieveRequest) GetIgnoreGrowing() bool { + if x != nil { + return x.IgnoreGrowing + } + return false +} + +func (x *RetrieveRequest) GetIsCount() bool { + if x != nil { + return x.IsCount + } + return false +} + +func (x *RetrieveRequest) GetIterationExtensionReduceRate() int64 { + if x != nil { + return x.IterationExtensionReduceRate + } + return 0 +} + +func (x *RetrieveRequest) GetUsername() string { + if x != nil { + return x.Username + } + return "" +} + +func (x *RetrieveRequest) GetReduceStopForBest() bool { + if x != nil { + return x.ReduceStopForBest + } + return false +} + +func (x *RetrieveRequest) GetReduceType() int32 { + if x != nil { + return x.ReduceType + } + return 0 +} + +type RetrieveResults struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + Status *commonpb.Status `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` + ReqID int64 `protobuf:"varint,3,opt,name=reqID,proto3" json:"reqID,omitempty"` + Ids *schemapb.IDs `protobuf:"bytes,4,opt,name=ids,proto3" json:"ids,omitempty"` + FieldsData []*schemapb.FieldData `protobuf:"bytes,5,rep,name=fields_data,json=fieldsData,proto3" json:"fields_data,omitempty"` + SealedSegmentIDsRetrieved []int64 `protobuf:"varint,6,rep,packed,name=sealed_segmentIDs_retrieved,json=sealedSegmentIDsRetrieved,proto3" json:"sealed_segmentIDs_retrieved,omitempty"` + ChannelIDsRetrieved []string `protobuf:"bytes,7,rep,name=channelIDs_retrieved,json=channelIDsRetrieved,proto3" json:"channelIDs_retrieved,omitempty"` + GlobalSealedSegmentIDs []int64 `protobuf:"varint,8,rep,packed,name=global_sealed_segmentIDs,json=globalSealedSegmentIDs,proto3" json:"global_sealed_segmentIDs,omitempty"` + // query request cost + CostAggregation *CostAggregation `protobuf:"bytes,13,opt,name=costAggregation,proto3" json:"costAggregation,omitempty"` + AllRetrieveCount int64 `protobuf:"varint,14,opt,name=all_retrieve_count,json=allRetrieveCount,proto3" json:"all_retrieve_count,omitempty"` + HasMoreResult bool `protobuf:"varint,15,opt,name=has_more_result,json=hasMoreResult,proto3" json:"has_more_result,omitempty"` +} + +func (x *RetrieveResults) Reset() { + *x = RetrieveResults{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RetrieveResults) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RetrieveResults) ProtoMessage() {} + +func (x *RetrieveResults) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[18] + 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 RetrieveResults.ProtoReflect.Descriptor instead. +func (*RetrieveResults) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{18} +} + +func (x *RetrieveResults) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *RetrieveResults) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *RetrieveResults) GetReqID() int64 { + if x != nil { + return x.ReqID + } + return 0 +} + +func (x *RetrieveResults) GetIds() *schemapb.IDs { + if x != nil { + return x.Ids + } + return nil +} + +func (x *RetrieveResults) GetFieldsData() []*schemapb.FieldData { + if x != nil { + return x.FieldsData + } + return nil +} + +func (x *RetrieveResults) GetSealedSegmentIDsRetrieved() []int64 { + if x != nil { + return x.SealedSegmentIDsRetrieved + } + return nil +} + +func (x *RetrieveResults) GetChannelIDsRetrieved() []string { + if x != nil { + return x.ChannelIDsRetrieved + } + return nil +} + +func (x *RetrieveResults) GetGlobalSealedSegmentIDs() []int64 { + if x != nil { + return x.GlobalSealedSegmentIDs + } + return nil +} + +func (x *RetrieveResults) GetCostAggregation() *CostAggregation { + if x != nil { + return x.CostAggregation + } + return nil +} + +func (x *RetrieveResults) GetAllRetrieveCount() int64 { + if x != nil { + return x.AllRetrieveCount + } + return 0 +} + +func (x *RetrieveResults) GetHasMoreResult() bool { + if x != nil { + return x.HasMoreResult + } + return false +} + +type LoadIndex struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + SegmentID int64 `protobuf:"varint,2,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + FieldName string `protobuf:"bytes,3,opt,name=fieldName,proto3" json:"fieldName,omitempty"` + FieldID int64 `protobuf:"varint,4,opt,name=fieldID,proto3" json:"fieldID,omitempty"` + IndexPaths []string `protobuf:"bytes,5,rep,name=index_paths,json=indexPaths,proto3" json:"index_paths,omitempty"` + IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,6,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"` +} + +func (x *LoadIndex) Reset() { + *x = LoadIndex{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[19] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *LoadIndex) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LoadIndex) ProtoMessage() {} + +func (x *LoadIndex) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[19] + 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 LoadIndex.ProtoReflect.Descriptor instead. +func (*LoadIndex) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{19} +} + +func (x *LoadIndex) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *LoadIndex) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *LoadIndex) GetFieldName() string { + if x != nil { + return x.FieldName + } + return "" +} + +func (x *LoadIndex) GetFieldID() int64 { + if x != nil { + return x.FieldID + } + return 0 +} + +func (x *LoadIndex) GetIndexPaths() []string { + if x != nil { + return x.IndexPaths + } + return nil +} + +func (x *LoadIndex) GetIndexParams() []*commonpb.KeyValuePair { + if x != nil { + return x.IndexParams + } + return nil +} + +type IndexStats struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,1,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"` + NumRelatedSegments int64 `protobuf:"varint,2,opt,name=num_related_segments,json=numRelatedSegments,proto3" json:"num_related_segments,omitempty"` +} + +func (x *IndexStats) Reset() { + *x = IndexStats{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[20] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *IndexStats) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*IndexStats) ProtoMessage() {} + +func (x *IndexStats) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[20] + 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 IndexStats.ProtoReflect.Descriptor instead. +func (*IndexStats) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{20} +} + +func (x *IndexStats) GetIndexParams() []*commonpb.KeyValuePair { + if x != nil { + return x.IndexParams + } + return nil +} + +func (x *IndexStats) GetNumRelatedSegments() int64 { + if x != nil { + return x.NumRelatedSegments + } + return 0 +} + +type FieldStats struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + FieldID int64 `protobuf:"varint,2,opt,name=fieldID,proto3" json:"fieldID,omitempty"` + IndexStats []*IndexStats `protobuf:"bytes,3,rep,name=index_stats,json=indexStats,proto3" json:"index_stats,omitempty"` +} + +func (x *FieldStats) Reset() { + *x = FieldStats{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[21] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *FieldStats) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FieldStats) ProtoMessage() {} + +func (x *FieldStats) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[21] + 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 FieldStats.ProtoReflect.Descriptor instead. +func (*FieldStats) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{21} +} + +func (x *FieldStats) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *FieldStats) GetFieldID() int64 { + if x != nil { + return x.FieldID + } + return 0 +} + +func (x *FieldStats) GetIndexStats() []*IndexStats { + if x != nil { + return x.IndexStats + } + return nil +} + +type SegmentStats struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SegmentID int64 `protobuf:"varint,1,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + MemorySize int64 `protobuf:"varint,2,opt,name=memory_size,json=memorySize,proto3" json:"memory_size,omitempty"` + NumRows int64 `protobuf:"varint,3,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"` + RecentlyModified bool `protobuf:"varint,4,opt,name=recently_modified,json=recentlyModified,proto3" json:"recently_modified,omitempty"` +} + +func (x *SegmentStats) Reset() { + *x = SegmentStats{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[22] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentStats) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentStats) ProtoMessage() {} + +func (x *SegmentStats) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[22] + 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 SegmentStats.ProtoReflect.Descriptor instead. +func (*SegmentStats) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{22} +} + +func (x *SegmentStats) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *SegmentStats) GetMemorySize() int64 { + if x != nil { + return x.MemorySize + } + return 0 +} + +func (x *SegmentStats) GetNumRows() int64 { + if x != nil { + return x.NumRows + } + return 0 +} + +func (x *SegmentStats) GetRecentlyModified() bool { + if x != nil { + return x.RecentlyModified + } + return false +} + +type ChannelTimeTickMsg struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + ChannelNames []string `protobuf:"bytes,2,rep,name=channelNames,proto3" json:"channelNames,omitempty"` + Timestamps []uint64 `protobuf:"varint,3,rep,packed,name=timestamps,proto3" json:"timestamps,omitempty"` + DefaultTimestamp uint64 `protobuf:"varint,4,opt,name=default_timestamp,json=defaultTimestamp,proto3" json:"default_timestamp,omitempty"` +} + +func (x *ChannelTimeTickMsg) Reset() { + *x = ChannelTimeTickMsg{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[23] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ChannelTimeTickMsg) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ChannelTimeTickMsg) ProtoMessage() {} + +func (x *ChannelTimeTickMsg) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[23] + 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 ChannelTimeTickMsg.ProtoReflect.Descriptor instead. +func (*ChannelTimeTickMsg) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{23} +} + +func (x *ChannelTimeTickMsg) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *ChannelTimeTickMsg) GetChannelNames() []string { + if x != nil { + return x.ChannelNames + } + return nil +} + +func (x *ChannelTimeTickMsg) GetTimestamps() []uint64 { + if x != nil { + return x.Timestamps + } + return nil +} + +func (x *ChannelTimeTickMsg) GetDefaultTimestamp() uint64 { + if x != nil { + return x.DefaultTimestamp + } + return 0 +} + +type CredentialInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Username string `protobuf:"bytes,1,opt,name=username,proto3" json:"username,omitempty"` + // encrypted by bcrypt (for higher security level) + EncryptedPassword string `protobuf:"bytes,2,opt,name=encrypted_password,json=encryptedPassword,proto3" json:"encrypted_password,omitempty"` + Tenant string `protobuf:"bytes,3,opt,name=tenant,proto3" json:"tenant,omitempty"` + IsSuper bool `protobuf:"varint,4,opt,name=is_super,json=isSuper,proto3" json:"is_super,omitempty"` + // encrypted by sha256 (for good performance in cache mapping) + Sha256Password string `protobuf:"bytes,5,opt,name=sha256_password,json=sha256Password,proto3" json:"sha256_password,omitempty"` +} + +func (x *CredentialInfo) Reset() { + *x = CredentialInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[24] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CredentialInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CredentialInfo) ProtoMessage() {} + +func (x *CredentialInfo) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[24] + 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 CredentialInfo.ProtoReflect.Descriptor instead. +func (*CredentialInfo) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{24} +} + +func (x *CredentialInfo) GetUsername() string { + if x != nil { + return x.Username + } + return "" +} + +func (x *CredentialInfo) GetEncryptedPassword() string { + if x != nil { + return x.EncryptedPassword + } + return "" +} + +func (x *CredentialInfo) GetTenant() string { + if x != nil { + return x.Tenant + } + return "" +} + +func (x *CredentialInfo) GetIsSuper() bool { + if x != nil { + return x.IsSuper + } + return false +} + +func (x *CredentialInfo) GetSha256Password() string { + if x != nil { + return x.Sha256Password + } + return "" +} + +type ListPolicyRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Not useful for now + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` +} + +func (x *ListPolicyRequest) Reset() { + *x = ListPolicyRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[25] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListPolicyRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListPolicyRequest) ProtoMessage() {} + +func (x *ListPolicyRequest) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[25] + 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 ListPolicyRequest.ProtoReflect.Descriptor instead. +func (*ListPolicyRequest) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{25} +} + +func (x *ListPolicyRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +type ListPolicyResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Contain error_code and reason + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + PolicyInfos []string `protobuf:"bytes,2,rep,name=policy_infos,json=policyInfos,proto3" json:"policy_infos,omitempty"` + UserRoles []string `protobuf:"bytes,3,rep,name=user_roles,json=userRoles,proto3" json:"user_roles,omitempty"` + PrivilegeGroups []*milvuspb.PrivilegeGroupInfo `protobuf:"bytes,4,rep,name=privilege_groups,json=privilegeGroups,proto3" json:"privilege_groups,omitempty"` +} + +func (x *ListPolicyResponse) Reset() { + *x = ListPolicyResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[26] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListPolicyResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListPolicyResponse) ProtoMessage() {} + +func (x *ListPolicyResponse) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[26] + 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 ListPolicyResponse.ProtoReflect.Descriptor instead. +func (*ListPolicyResponse) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{26} +} + +func (x *ListPolicyResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *ListPolicyResponse) GetPolicyInfos() []string { + if x != nil { + return x.PolicyInfos + } + return nil +} + +func (x *ListPolicyResponse) GetUserRoles() []string { + if x != nil { + return x.UserRoles + } + return nil +} + +func (x *ListPolicyResponse) GetPrivilegeGroups() []*milvuspb.PrivilegeGroupInfo { + if x != nil { + return x.PrivilegeGroups + } + return nil +} + +type ShowConfigurationsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + Pattern string `protobuf:"bytes,2,opt,name=pattern,proto3" json:"pattern,omitempty"` +} + +func (x *ShowConfigurationsRequest) Reset() { + *x = ShowConfigurationsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[27] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ShowConfigurationsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ShowConfigurationsRequest) ProtoMessage() {} + +func (x *ShowConfigurationsRequest) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[27] + 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 ShowConfigurationsRequest.ProtoReflect.Descriptor instead. +func (*ShowConfigurationsRequest) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{27} +} + +func (x *ShowConfigurationsRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *ShowConfigurationsRequest) GetPattern() string { + if x != nil { + return x.Pattern + } + return "" +} + +type ShowConfigurationsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + Configuations []*commonpb.KeyValuePair `protobuf:"bytes,2,rep,name=configuations,proto3" json:"configuations,omitempty"` +} + +func (x *ShowConfigurationsResponse) Reset() { + *x = ShowConfigurationsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[28] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ShowConfigurationsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ShowConfigurationsResponse) ProtoMessage() {} + +func (x *ShowConfigurationsResponse) ProtoReflect() protoreflect.Message { + mi := &file_internal_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 ShowConfigurationsResponse.ProtoReflect.Descriptor instead. +func (*ShowConfigurationsResponse) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{28} +} + +func (x *ShowConfigurationsResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *ShowConfigurationsResponse) GetConfiguations() []*commonpb.KeyValuePair { + if x != nil { + return x.Configuations + } + return nil +} + +type Rate struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Rt RateType `protobuf:"varint,1,opt,name=rt,proto3,enum=milvus.proto.internal.RateType" json:"rt,omitempty"` + R float64 `protobuf:"fixed64,2,opt,name=r,proto3" json:"r,omitempty"` +} + +func (x *Rate) Reset() { + *x = Rate{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[29] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Rate) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Rate) ProtoMessage() {} + +func (x *Rate) ProtoReflect() protoreflect.Message { + mi := &file_internal_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 Rate.ProtoReflect.Descriptor instead. +func (*Rate) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{29} +} + +func (x *Rate) GetRt() RateType { + if x != nil { + return x.Rt + } + return RateType_DDLCollection +} + +func (x *Rate) GetR() float64 { + if x != nil { + return x.R + } + return 0 +} + +type ImportFile struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Id int64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` + // A singular row-based file or multiple column-based files. + Paths []string `protobuf:"bytes,2,rep,name=paths,proto3" json:"paths,omitempty"` +} + +func (x *ImportFile) Reset() { + *x = ImportFile{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[30] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ImportFile) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ImportFile) ProtoMessage() {} + +func (x *ImportFile) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[30] + 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 ImportFile.ProtoReflect.Descriptor instead. +func (*ImportFile) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{30} +} + +func (x *ImportFile) GetId() int64 { + if x != nil { + return x.Id + } + return 0 +} + +func (x *ImportFile) GetPaths() []string { + if x != nil { + return x.Paths + } + return nil +} + +type ImportRequestInternal struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + DbID int64 `protobuf:"varint,1,opt,name=dbID,proto3" json:"dbID,omitempty"` + CollectionID int64 `protobuf:"varint,2,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + CollectionName string `protobuf:"bytes,3,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` + PartitionIDs []int64 `protobuf:"varint,4,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` + ChannelNames []string `protobuf:"bytes,5,rep,name=channel_names,json=channelNames,proto3" json:"channel_names,omitempty"` + Schema *schemapb.CollectionSchema `protobuf:"bytes,6,opt,name=schema,proto3" json:"schema,omitempty"` + Files []*ImportFile `protobuf:"bytes,7,rep,name=files,proto3" json:"files,omitempty"` + Options []*commonpb.KeyValuePair `protobuf:"bytes,8,rep,name=options,proto3" json:"options,omitempty"` +} + +func (x *ImportRequestInternal) Reset() { + *x = ImportRequestInternal{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[31] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ImportRequestInternal) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ImportRequestInternal) ProtoMessage() {} + +func (x *ImportRequestInternal) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[31] + 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 ImportRequestInternal.ProtoReflect.Descriptor instead. +func (*ImportRequestInternal) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{31} +} + +func (x *ImportRequestInternal) GetDbID() int64 { + if x != nil { + return x.DbID + } + return 0 +} + +func (x *ImportRequestInternal) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *ImportRequestInternal) GetCollectionName() string { + if x != nil { + return x.CollectionName + } + return "" +} + +func (x *ImportRequestInternal) GetPartitionIDs() []int64 { + if x != nil { + return x.PartitionIDs + } + return nil +} + +func (x *ImportRequestInternal) GetChannelNames() []string { + if x != nil { + return x.ChannelNames + } + return nil +} + +func (x *ImportRequestInternal) GetSchema() *schemapb.CollectionSchema { + if x != nil { + return x.Schema + } + return nil +} + +func (x *ImportRequestInternal) GetFiles() []*ImportFile { + if x != nil { + return x.Files + } + return nil +} + +func (x *ImportRequestInternal) GetOptions() []*commonpb.KeyValuePair { + if x != nil { + return x.Options + } + return nil +} + +type ImportRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + DbName string `protobuf:"bytes,1,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` + CollectionName string `protobuf:"bytes,2,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` + PartitionName string `protobuf:"bytes,3,opt,name=partition_name,json=partitionName,proto3" json:"partition_name,omitempty"` + Files []*ImportFile `protobuf:"bytes,4,rep,name=files,proto3" json:"files,omitempty"` + Options []*commonpb.KeyValuePair `protobuf:"bytes,5,rep,name=options,proto3" json:"options,omitempty"` +} + +func (x *ImportRequest) Reset() { + *x = ImportRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[32] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ImportRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ImportRequest) ProtoMessage() {} + +func (x *ImportRequest) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[32] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ImportRequest.ProtoReflect.Descriptor instead. +func (*ImportRequest) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{32} +} + +func (x *ImportRequest) GetDbName() string { + if x != nil { + return x.DbName + } + return "" +} + +func (x *ImportRequest) GetCollectionName() string { + if x != nil { + return x.CollectionName + } + return "" +} + +func (x *ImportRequest) GetPartitionName() string { + if x != nil { + return x.PartitionName + } + return "" +} + +func (x *ImportRequest) GetFiles() []*ImportFile { + if x != nil { + return x.Files + } + return nil +} + +func (x *ImportRequest) GetOptions() []*commonpb.KeyValuePair { + if x != nil { + return x.Options + } + return nil +} + +type ImportResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + JobID string `protobuf:"bytes,2,opt,name=jobID,proto3" json:"jobID,omitempty"` +} + +func (x *ImportResponse) Reset() { + *x = ImportResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[33] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ImportResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ImportResponse) ProtoMessage() {} + +func (x *ImportResponse) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[33] + 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 ImportResponse.ProtoReflect.Descriptor instead. +func (*ImportResponse) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{33} +} + +func (x *ImportResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *ImportResponse) GetJobID() string { + if x != nil { + return x.JobID + } + return "" +} + +type GetImportProgressRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + DbName string `protobuf:"bytes,1,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` + JobID string `protobuf:"bytes,2,opt,name=jobID,proto3" json:"jobID,omitempty"` +} + +func (x *GetImportProgressRequest) Reset() { + *x = GetImportProgressRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[34] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetImportProgressRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetImportProgressRequest) ProtoMessage() {} + +func (x *GetImportProgressRequest) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[34] + 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 GetImportProgressRequest.ProtoReflect.Descriptor instead. +func (*GetImportProgressRequest) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{34} +} + +func (x *GetImportProgressRequest) GetDbName() string { + if x != nil { + return x.DbName + } + return "" +} + +func (x *GetImportProgressRequest) GetJobID() string { + if x != nil { + return x.JobID + } + return "" +} + +type ImportTaskProgress struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + FileName string `protobuf:"bytes,1,opt,name=file_name,json=fileName,proto3" json:"file_name,omitempty"` + FileSize int64 `protobuf:"varint,2,opt,name=file_size,json=fileSize,proto3" json:"file_size,omitempty"` + Reason string `protobuf:"bytes,3,opt,name=reason,proto3" json:"reason,omitempty"` + Progress int64 `protobuf:"varint,4,opt,name=progress,proto3" json:"progress,omitempty"` + CompleteTime string `protobuf:"bytes,5,opt,name=complete_time,json=completeTime,proto3" json:"complete_time,omitempty"` + State string `protobuf:"bytes,6,opt,name=state,proto3" json:"state,omitempty"` + ImportedRows int64 `protobuf:"varint,7,opt,name=imported_rows,json=importedRows,proto3" json:"imported_rows,omitempty"` + TotalRows int64 `protobuf:"varint,8,opt,name=total_rows,json=totalRows,proto3" json:"total_rows,omitempty"` +} + +func (x *ImportTaskProgress) Reset() { + *x = ImportTaskProgress{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[35] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ImportTaskProgress) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ImportTaskProgress) ProtoMessage() {} + +func (x *ImportTaskProgress) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[35] + 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 ImportTaskProgress.ProtoReflect.Descriptor instead. +func (*ImportTaskProgress) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{35} +} + +func (x *ImportTaskProgress) GetFileName() string { + if x != nil { + return x.FileName + } + return "" +} + +func (x *ImportTaskProgress) GetFileSize() int64 { + if x != nil { + return x.FileSize + } + return 0 +} + +func (x *ImportTaskProgress) GetReason() string { + if x != nil { + return x.Reason + } + return "" +} + +func (x *ImportTaskProgress) GetProgress() int64 { + if x != nil { + return x.Progress + } + return 0 +} + +func (x *ImportTaskProgress) GetCompleteTime() string { + if x != nil { + return x.CompleteTime + } + return "" +} + +func (x *ImportTaskProgress) GetState() string { + if x != nil { + return x.State + } + return "" +} + +func (x *ImportTaskProgress) GetImportedRows() int64 { + if x != nil { + return x.ImportedRows + } + return 0 +} + +func (x *ImportTaskProgress) GetTotalRows() int64 { + if x != nil { + return x.TotalRows + } + return 0 +} + +type GetImportProgressResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + State ImportJobState `protobuf:"varint,2,opt,name=state,proto3,enum=milvus.proto.internal.ImportJobState" json:"state,omitempty"` + Reason string `protobuf:"bytes,3,opt,name=reason,proto3" json:"reason,omitempty"` + Progress int64 `protobuf:"varint,4,opt,name=progress,proto3" json:"progress,omitempty"` + CollectionName string `protobuf:"bytes,5,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` + CompleteTime string `protobuf:"bytes,6,opt,name=complete_time,json=completeTime,proto3" json:"complete_time,omitempty"` + TaskProgresses []*ImportTaskProgress `protobuf:"bytes,7,rep,name=task_progresses,json=taskProgresses,proto3" json:"task_progresses,omitempty"` + ImportedRows int64 `protobuf:"varint,8,opt,name=imported_rows,json=importedRows,proto3" json:"imported_rows,omitempty"` + TotalRows int64 `protobuf:"varint,9,opt,name=total_rows,json=totalRows,proto3" json:"total_rows,omitempty"` + StartTime string `protobuf:"bytes,10,opt,name=start_time,json=startTime,proto3" json:"start_time,omitempty"` +} + +func (x *GetImportProgressResponse) Reset() { + *x = GetImportProgressResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[36] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetImportProgressResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetImportProgressResponse) ProtoMessage() {} + +func (x *GetImportProgressResponse) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[36] + 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 GetImportProgressResponse.ProtoReflect.Descriptor instead. +func (*GetImportProgressResponse) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{36} +} + +func (x *GetImportProgressResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *GetImportProgressResponse) GetState() ImportJobState { + if x != nil { + return x.State + } + return ImportJobState_None +} + +func (x *GetImportProgressResponse) GetReason() string { + if x != nil { + return x.Reason + } + return "" +} + +func (x *GetImportProgressResponse) GetProgress() int64 { + if x != nil { + return x.Progress + } + return 0 +} + +func (x *GetImportProgressResponse) GetCollectionName() string { + if x != nil { + return x.CollectionName + } + return "" +} + +func (x *GetImportProgressResponse) GetCompleteTime() string { + if x != nil { + return x.CompleteTime + } + return "" +} + +func (x *GetImportProgressResponse) GetTaskProgresses() []*ImportTaskProgress { + if x != nil { + return x.TaskProgresses + } + return nil +} + +func (x *GetImportProgressResponse) GetImportedRows() int64 { + if x != nil { + return x.ImportedRows + } + return 0 +} + +func (x *GetImportProgressResponse) GetTotalRows() int64 { + if x != nil { + return x.TotalRows + } + return 0 +} + +func (x *GetImportProgressResponse) GetStartTime() string { + if x != nil { + return x.StartTime + } + return "" +} + +type ListImportsRequestInternal struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + DbID int64 `protobuf:"varint,1,opt,name=dbID,proto3" json:"dbID,omitempty"` + CollectionID int64 `protobuf:"varint,2,opt,name=collectionID,proto3" json:"collectionID,omitempty"` +} + +func (x *ListImportsRequestInternal) Reset() { + *x = ListImportsRequestInternal{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[37] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListImportsRequestInternal) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListImportsRequestInternal) ProtoMessage() {} + +func (x *ListImportsRequestInternal) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[37] + 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 ListImportsRequestInternal.ProtoReflect.Descriptor instead. +func (*ListImportsRequestInternal) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{37} +} + +func (x *ListImportsRequestInternal) GetDbID() int64 { + if x != nil { + return x.DbID + } + return 0 +} + +func (x *ListImportsRequestInternal) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +type ListImportsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + DbName string `protobuf:"bytes,1,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` + CollectionName string `protobuf:"bytes,2,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` +} + +func (x *ListImportsRequest) Reset() { + *x = ListImportsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[38] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListImportsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListImportsRequest) ProtoMessage() {} + +func (x *ListImportsRequest) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[38] + 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 ListImportsRequest.ProtoReflect.Descriptor instead. +func (*ListImportsRequest) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{38} +} + +func (x *ListImportsRequest) GetDbName() string { + if x != nil { + return x.DbName + } + return "" +} + +func (x *ListImportsRequest) GetCollectionName() string { + if x != nil { + return x.CollectionName + } + return "" +} + +type ListImportsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + JobIDs []string `protobuf:"bytes,2,rep,name=jobIDs,proto3" json:"jobIDs,omitempty"` + States []ImportJobState `protobuf:"varint,3,rep,packed,name=states,proto3,enum=milvus.proto.internal.ImportJobState" json:"states,omitempty"` + Reasons []string `protobuf:"bytes,4,rep,name=reasons,proto3" json:"reasons,omitempty"` + Progresses []int64 `protobuf:"varint,5,rep,packed,name=progresses,proto3" json:"progresses,omitempty"` + CollectionNames []string `protobuf:"bytes,6,rep,name=collection_names,json=collectionNames,proto3" json:"collection_names,omitempty"` +} + +func (x *ListImportsResponse) Reset() { + *x = ListImportsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[39] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListImportsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListImportsResponse) ProtoMessage() {} + +func (x *ListImportsResponse) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[39] + 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 ListImportsResponse.ProtoReflect.Descriptor instead. +func (*ListImportsResponse) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{39} +} + +func (x *ListImportsResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *ListImportsResponse) GetJobIDs() []string { + if x != nil { + return x.JobIDs + } + return nil +} + +func (x *ListImportsResponse) GetStates() []ImportJobState { + if x != nil { + return x.States + } + return nil +} + +func (x *ListImportsResponse) GetReasons() []string { + if x != nil { + return x.Reasons + } + return nil +} + +func (x *ListImportsResponse) GetProgresses() []int64 { + if x != nil { + return x.Progresses + } + return nil +} + +func (x *ListImportsResponse) GetCollectionNames() []string { + if x != nil { + return x.CollectionNames + } + return nil +} + +var File_internal_proto protoreflect.FileDescriptor + +var file_internal_proto_rawDesc = []byte{ + 0x0a, 0x0e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x12, 0x15, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, + 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x1a, 0x0c, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x0c, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x1a, 0x0c, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x22, 0x1b, 0x0a, 0x19, 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, 0x22, 0x1d, + 0x0a, 0x1b, 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, 0x22, 0x15, 0x0a, + 0x13, 0x47, 0x65, 0x74, 0x44, 0x64, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x22, 0x56, 0x0a, 0x08, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, + 0x12, 0x36, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 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, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x52, + 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x72, 0x6f, 0x6c, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x72, 0x6f, 0x6c, 0x65, 0x22, 0x6a, 0x0a, 0x0a, + 0x49, 0x6e, 0x69, 0x74, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, + 0x64, 0x65, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, + 0x49, 0x44, 0x12, 0x44, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x70, 0x61, 0x72, 0x61, + 0x6d, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, + 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0b, 0x73, 0x74, 0x61, + 0x72, 0x74, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x22, 0x59, 0x0a, 0x0a, 0x53, 0x74, 0x72, 0x69, + 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, + 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x12, 0x33, + 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x22, 0xad, 0x02, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, + 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, + 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x12, + 0x0a, 0x04, 0x64, 0x62, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, + 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x72, + 0x61, 0x76, 0x65, 0x6c, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x04, 0x52, 0x0f, 0x74, 0x72, 0x61, 0x76, 0x65, 0x6c, 0x54, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x2f, 0x0a, 0x13, 0x67, 0x75, 0x61, 0x72, 0x61, 0x6e, 0x74, + 0x65, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x04, 0x52, 0x12, 0x67, 0x75, 0x61, 0x72, 0x61, 0x6e, 0x74, 0x65, 0x65, 0x54, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x2b, 0x0a, 0x11, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, + 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x07, 0x20, 0x01, 0x28, + 0x04, 0x52, 0x10, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, + 0x61, 0x6d, 0x70, 0x22, 0xb7, 0x01, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, + 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 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, + 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x12, 0x37, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x03, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, + 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x73, 0x22, 0x9e, 0x01, + 0x0a, 0x12, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, + 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, + 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x61, 0x6c, 0x69, 0x61, + 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x73, + 0x0a, 0x10, 0x44, 0x72, 0x6f, 0x70, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, + 0x62, 0x61, 0x73, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, + 0x05, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x61, 0x6c, + 0x69, 0x61, 0x73, 0x22, 0x9d, 0x01, 0x0a, 0x11, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x41, 0x6c, 0x69, + 0x61, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, + 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x64, + 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, + 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, + 0x05, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x61, 0x6c, + 0x69, 0x61, 0x73, 0x22, 0xbf, 0x02, 0x0a, 0x12, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, + 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, + 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, + 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x17, 0x0a, 0x07, + 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, + 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, + 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1d, + 0x0a, 0x0a, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x09, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x12, 0x0a, + 0x04, 0x64, 0x62, 0x49, 0x44, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, + 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, + 0x44, 0x0a, 0x0c, 0x65, 0x78, 0x74, 0x72, 0x61, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, + 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, + 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0b, 0x65, 0x78, 0x74, 0x72, 0x61, 0x50, + 0x61, 0x72, 0x61, 0x6d, 0x73, 0x22, 0xa1, 0x03, 0x0a, 0x10, 0x53, 0x75, 0x62, 0x53, 0x65, 0x61, + 0x72, 0x63, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x64, 0x73, + 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x64, 0x73, 0x6c, 0x12, 0x2b, 0x0a, 0x11, + 0x70, 0x6c, 0x61, 0x63, 0x65, 0x68, 0x6f, 0x6c, 0x64, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, 0x75, + 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x10, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x68, 0x6f, + 0x6c, 0x64, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x37, 0x0a, 0x08, 0x64, 0x73, 0x6c, + 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x44, 0x73, 0x6c, 0x54, 0x79, 0x70, 0x65, 0x52, 0x07, 0x64, 0x73, 0x6c, 0x54, 0x79, + 0x70, 0x65, 0x12, 0x30, 0x0a, 0x14, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, + 0x5f, 0x65, 0x78, 0x70, 0x72, 0x5f, 0x70, 0x6c, 0x61, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, + 0x52, 0x12, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x45, 0x78, 0x70, 0x72, + 0x50, 0x6c, 0x61, 0x6e, 0x12, 0x0e, 0x0a, 0x02, 0x6e, 0x71, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x02, 0x6e, 0x71, 0x12, 0x22, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x44, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x6f, 0x70, 0x6b, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, 0x6f, 0x70, 0x6b, 0x12, 0x16, 0x0a, 0x06, + 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6f, 0x66, + 0x66, 0x73, 0x65, 0x74, 0x12, 0x1e, 0x0a, 0x0a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x54, 0x79, + 0x70, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, + 0x54, 0x79, 0x70, 0x65, 0x12, 0x29, 0x0a, 0x11, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x62, 0x79, + 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0e, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x42, 0x79, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x64, 0x12, + 0x1d, 0x0a, 0x0a, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x0b, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x09, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x19, + 0x0a, 0x08, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x64, 0x22, 0xa0, 0x08, 0x0a, 0x0d, 0x53, 0x65, + 0x61, 0x72, 0x63, 0x68, 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, 0x14, 0x0a, + 0x05, 0x72, 0x65, 0x71, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x72, 0x65, + 0x71, 0x49, 0x44, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x62, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, + 0x03, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x12, + 0x10, 0x0a, 0x03, 0x64, 0x73, 0x6c, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x64, 0x73, + 0x6c, 0x12, 0x2b, 0x0a, 0x11, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x68, 0x6f, 0x6c, 0x64, 0x65, 0x72, + 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x10, 0x70, 0x6c, + 0x61, 0x63, 0x65, 0x68, 0x6f, 0x6c, 0x64, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x37, + 0x0a, 0x08, 0x64, 0x73, 0x6c, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x44, 0x73, 0x6c, 0x54, 0x79, 0x70, 0x65, 0x52, 0x07, + 0x64, 0x73, 0x6c, 0x54, 0x79, 0x70, 0x65, 0x12, 0x30, 0x0a, 0x14, 0x73, 0x65, 0x72, 0x69, 0x61, + 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x5f, 0x70, 0x6c, 0x61, 0x6e, 0x18, + 0x09, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x12, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, + 0x64, 0x45, 0x78, 0x70, 0x72, 0x50, 0x6c, 0x61, 0x6e, 0x12, 0x28, 0x0a, 0x10, 0x6f, 0x75, 0x74, + 0x70, 0x75, 0x74, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x5f, 0x69, 0x64, 0x18, 0x0a, 0x20, + 0x03, 0x28, 0x03, 0x52, 0x0e, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x46, 0x69, 0x65, 0x6c, 0x64, + 0x73, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x6d, 0x76, 0x63, 0x63, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x6d, 0x76, 0x63, + 0x63, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x2f, 0x0a, 0x13, 0x67, 0x75, + 0x61, 0x72, 0x61, 0x6e, 0x74, 0x65, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x04, 0x52, 0x12, 0x67, 0x75, 0x61, 0x72, 0x61, 0x6e, 0x74, + 0x65, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x2b, 0x0a, 0x11, 0x74, + 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, + 0x18, 0x0d, 0x20, 0x01, 0x28, 0x04, 0x52, 0x10, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x54, + 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x0e, 0x0a, 0x02, 0x6e, 0x71, 0x18, 0x0e, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x6e, 0x71, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x6f, 0x70, 0x6b, + 0x18, 0x0f, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, 0x6f, 0x70, 0x6b, 0x12, 0x1e, 0x0a, 0x0a, + 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x54, 0x79, 0x70, 0x65, 0x18, 0x10, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x54, 0x79, 0x70, 0x65, 0x12, 0x24, 0x0a, 0x0d, + 0x69, 0x67, 0x6e, 0x6f, 0x72, 0x65, 0x47, 0x72, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x18, 0x11, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x0d, 0x69, 0x67, 0x6e, 0x6f, 0x72, 0x65, 0x47, 0x72, 0x6f, 0x77, 0x69, + 0x6e, 0x67, 0x12, 0x1a, 0x0a, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x12, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x42, + 0x0a, 0x08, 0x73, 0x75, 0x62, 0x5f, 0x72, 0x65, 0x71, 0x73, 0x18, 0x13, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x75, 0x62, 0x53, 0x65, 0x61, 0x72, + 0x63, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x07, 0x73, 0x75, 0x62, 0x52, 0x65, + 0x71, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x69, 0x73, 0x5f, 0x61, 0x64, 0x76, 0x61, 0x6e, 0x63, 0x65, + 0x64, 0x18, 0x14, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x69, 0x73, 0x41, 0x64, 0x76, 0x61, 0x6e, + 0x63, 0x65, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x15, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x52, 0x0a, 0x11, 0x63, + 0x6f, 0x6e, 0x73, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x63, 0x79, 0x5f, 0x6c, 0x65, 0x76, 0x65, 0x6c, + 0x18, 0x16, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x43, 0x6f, 0x6e, + 0x73, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x63, 0x79, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x10, 0x63, + 0x6f, 0x6e, 0x73, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x63, 0x79, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, + 0x29, 0x0a, 0x11, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x62, 0x79, 0x5f, 0x66, 0x69, 0x65, 0x6c, + 0x64, 0x5f, 0x69, 0x64, 0x18, 0x17, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x67, 0x72, 0x6f, 0x75, + 0x70, 0x42, 0x79, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x67, 0x72, + 0x6f, 0x75, 0x70, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x18, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, + 0x67, 0x72, 0x6f, 0x75, 0x70, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x66, 0x69, 0x65, + 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x19, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, + 0x6c, 0x64, 0x49, 0x64, 0x12, 0x24, 0x0a, 0x0e, 0x69, 0x73, 0x5f, 0x74, 0x6f, 0x70, 0x6b, 0x5f, + 0x72, 0x65, 0x64, 0x75, 0x63, 0x65, 0x18, 0x1a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, 0x73, + 0x54, 0x6f, 0x70, 0x6b, 0x52, 0x65, 0x64, 0x75, 0x63, 0x65, 0x12, 0x30, 0x0a, 0x14, 0x69, 0x73, + 0x5f, 0x72, 0x65, 0x63, 0x61, 0x6c, 0x6c, 0x5f, 0x65, 0x76, 0x61, 0x6c, 0x75, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x18, 0x1b, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x69, 0x73, 0x52, 0x65, 0x63, 0x61, + 0x6c, 0x6c, 0x45, 0x76, 0x61, 0x6c, 0x75, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xf6, 0x01, 0x0a, + 0x10, 0x53, 0x75, 0x62, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, + 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x5f, 0x74, 0x79, 0x70, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x54, 0x79, + 0x70, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x6e, 0x75, 0x6d, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x69, 0x65, + 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x6e, 0x75, 0x6d, 0x51, 0x75, 0x65, 0x72, + 0x69, 0x65, 0x73, 0x12, 0x13, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x5f, 0x6b, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x04, 0x74, 0x6f, 0x70, 0x4b, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x6c, 0x69, 0x63, + 0x65, 0x64, 0x5f, 0x62, 0x6c, 0x6f, 0x62, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0a, 0x73, + 0x6c, 0x69, 0x63, 0x65, 0x64, 0x42, 0x6c, 0x6f, 0x62, 0x12, 0x28, 0x0a, 0x10, 0x73, 0x6c, 0x69, + 0x63, 0x65, 0x64, 0x5f, 0x6e, 0x75, 0x6d, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0e, 0x73, 0x6c, 0x69, 0x63, 0x65, 0x64, 0x4e, 0x75, 0x6d, 0x43, 0x6f, + 0x75, 0x6e, 0x74, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x6c, 0x69, 0x63, 0x65, 0x64, 0x5f, 0x6f, 0x66, + 0x66, 0x73, 0x65, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x73, 0x6c, 0x69, 0x63, + 0x65, 0x64, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x72, 0x65, 0x71, 0x5f, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x72, 0x65, 0x71, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x22, 0xd9, 0x07, 0x0a, 0x0d, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, + 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 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, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x14, + 0x0a, 0x05, 0x72, 0x65, 0x71, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x72, + 0x65, 0x71, 0x49, 0x44, 0x12, 0x1f, 0x0a, 0x0b, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x5f, 0x74, + 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x6d, 0x65, 0x74, 0x72, 0x69, + 0x63, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x6e, 0x75, 0x6d, 0x5f, 0x71, 0x75, 0x65, + 0x72, 0x69, 0x65, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x6e, 0x75, 0x6d, 0x51, + 0x75, 0x65, 0x72, 0x69, 0x65, 0x73, 0x12, 0x13, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x5f, 0x6b, 0x18, + 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, 0x6f, 0x70, 0x4b, 0x12, 0x3c, 0x0a, 0x1a, 0x73, + 0x65, 0x61, 0x6c, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, + 0x5f, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x65, 0x64, 0x18, 0x07, 0x20, 0x03, 0x28, 0x03, 0x52, + 0x18, 0x73, 0x65, 0x61, 0x6c, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, + 0x73, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x65, 0x64, 0x12, 0x2f, 0x0a, 0x13, 0x63, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x44, 0x73, 0x5f, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x65, 0x64, + 0x18, 0x08, 0x20, 0x03, 0x28, 0x09, 0x52, 0x12, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, + 0x44, 0x73, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x65, 0x64, 0x12, 0x38, 0x0a, 0x18, 0x67, 0x6c, + 0x6f, 0x62, 0x61, 0x6c, 0x5f, 0x73, 0x65, 0x61, 0x6c, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x03, 0x52, 0x16, 0x67, 0x6c, + 0x6f, 0x62, 0x61, 0x6c, 0x53, 0x65, 0x61, 0x6c, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x44, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x6c, 0x69, 0x63, 0x65, 0x64, 0x5f, 0x62, + 0x6c, 0x6f, 0x62, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0a, 0x73, 0x6c, 0x69, 0x63, 0x65, + 0x64, 0x42, 0x6c, 0x6f, 0x62, 0x12, 0x28, 0x0a, 0x10, 0x73, 0x6c, 0x69, 0x63, 0x65, 0x64, 0x5f, + 0x6e, 0x75, 0x6d, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0e, 0x73, 0x6c, 0x69, 0x63, 0x65, 0x64, 0x4e, 0x75, 0x6d, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, + 0x23, 0x0a, 0x0d, 0x73, 0x6c, 0x69, 0x63, 0x65, 0x64, 0x5f, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, + 0x18, 0x0c, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x73, 0x6c, 0x69, 0x63, 0x65, 0x64, 0x4f, 0x66, + 0x66, 0x73, 0x65, 0x74, 0x12, 0x50, 0x0a, 0x0f, 0x63, 0x6f, 0x73, 0x74, 0x41, 0x67, 0x67, 0x72, + 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, + 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x43, 0x6f, 0x73, 0x74, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0f, 0x63, 0x6f, 0x73, 0x74, 0x41, 0x67, 0x67, 0x72, 0x65, + 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x5b, 0x0a, 0x0d, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 0x73, 0x5f, 0x6d, 0x76, 0x63, 0x63, 0x18, 0x0e, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 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, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x4d, 0x76, 0x63, 0x63, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x4d, + 0x76, 0x63, 0x63, 0x12, 0x48, 0x0a, 0x0b, 0x73, 0x75, 0x62, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, + 0x74, 0x73, 0x18, 0x0f, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, + 0x2e, 0x53, 0x75, 0x62, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, + 0x73, 0x52, 0x0a, 0x73, 0x75, 0x62, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x1f, 0x0a, + 0x0b, 0x69, 0x73, 0x5f, 0x61, 0x64, 0x76, 0x61, 0x6e, 0x63, 0x65, 0x64, 0x18, 0x10, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0a, 0x69, 0x73, 0x41, 0x64, 0x76, 0x61, 0x6e, 0x63, 0x65, 0x64, 0x12, 0x28, + 0x0a, 0x10, 0x61, 0x6c, 0x6c, 0x5f, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x5f, 0x63, 0x6f, 0x75, + 0x6e, 0x74, 0x18, 0x11, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x61, 0x6c, 0x6c, 0x53, 0x65, 0x61, + 0x72, 0x63, 0x68, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x24, 0x0a, 0x0e, 0x69, 0x73, 0x5f, 0x74, + 0x6f, 0x70, 0x6b, 0x5f, 0x72, 0x65, 0x64, 0x75, 0x63, 0x65, 0x18, 0x12, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x0c, 0x69, 0x73, 0x54, 0x6f, 0x70, 0x6b, 0x52, 0x65, 0x64, 0x75, 0x63, 0x65, 0x12, 0x30, + 0x0a, 0x14, 0x69, 0x73, 0x5f, 0x72, 0x65, 0x63, 0x61, 0x6c, 0x6c, 0x5f, 0x65, 0x76, 0x61, 0x6c, + 0x75, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x13, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x69, 0x73, + 0x52, 0x65, 0x63, 0x61, 0x6c, 0x6c, 0x45, 0x76, 0x61, 0x6c, 0x75, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x1a, 0x3f, 0x0a, 0x11, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x4d, 0x76, 0x63, 0x63, + 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, 0x04, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x22, 0xa5, 0x01, 0x0a, 0x0f, 0x43, 0x6f, 0x73, 0x74, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x22, 0x0a, 0x0c, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x54, 0x69, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x72, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x73, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, + 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x74, + 0x6f, 0x74, 0x61, 0x6c, 0x4e, 0x51, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x74, 0x6f, + 0x74, 0x61, 0x6c, 0x4e, 0x51, 0x12, 0x32, 0x0a, 0x14, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x52, 0x65, + 0x6c, 0x61, 0x74, 0x65, 0x64, 0x44, 0x61, 0x74, 0x61, 0x53, 0x69, 0x7a, 0x65, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x14, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x65, + 0x64, 0x44, 0x61, 0x74, 0x61, 0x53, 0x69, 0x7a, 0x65, 0x22, 0xa2, 0x05, 0x0a, 0x0f, 0x52, 0x65, + 0x74, 0x72, 0x69, 0x65, 0x76, 0x65, 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, + 0x14, 0x0a, 0x05, 0x72, 0x65, 0x71, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, + 0x72, 0x65, 0x71, 0x49, 0x44, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x62, 0x49, 0x44, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x22, 0x0a, + 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x05, 0x20, + 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, + 0x73, 0x12, 0x30, 0x0a, 0x14, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, + 0x65, 0x78, 0x70, 0x72, 0x5f, 0x70, 0x6c, 0x61, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0c, 0x52, + 0x12, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x45, 0x78, 0x70, 0x72, 0x50, + 0x6c, 0x61, 0x6e, 0x12, 0x28, 0x0a, 0x10, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x66, 0x69, + 0x65, 0x6c, 0x64, 0x73, 0x5f, 0x69, 0x64, 0x18, 0x07, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0e, 0x6f, + 0x75, 0x74, 0x70, 0x75, 0x74, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x49, 0x64, 0x12, 0x25, 0x0a, + 0x0e, 0x6d, 0x76, 0x63, 0x63, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, + 0x08, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x6d, 0x76, 0x63, 0x63, 0x54, 0x69, 0x6d, 0x65, 0x73, + 0x74, 0x61, 0x6d, 0x70, 0x12, 0x2f, 0x0a, 0x13, 0x67, 0x75, 0x61, 0x72, 0x61, 0x6e, 0x74, 0x65, + 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x09, 0x20, 0x01, 0x28, + 0x04, 0x52, 0x12, 0x67, 0x75, 0x61, 0x72, 0x61, 0x6e, 0x74, 0x65, 0x65, 0x54, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x2b, 0x0a, 0x11, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, + 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x04, + 0x52, 0x10, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, + 0x6d, 0x70, 0x12, 0x14, 0x0a, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x0b, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x12, 0x24, 0x0a, 0x0d, 0x69, 0x67, 0x6e, 0x6f, + 0x72, 0x65, 0x47, 0x72, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x0d, 0x69, 0x67, 0x6e, 0x6f, 0x72, 0x65, 0x47, 0x72, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x12, 0x19, + 0x0a, 0x08, 0x69, 0x73, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x07, 0x69, 0x73, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x45, 0x0a, 0x1f, 0x69, 0x74, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, + 0x5f, 0x72, 0x65, 0x64, 0x75, 0x63, 0x65, 0x5f, 0x72, 0x61, 0x74, 0x65, 0x18, 0x0e, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x1c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x78, 0x74, + 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x64, 0x75, 0x63, 0x65, 0x52, 0x61, 0x74, 0x65, + 0x12, 0x1a, 0x0a, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x0f, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x2f, 0x0a, 0x14, + 0x72, 0x65, 0x64, 0x75, 0x63, 0x65, 0x5f, 0x73, 0x74, 0x6f, 0x70, 0x5f, 0x66, 0x6f, 0x72, 0x5f, + 0x62, 0x65, 0x73, 0x74, 0x18, 0x10, 0x20, 0x01, 0x28, 0x08, 0x52, 0x11, 0x72, 0x65, 0x64, 0x75, + 0x63, 0x65, 0x53, 0x74, 0x6f, 0x70, 0x46, 0x6f, 0x72, 0x42, 0x65, 0x73, 0x74, 0x12, 0x1f, 0x0a, + 0x0b, 0x72, 0x65, 0x64, 0x75, 0x63, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x11, 0x20, 0x01, + 0x28, 0x05, 0x52, 0x0a, 0x72, 0x65, 0x64, 0x75, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x22, 0xd0, + 0x04, 0x0a, 0x0f, 0x52, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, + 0x74, 0x73, 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, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x72, 0x65, 0x71, + 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x72, 0x65, 0x71, 0x49, 0x44, 0x12, + 0x2a, 0x0a, 0x03, 0x69, 0x64, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x2e, 0x49, 0x44, 0x73, 0x52, 0x03, 0x69, 0x64, 0x73, 0x12, 0x3f, 0x0a, 0x0b, 0x66, + 0x69, 0x65, 0x6c, 0x64, 0x73, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x44, 0x61, 0x74, 0x61, + 0x52, 0x0a, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x44, 0x61, 0x74, 0x61, 0x12, 0x3e, 0x0a, 0x1b, + 0x73, 0x65, 0x61, 0x6c, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, + 0x73, 0x5f, 0x72, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x65, 0x64, 0x18, 0x06, 0x20, 0x03, 0x28, + 0x03, 0x52, 0x19, 0x73, 0x65, 0x61, 0x6c, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x49, 0x44, 0x73, 0x52, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x65, 0x64, 0x12, 0x31, 0x0a, 0x14, + 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x44, 0x73, 0x5f, 0x72, 0x65, 0x74, 0x72, 0x69, + 0x65, 0x76, 0x65, 0x64, 0x18, 0x07, 0x20, 0x03, 0x28, 0x09, 0x52, 0x13, 0x63, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x49, 0x44, 0x73, 0x52, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x65, 0x64, 0x12, + 0x38, 0x0a, 0x18, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x5f, 0x73, 0x65, 0x61, 0x6c, 0x65, 0x64, + 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, + 0x03, 0x52, 0x16, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x53, 0x65, 0x61, 0x6c, 0x65, 0x64, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, 0x50, 0x0a, 0x0f, 0x63, 0x6f, 0x73, + 0x74, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x0d, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x43, 0x6f, 0x73, 0x74, 0x41, + 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0f, 0x63, 0x6f, 0x73, 0x74, + 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2c, 0x0a, 0x12, 0x61, + 0x6c, 0x6c, 0x5f, 0x72, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x65, 0x5f, 0x63, 0x6f, 0x75, 0x6e, + 0x74, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x61, 0x6c, 0x6c, 0x52, 0x65, 0x74, 0x72, + 0x69, 0x65, 0x76, 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x26, 0x0a, 0x0f, 0x68, 0x61, 0x73, + 0x5f, 0x6d, 0x6f, 0x72, 0x65, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x0f, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0d, 0x68, 0x61, 0x73, 0x4d, 0x6f, 0x72, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, + 0x74, 0x22, 0xfa, 0x01, 0x0a, 0x09, 0x4c, 0x6f, 0x61, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, + 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, + 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, + 0x1c, 0x0a, 0x09, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x09, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x18, 0x0a, + 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, + 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x1f, 0x0a, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x5f, 0x70, 0x61, 0x74, 0x68, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x50, 0x61, 0x74, 0x68, 0x73, 0x12, 0x44, 0x0a, 0x0c, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, + 0x72, 0x52, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x22, 0x84, + 0x01, 0x0a, 0x0a, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x44, 0x0a, + 0x0c, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, + 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x50, 0x61, 0x72, + 0x61, 0x6d, 0x73, 0x12, 0x30, 0x0a, 0x14, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x65, 0x6c, 0x61, 0x74, + 0x65, 0x64, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x12, 0x6e, 0x75, 0x6d, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x65, 0x64, 0x53, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x8e, 0x01, 0x0a, 0x0a, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x53, + 0x74, 0x61, 0x74, 0x73, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x65, 0x6c, + 0x64, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, + 0x49, 0x44, 0x12, 0x42, 0x0a, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x73, 0x74, 0x61, 0x74, + 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x0a, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x53, 0x74, 0x61, 0x74, 0x73, 0x22, 0x95, 0x01, 0x0a, 0x0c, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x1f, 0x0a, 0x0b, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x5f, + 0x73, 0x69, 0x7a, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x6d, 0x65, 0x6d, 0x6f, + 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, + 0x77, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, + 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x72, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x6c, 0x79, 0x5f, 0x6d, 0x6f, + 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x72, 0x65, + 0x63, 0x65, 0x6e, 0x74, 0x6c, 0x79, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x22, 0xb7, + 0x01, 0x0a, 0x12, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, + 0x63, 0x6b, 0x4d, 0x73, 0x67, 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, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x63, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x74, + 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x04, 0x52, + 0x0a, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x64, + 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x10, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x54, + 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x22, 0xb7, 0x01, 0x0a, 0x0e, 0x43, 0x72, 0x65, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1a, 0x0a, 0x08, 0x75, + 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x75, + 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x2d, 0x0a, 0x12, 0x65, 0x6e, 0x63, 0x72, 0x79, + 0x70, 0x74, 0x65, 0x64, 0x5f, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x11, 0x65, 0x6e, 0x63, 0x72, 0x79, 0x70, 0x74, 0x65, 0x64, 0x50, 0x61, + 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x12, 0x19, + 0x0a, 0x08, 0x69, 0x73, 0x5f, 0x73, 0x75, 0x70, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x07, 0x69, 0x73, 0x53, 0x75, 0x70, 0x65, 0x72, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x68, 0x61, + 0x32, 0x35, 0x36, 0x5f, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0e, 0x73, 0x68, 0x61, 0x32, 0x35, 0x36, 0x50, 0x61, 0x73, 0x73, 0x77, 0x6f, + 0x72, 0x64, 0x22, 0x45, 0x0a, 0x11, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, + 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x22, 0xdf, 0x01, 0x0a, 0x12, 0x4c, 0x69, + 0x73, 0x74, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x5f, + 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0b, 0x70, 0x6f, 0x6c, + 0x69, 0x63, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x75, 0x73, 0x65, 0x72, + 0x5f, 0x72, 0x6f, 0x6c, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x75, 0x73, + 0x65, 0x72, 0x52, 0x6f, 0x6c, 0x65, 0x73, 0x12, 0x52, 0x0a, 0x10, 0x70, 0x72, 0x69, 0x76, 0x69, + 0x6c, 0x65, 0x67, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x50, 0x72, 0x69, 0x76, 0x69, 0x6c, 0x65, 0x67, + 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0f, 0x70, 0x72, 0x69, 0x76, + 0x69, 0x6c, 0x65, 0x67, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x22, 0x67, 0x0a, 0x19, 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, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, + 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, + 0x74, 0x74, 0x65, 0x72, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x70, 0x61, 0x74, + 0x74, 0x65, 0x72, 0x6e, 0x22, 0x9a, 0x01, 0x0a, 0x1a, 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, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x47, 0x0a, 0x0d, 0x63, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x75, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, + 0x69, 0x72, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x22, 0x45, 0x0a, 0x04, 0x52, 0x61, 0x74, 0x65, 0x12, 0x2f, 0x0a, 0x02, 0x72, 0x74, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x52, 0x61, + 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x02, 0x72, 0x74, 0x12, 0x0c, 0x0a, 0x01, 0x72, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x01, 0x52, 0x01, 0x72, 0x22, 0x32, 0x0a, 0x0a, 0x49, 0x6d, 0x70, 0x6f, + 0x72, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x70, 0x61, 0x74, 0x68, 0x73, 0x18, + 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x70, 0x61, 0x74, 0x68, 0x73, 0x22, 0xf6, 0x02, 0x0a, + 0x15, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x6e, + 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x62, 0x49, 0x44, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x44, 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, 0x27, + 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x63, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x0c, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x73, + 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x06, 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, + 0x37, 0x0a, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, + 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x46, 0x69, 0x6c, + 0x65, 0x52, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x12, 0x3b, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x07, 0x6f, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xee, 0x01, 0x0a, 0x0d, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, + 0x12, 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0d, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, + 0x12, 0x37, 0x0a, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, + 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x46, 0x69, + 0x6c, 0x65, 0x52, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x12, 0x3b, 0x0a, 0x07, 0x6f, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x07, 0x6f, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x5b, 0x0a, 0x0e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x14, 0x0a, + 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f, + 0x62, 0x49, 0x44, 0x22, 0x49, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, + 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x6a, 0x6f, 0x62, 0x49, + 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x22, 0x81, + 0x02, 0x0a, 0x12, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x50, 0x72, 0x6f, + 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x66, 0x69, 0x6c, 0x65, 0x4e, 0x61, + 0x6d, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x66, 0x69, 0x6c, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x12, + 0x16, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x72, 0x6f, 0x67, 0x72, + 0x65, 0x73, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x67, 0x72, + 0x65, 0x73, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x5f, + 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6d, 0x70, + 0x6c, 0x65, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, + 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x23, + 0x0a, 0x0d, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, + 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x52, + 0x6f, 0x77, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x72, 0x6f, 0x77, + 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x52, 0x6f, + 0x77, 0x73, 0x22, 0xc6, 0x03, 0x0a, 0x19, 0x47, 0x65, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, + 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x3b, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, + 0x6f, 0x72, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, + 0x74, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x72, + 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x70, 0x72, + 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0e, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, + 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, + 0x54, 0x69, 0x6d, 0x65, 0x12, 0x52, 0x0a, 0x0f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x70, 0x72, 0x6f, + 0x67, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x29, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, + 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, 0x6b, + 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x0e, 0x74, 0x61, 0x73, 0x6b, 0x50, 0x72, + 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6d, 0x70, 0x6f, + 0x72, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0c, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x1d, 0x0a, + 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x09, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x1d, 0x0a, 0x0a, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x22, 0x54, 0x0a, 0x1a, 0x4c, + 0x69, 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x62, 0x49, + 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x44, 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, 0x22, 0x56, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, + 0x12, 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x86, 0x02, 0x0a, 0x13, 0x4c, 0x69, + 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, + 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x73, 0x12, 0x3d, + 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0e, 0x32, 0x25, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, + 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x4a, 0x6f, 0x62, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x18, 0x0a, + 0x07, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, + 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x67, 0x72, + 0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x70, 0x72, 0x6f, + 0x67, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x12, 0x29, 0x0a, 0x10, 0x63, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, + 0x09, 0x52, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, + 0x65, 0x73, 0x2a, 0x45, 0x0a, 0x09, 0x52, 0x61, 0x74, 0x65, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x12, + 0x0b, 0x0a, 0x07, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, + 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x43, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x03, 0x2a, 0xb9, 0x01, 0x0a, 0x08, 0x52, 0x61, + 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x44, 0x4c, 0x43, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x00, 0x12, 0x10, 0x0a, 0x0c, 0x44, 0x44, 0x4c, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x01, 0x12, 0x0c, 0x0a, 0x08, 0x44, + 0x44, 0x4c, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x10, 0x02, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x44, 0x4c, + 0x46, 0x6c, 0x75, 0x73, 0x68, 0x10, 0x03, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x44, 0x4c, 0x43, 0x6f, + 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x4d, + 0x4c, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x10, 0x05, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x4d, 0x4c, + 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x06, 0x12, 0x0f, 0x0a, 0x0b, 0x44, 0x4d, 0x4c, 0x42, + 0x75, 0x6c, 0x6b, 0x4c, 0x6f, 0x61, 0x64, 0x10, 0x07, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x51, 0x4c, + 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x10, 0x08, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x51, 0x4c, 0x51, + 0x75, 0x65, 0x72, 0x79, 0x10, 0x09, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x4d, 0x4c, 0x55, 0x70, 0x73, + 0x65, 0x72, 0x74, 0x10, 0x0a, 0x2a, 0x81, 0x01, 0x0a, 0x0e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, + 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e, 0x65, + 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x10, 0x01, 0x12, + 0x10, 0x0a, 0x0c, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x10, + 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x10, 0x03, + 0x12, 0x0a, 0x0a, 0x06, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, + 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x10, 0x05, 0x12, 0x11, 0x0a, 0x0d, 0x49, + 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x69, 0x6e, 0x67, 0x10, 0x06, 0x12, 0x09, + 0x0a, 0x05, 0x53, 0x74, 0x61, 0x74, 0x73, 0x10, 0x07, 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, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x62, 0x06, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_internal_proto_rawDescOnce sync.Once + file_internal_proto_rawDescData = file_internal_proto_rawDesc +) + +func file_internal_proto_rawDescGZIP() []byte { + file_internal_proto_rawDescOnce.Do(func() { + file_internal_proto_rawDescData = protoimpl.X.CompressGZIP(file_internal_proto_rawDescData) + }) + return file_internal_proto_rawDescData +} + +var file_internal_proto_enumTypes = make([]protoimpl.EnumInfo, 3) +var file_internal_proto_msgTypes = make([]protoimpl.MessageInfo, 41) +var file_internal_proto_goTypes = []interface{}{ + (RateScope)(0), // 0: milvus.proto.internal.RateScope + (RateType)(0), // 1: milvus.proto.internal.RateType + (ImportJobState)(0), // 2: milvus.proto.internal.ImportJobState + (*GetTimeTickChannelRequest)(nil), // 3: milvus.proto.internal.GetTimeTickChannelRequest + (*GetStatisticsChannelRequest)(nil), // 4: milvus.proto.internal.GetStatisticsChannelRequest + (*GetDdChannelRequest)(nil), // 5: milvus.proto.internal.GetDdChannelRequest + (*NodeInfo)(nil), // 6: milvus.proto.internal.NodeInfo + (*InitParams)(nil), // 7: milvus.proto.internal.InitParams + (*StringList)(nil), // 8: milvus.proto.internal.StringList + (*GetStatisticsRequest)(nil), // 9: milvus.proto.internal.GetStatisticsRequest + (*GetStatisticsResponse)(nil), // 10: milvus.proto.internal.GetStatisticsResponse + (*CreateAliasRequest)(nil), // 11: milvus.proto.internal.CreateAliasRequest + (*DropAliasRequest)(nil), // 12: milvus.proto.internal.DropAliasRequest + (*AlterAliasRequest)(nil), // 13: milvus.proto.internal.AlterAliasRequest + (*CreateIndexRequest)(nil), // 14: milvus.proto.internal.CreateIndexRequest + (*SubSearchRequest)(nil), // 15: milvus.proto.internal.SubSearchRequest + (*SearchRequest)(nil), // 16: milvus.proto.internal.SearchRequest + (*SubSearchResults)(nil), // 17: milvus.proto.internal.SubSearchResults + (*SearchResults)(nil), // 18: milvus.proto.internal.SearchResults + (*CostAggregation)(nil), // 19: milvus.proto.internal.CostAggregation + (*RetrieveRequest)(nil), // 20: milvus.proto.internal.RetrieveRequest + (*RetrieveResults)(nil), // 21: milvus.proto.internal.RetrieveResults + (*LoadIndex)(nil), // 22: milvus.proto.internal.LoadIndex + (*IndexStats)(nil), // 23: milvus.proto.internal.IndexStats + (*FieldStats)(nil), // 24: milvus.proto.internal.FieldStats + (*SegmentStats)(nil), // 25: milvus.proto.internal.SegmentStats + (*ChannelTimeTickMsg)(nil), // 26: milvus.proto.internal.ChannelTimeTickMsg + (*CredentialInfo)(nil), // 27: milvus.proto.internal.CredentialInfo + (*ListPolicyRequest)(nil), // 28: milvus.proto.internal.ListPolicyRequest + (*ListPolicyResponse)(nil), // 29: milvus.proto.internal.ListPolicyResponse + (*ShowConfigurationsRequest)(nil), // 30: milvus.proto.internal.ShowConfigurationsRequest + (*ShowConfigurationsResponse)(nil), // 31: milvus.proto.internal.ShowConfigurationsResponse + (*Rate)(nil), // 32: milvus.proto.internal.Rate + (*ImportFile)(nil), // 33: milvus.proto.internal.ImportFile + (*ImportRequestInternal)(nil), // 34: milvus.proto.internal.ImportRequestInternal + (*ImportRequest)(nil), // 35: milvus.proto.internal.ImportRequest + (*ImportResponse)(nil), // 36: milvus.proto.internal.ImportResponse + (*GetImportProgressRequest)(nil), // 37: milvus.proto.internal.GetImportProgressRequest + (*ImportTaskProgress)(nil), // 38: milvus.proto.internal.ImportTaskProgress + (*GetImportProgressResponse)(nil), // 39: milvus.proto.internal.GetImportProgressResponse + (*ListImportsRequestInternal)(nil), // 40: milvus.proto.internal.ListImportsRequestInternal + (*ListImportsRequest)(nil), // 41: milvus.proto.internal.ListImportsRequest + (*ListImportsResponse)(nil), // 42: milvus.proto.internal.ListImportsResponse + nil, // 43: milvus.proto.internal.SearchResults.ChannelsMvccEntry + (*commonpb.Address)(nil), // 44: milvus.proto.common.Address + (*commonpb.KeyValuePair)(nil), // 45: milvus.proto.common.KeyValuePair + (*commonpb.Status)(nil), // 46: milvus.proto.common.Status + (*commonpb.MsgBase)(nil), // 47: milvus.proto.common.MsgBase + (commonpb.DslType)(0), // 48: milvus.proto.common.DslType + (commonpb.ConsistencyLevel)(0), // 49: milvus.proto.common.ConsistencyLevel + (*schemapb.IDs)(nil), // 50: milvus.proto.schema.IDs + (*schemapb.FieldData)(nil), // 51: milvus.proto.schema.FieldData + (*milvuspb.PrivilegeGroupInfo)(nil), // 52: milvus.proto.milvus.PrivilegeGroupInfo + (*schemapb.CollectionSchema)(nil), // 53: milvus.proto.schema.CollectionSchema +} +var file_internal_proto_depIdxs = []int32{ + 44, // 0: milvus.proto.internal.NodeInfo.address:type_name -> milvus.proto.common.Address + 45, // 1: milvus.proto.internal.InitParams.start_params:type_name -> milvus.proto.common.KeyValuePair + 46, // 2: milvus.proto.internal.StringList.status:type_name -> milvus.proto.common.Status + 47, // 3: milvus.proto.internal.GetStatisticsRequest.base:type_name -> milvus.proto.common.MsgBase + 47, // 4: milvus.proto.internal.GetStatisticsResponse.base:type_name -> milvus.proto.common.MsgBase + 46, // 5: milvus.proto.internal.GetStatisticsResponse.status:type_name -> milvus.proto.common.Status + 45, // 6: milvus.proto.internal.GetStatisticsResponse.stats:type_name -> milvus.proto.common.KeyValuePair + 47, // 7: milvus.proto.internal.CreateAliasRequest.base:type_name -> milvus.proto.common.MsgBase + 47, // 8: milvus.proto.internal.DropAliasRequest.base:type_name -> milvus.proto.common.MsgBase + 47, // 9: milvus.proto.internal.AlterAliasRequest.base:type_name -> milvus.proto.common.MsgBase + 47, // 10: milvus.proto.internal.CreateIndexRequest.base:type_name -> milvus.proto.common.MsgBase + 45, // 11: milvus.proto.internal.CreateIndexRequest.extra_params:type_name -> milvus.proto.common.KeyValuePair + 48, // 12: milvus.proto.internal.SubSearchRequest.dsl_type:type_name -> milvus.proto.common.DslType + 47, // 13: milvus.proto.internal.SearchRequest.base:type_name -> milvus.proto.common.MsgBase + 48, // 14: milvus.proto.internal.SearchRequest.dsl_type:type_name -> milvus.proto.common.DslType + 15, // 15: milvus.proto.internal.SearchRequest.sub_reqs:type_name -> milvus.proto.internal.SubSearchRequest + 49, // 16: milvus.proto.internal.SearchRequest.consistency_level:type_name -> milvus.proto.common.ConsistencyLevel + 47, // 17: milvus.proto.internal.SearchResults.base:type_name -> milvus.proto.common.MsgBase + 46, // 18: milvus.proto.internal.SearchResults.status:type_name -> milvus.proto.common.Status + 19, // 19: milvus.proto.internal.SearchResults.costAggregation:type_name -> milvus.proto.internal.CostAggregation + 43, // 20: milvus.proto.internal.SearchResults.channels_mvcc:type_name -> milvus.proto.internal.SearchResults.ChannelsMvccEntry + 17, // 21: milvus.proto.internal.SearchResults.sub_results:type_name -> milvus.proto.internal.SubSearchResults + 47, // 22: milvus.proto.internal.RetrieveRequest.base:type_name -> milvus.proto.common.MsgBase + 47, // 23: milvus.proto.internal.RetrieveResults.base:type_name -> milvus.proto.common.MsgBase + 46, // 24: milvus.proto.internal.RetrieveResults.status:type_name -> milvus.proto.common.Status + 50, // 25: milvus.proto.internal.RetrieveResults.ids:type_name -> milvus.proto.schema.IDs + 51, // 26: milvus.proto.internal.RetrieveResults.fields_data:type_name -> milvus.proto.schema.FieldData + 19, // 27: milvus.proto.internal.RetrieveResults.costAggregation:type_name -> milvus.proto.internal.CostAggregation + 47, // 28: milvus.proto.internal.LoadIndex.base:type_name -> milvus.proto.common.MsgBase + 45, // 29: milvus.proto.internal.LoadIndex.index_params:type_name -> milvus.proto.common.KeyValuePair + 45, // 30: milvus.proto.internal.IndexStats.index_params:type_name -> milvus.proto.common.KeyValuePair + 23, // 31: milvus.proto.internal.FieldStats.index_stats:type_name -> milvus.proto.internal.IndexStats + 47, // 32: milvus.proto.internal.ChannelTimeTickMsg.base:type_name -> milvus.proto.common.MsgBase + 47, // 33: milvus.proto.internal.ListPolicyRequest.base:type_name -> milvus.proto.common.MsgBase + 46, // 34: milvus.proto.internal.ListPolicyResponse.status:type_name -> milvus.proto.common.Status + 52, // 35: milvus.proto.internal.ListPolicyResponse.privilege_groups:type_name -> milvus.proto.milvus.PrivilegeGroupInfo + 47, // 36: milvus.proto.internal.ShowConfigurationsRequest.base:type_name -> milvus.proto.common.MsgBase + 46, // 37: milvus.proto.internal.ShowConfigurationsResponse.status:type_name -> milvus.proto.common.Status + 45, // 38: milvus.proto.internal.ShowConfigurationsResponse.configuations:type_name -> milvus.proto.common.KeyValuePair + 1, // 39: milvus.proto.internal.Rate.rt:type_name -> milvus.proto.internal.RateType + 53, // 40: milvus.proto.internal.ImportRequestInternal.schema:type_name -> milvus.proto.schema.CollectionSchema + 33, // 41: milvus.proto.internal.ImportRequestInternal.files:type_name -> milvus.proto.internal.ImportFile + 45, // 42: milvus.proto.internal.ImportRequestInternal.options:type_name -> milvus.proto.common.KeyValuePair + 33, // 43: milvus.proto.internal.ImportRequest.files:type_name -> milvus.proto.internal.ImportFile + 45, // 44: milvus.proto.internal.ImportRequest.options:type_name -> milvus.proto.common.KeyValuePair + 46, // 45: milvus.proto.internal.ImportResponse.status:type_name -> milvus.proto.common.Status + 46, // 46: milvus.proto.internal.GetImportProgressResponse.status:type_name -> milvus.proto.common.Status + 2, // 47: milvus.proto.internal.GetImportProgressResponse.state:type_name -> milvus.proto.internal.ImportJobState + 38, // 48: milvus.proto.internal.GetImportProgressResponse.task_progresses:type_name -> milvus.proto.internal.ImportTaskProgress + 46, // 49: milvus.proto.internal.ListImportsResponse.status:type_name -> milvus.proto.common.Status + 2, // 50: milvus.proto.internal.ListImportsResponse.states:type_name -> milvus.proto.internal.ImportJobState + 51, // [51:51] is the sub-list for method output_type + 51, // [51:51] is the sub-list for method input_type + 51, // [51:51] is the sub-list for extension type_name + 51, // [51:51] is the sub-list for extension extendee + 0, // [0:51] is the sub-list for field type_name +} + +func init() { file_internal_proto_init() } +func file_internal_proto_init() { + if File_internal_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_internal_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetTimeTickChannelRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetStatisticsChannelRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetDdChannelRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*NodeInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*InitParams); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*StringList); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetStatisticsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetStatisticsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CreateAliasRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DropAliasRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AlterAliasRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CreateIndexRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SubSearchRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SearchRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SubSearchResults); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SearchResults); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CostAggregation); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RetrieveRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RetrieveResults); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*LoadIndex); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*IndexStats); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*FieldStats); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentStats); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ChannelTimeTickMsg); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CredentialInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListPolicyRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListPolicyResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ShowConfigurationsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ShowConfigurationsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Rate); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ImportFile); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ImportRequestInternal); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ImportRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ImportResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetImportProgressRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ImportTaskProgress); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetImportProgressResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListImportsRequestInternal); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListImportsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListImportsResponse); 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{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_internal_proto_rawDesc, + NumEnums: 3, + NumMessages: 41, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_internal_proto_goTypes, + DependencyIndexes: file_internal_proto_depIdxs, + EnumInfos: file_internal_proto_enumTypes, + MessageInfos: file_internal_proto_msgTypes, + }.Build() + File_internal_proto = out.File + file_internal_proto_rawDesc = nil + file_internal_proto_goTypes = nil + file_internal_proto_depIdxs = nil +} diff --git a/pkg/streaming/proto/messages.proto b/pkg/proto/messages.proto similarity index 98% rename from pkg/streaming/proto/messages.proto rename to pkg/proto/messages.proto index 843556ce4541c..7ba14b70c42bd 100644 --- a/pkg/streaming/proto/messages.proto +++ b/pkg/proto/messages.proto @@ -2,7 +2,7 @@ syntax = "proto3"; package milvus.proto.messages; -option go_package = "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb"; +option go_package = "github.com/milvus-io/milvus/pkg/proto/messagespb"; // MessageID is the unique identifier of a message. message MessageID { diff --git a/pkg/proto/messagespb/messages.pb.go b/pkg/proto/messagespb/messages.pb.go new file mode 100644 index 0000000000000..809c0f1b2daab --- /dev/null +++ b/pkg/proto/messagespb/messages.pb.go @@ -0,0 +1,2399 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.33.0 +// protoc v3.21.4 +// source: messages.proto + +package messagespb + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// MessageType is the type of message. +type MessageType int32 + +const ( + MessageType_Unknown MessageType = 0 + MessageType_TimeTick MessageType = 1 + MessageType_Insert MessageType = 2 + MessageType_Delete MessageType = 3 + MessageType_Flush MessageType = 4 + MessageType_CreateCollection MessageType = 5 + MessageType_DropCollection MessageType = 6 + MessageType_CreatePartition MessageType = 7 + MessageType_DropPartition MessageType = 8 + MessageType_ManualFlush MessageType = 9 + MessageType_CreateSegment MessageType = 10 + // 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 + // is received. + MessageType_BeginTxn MessageType = 900 + // commit transaction message is only used for transaction, once a commit + // transaction message is received, all messages combined with the + // transaction message can be consumed, the message combined with the + // transaction which is received after the commit transaction message will + // be drop. + MessageType_CommitTxn MessageType = 901 + // rollback transaction message is only used for transaction, once a + // rollback transaction message is received, all messages combined with the + // transaction message can be discarded, the message combined with the + // transaction which is received after the rollback transaction message will + // be drop. + MessageType_RollbackTxn MessageType = 902 + // txn message is a set of messages combined by multiple messages in a + // transaction. the txn properties is consist of the begin txn message and + // commit txn message. + MessageType_Txn MessageType = 999 +) + +// Enum value maps for MessageType. +var ( + MessageType_name = map[int32]string{ + 0: "Unknown", + 1: "TimeTick", + 2: "Insert", + 3: "Delete", + 4: "Flush", + 5: "CreateCollection", + 6: "DropCollection", + 7: "CreatePartition", + 8: "DropPartition", + 9: "ManualFlush", + 10: "CreateSegment", + 900: "BeginTxn", + 901: "CommitTxn", + 902: "RollbackTxn", + 999: "Txn", + } + MessageType_value = map[string]int32{ + "Unknown": 0, + "TimeTick": 1, + "Insert": 2, + "Delete": 3, + "Flush": 4, + "CreateCollection": 5, + "DropCollection": 6, + "CreatePartition": 7, + "DropPartition": 8, + "ManualFlush": 9, + "CreateSegment": 10, + "BeginTxn": 900, + "CommitTxn": 901, + "RollbackTxn": 902, + "Txn": 999, + } +) + +func (x MessageType) Enum() *MessageType { + p := new(MessageType) + *p = x + return p +} + +func (x MessageType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (MessageType) Descriptor() protoreflect.EnumDescriptor { + return file_messages_proto_enumTypes[0].Descriptor() +} + +func (MessageType) Type() protoreflect.EnumType { + return &file_messages_proto_enumTypes[0] +} + +func (x MessageType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use MessageType.Descriptor instead. +func (MessageType) EnumDescriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{0} +} + +type TxnState int32 + +const ( + // should never be used. + TxnState_TxnUnknown TxnState = 0 + // the transaction begin. + TxnState_TxnBegin TxnState = 1 + // the transaction is in flight. + TxnState_TxnInFlight TxnState = 2 + // the transaction is on commit. + TxnState_TxnOnCommit TxnState = 3 + // the transaction is committed. + TxnState_TxnCommitted TxnState = 4 + // the transaction is on rollback. + TxnState_TxnOnRollback TxnState = 5 + // the transaction is rollbacked. + TxnState_TxnRollbacked TxnState = 6 +) + +// Enum value maps for TxnState. +var ( + TxnState_name = map[int32]string{ + 0: "TxnUnknown", + 1: "TxnBegin", + 2: "TxnInFlight", + 3: "TxnOnCommit", + 4: "TxnCommitted", + 5: "TxnOnRollback", + 6: "TxnRollbacked", + } + TxnState_value = map[string]int32{ + "TxnUnknown": 0, + "TxnBegin": 1, + "TxnInFlight": 2, + "TxnOnCommit": 3, + "TxnCommitted": 4, + "TxnOnRollback": 5, + "TxnRollbacked": 6, + } +) + +func (x TxnState) Enum() *TxnState { + p := new(TxnState) + *p = x + return p +} + +func (x TxnState) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (TxnState) Descriptor() protoreflect.EnumDescriptor { + return file_messages_proto_enumTypes[1].Descriptor() +} + +func (TxnState) Type() protoreflect.EnumType { + return &file_messages_proto_enumTypes[1] +} + +func (x TxnState) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use TxnState.Descriptor instead. +func (TxnState) EnumDescriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{1} +} + +// MessageID is the unique identifier of a message. +type MessageID struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` +} + +func (x *MessageID) Reset() { + *x = MessageID{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *MessageID) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MessageID) ProtoMessage() {} + +func (x *MessageID) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[0] + 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 MessageID.ProtoReflect.Descriptor instead. +func (*MessageID) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{0} +} + +func (x *MessageID) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +// Message is the basic unit of communication between publisher and consumer. +type Message struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Payload []byte `protobuf:"bytes,1,opt,name=payload,proto3" json:"payload,omitempty"` // message body + Properties map[string]string `protobuf:"bytes,2,rep,name=properties,proto3" json:"properties,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // message properties +} + +func (x *Message) Reset() { + *x = Message{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Message) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Message) ProtoMessage() {} + +func (x *Message) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[1] + 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 Message.ProtoReflect.Descriptor instead. +func (*Message) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{1} +} + +func (x *Message) GetPayload() []byte { + if x != nil { + return x.Payload + } + return nil +} + +func (x *Message) GetProperties() map[string]string { + if x != nil { + return x.Properties + } + return nil +} + +// ImmutableMessage is the message that can not be modified anymore. +type ImmutableMessage struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Id *MessageID `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Payload []byte `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"` // message body + Properties map[string]string `protobuf:"bytes,3,rep,name=properties,proto3" json:"properties,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // message properties +} + +func (x *ImmutableMessage) Reset() { + *x = ImmutableMessage{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ImmutableMessage) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ImmutableMessage) ProtoMessage() {} + +func (x *ImmutableMessage) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[2] + 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 ImmutableMessage.ProtoReflect.Descriptor instead. +func (*ImmutableMessage) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{2} +} + +func (x *ImmutableMessage) GetId() *MessageID { + if x != nil { + return x.Id + } + return nil +} + +func (x *ImmutableMessage) GetPayload() []byte { + if x != nil { + return x.Payload + } + return nil +} + +func (x *ImmutableMessage) GetProperties() map[string]string { + if x != nil { + return x.Properties + } + return nil +} + +// FlushMessageBody is the body of flush message. +type FlushMessageBody struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // indicate which the collection that segment belong to. + CollectionId int64 `protobuf:"varint,1,opt,name=collection_id,json=collectionId,proto3" json:"collection_id,omitempty"` + SegmentId []int64 `protobuf:"varint,2,rep,packed,name=segment_id,json=segmentId,proto3" json:"segment_id,omitempty"` // indicate which segment to flush. +} + +func (x *FlushMessageBody) Reset() { + *x = FlushMessageBody{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *FlushMessageBody) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FlushMessageBody) ProtoMessage() {} + +func (x *FlushMessageBody) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[3] + 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 FlushMessageBody.ProtoReflect.Descriptor instead. +func (*FlushMessageBody) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{3} +} + +func (x *FlushMessageBody) GetCollectionId() int64 { + if x != nil { + return x.CollectionId + } + return 0 +} + +func (x *FlushMessageBody) GetSegmentId() []int64 { + if x != nil { + return x.SegmentId + } + return nil +} + +// ManualFlushMessageBody is the body of manual flush message. +type ManualFlushMessageBody struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *ManualFlushMessageBody) Reset() { + *x = ManualFlushMessageBody{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ManualFlushMessageBody) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ManualFlushMessageBody) ProtoMessage() {} + +func (x *ManualFlushMessageBody) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[4] + 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 ManualFlushMessageBody.ProtoReflect.Descriptor instead. +func (*ManualFlushMessageBody) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{4} +} + +// CreateSegmentMessageBody is the body of create segment message. +type CreateSegmentMessageBody 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"` + Segments []*CreateSegmentInfo `protobuf:"bytes,2,rep,name=segments,proto3" json:"segments,omitempty"` +} + +func (x *CreateSegmentMessageBody) Reset() { + *x = CreateSegmentMessageBody{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateSegmentMessageBody) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateSegmentMessageBody) ProtoMessage() {} + +func (x *CreateSegmentMessageBody) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[5] + 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 CreateSegmentMessageBody.ProtoReflect.Descriptor instead. +func (*CreateSegmentMessageBody) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{5} +} + +func (x *CreateSegmentMessageBody) GetCollectionId() int64 { + if x != nil { + return x.CollectionId + } + return 0 +} + +func (x *CreateSegmentMessageBody) GetSegments() []*CreateSegmentInfo { + if x != nil { + return x.Segments + } + return nil +} + +// CreateSegmentInfo is the info of create segment. +type CreateSegmentInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + PartitionId int64 `protobuf:"varint,1,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"` + SegmentId int64 `protobuf:"varint,2,opt,name=segment_id,json=segmentId,proto3" json:"segment_id,omitempty"` +} + +func (x *CreateSegmentInfo) Reset() { + *x = CreateSegmentInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateSegmentInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateSegmentInfo) ProtoMessage() {} + +func (x *CreateSegmentInfo) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[6] + 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 CreateSegmentInfo.ProtoReflect.Descriptor instead. +func (*CreateSegmentInfo) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{6} +} + +func (x *CreateSegmentInfo) GetPartitionId() int64 { + if x != nil { + return x.PartitionId + } + return 0 +} + +func (x *CreateSegmentInfo) GetSegmentId() int64 { + if x != nil { + return x.SegmentId + } + return 0 +} + +// BeginTxnMessageBody is the body of begin transaction message. +// Just do nothing now. +type BeginTxnMessageBody struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *BeginTxnMessageBody) Reset() { + *x = BeginTxnMessageBody{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BeginTxnMessageBody) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BeginTxnMessageBody) ProtoMessage() {} + +func (x *BeginTxnMessageBody) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[7] + 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 BeginTxnMessageBody.ProtoReflect.Descriptor instead. +func (*BeginTxnMessageBody) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{7} +} + +// CommitTxnMessageBody is the body of commit transaction message. +// Just do nothing now. +type CommitTxnMessageBody struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *CommitTxnMessageBody) Reset() { + *x = CommitTxnMessageBody{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CommitTxnMessageBody) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CommitTxnMessageBody) ProtoMessage() {} + +func (x *CommitTxnMessageBody) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[8] + 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 CommitTxnMessageBody.ProtoReflect.Descriptor instead. +func (*CommitTxnMessageBody) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{8} +} + +// RollbackTxnMessageBody is the body of rollback transaction message. +// Just do nothing now. +type RollbackTxnMessageBody struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *RollbackTxnMessageBody) Reset() { + *x = RollbackTxnMessageBody{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RollbackTxnMessageBody) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RollbackTxnMessageBody) ProtoMessage() {} + +func (x *RollbackTxnMessageBody) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[9] + 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 RollbackTxnMessageBody.ProtoReflect.Descriptor instead. +func (*RollbackTxnMessageBody) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{9} +} + +// TxnMessageBody is the body of transaction message. +// A transaction message is combined by multiple messages. +// It's only can be seen at consume side. +// All message in a transaction message only has same timetick which is equal to +// the CommitTransationMessage. +type TxnMessageBody struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Messages []*Message `protobuf:"bytes,1,rep,name=messages,proto3" json:"messages,omitempty"` +} + +func (x *TxnMessageBody) Reset() { + *x = TxnMessageBody{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TxnMessageBody) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TxnMessageBody) ProtoMessage() {} + +func (x *TxnMessageBody) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[10] + 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 TxnMessageBody.ProtoReflect.Descriptor instead. +func (*TxnMessageBody) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{10} +} + +func (x *TxnMessageBody) GetMessages() []*Message { + if x != nil { + return x.Messages + } + return nil +} + +// TimeTickMessageHeader just nothing. +type TimeTickMessageHeader struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *TimeTickMessageHeader) Reset() { + *x = TimeTickMessageHeader{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TimeTickMessageHeader) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TimeTickMessageHeader) ProtoMessage() {} + +func (x *TimeTickMessageHeader) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[11] + 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 TimeTickMessageHeader.ProtoReflect.Descriptor instead. +func (*TimeTickMessageHeader) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{11} +} + +// InsertMessageHeader is the header of insert message. +type InsertMessageHeader 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"` + Partitions []*PartitionSegmentAssignment `protobuf:"bytes,2,rep,name=partitions,proto3" json:"partitions,omitempty"` +} + +func (x *InsertMessageHeader) Reset() { + *x = InsertMessageHeader{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *InsertMessageHeader) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*InsertMessageHeader) ProtoMessage() {} + +func (x *InsertMessageHeader) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[12] + 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 InsertMessageHeader.ProtoReflect.Descriptor instead. +func (*InsertMessageHeader) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{12} +} + +func (x *InsertMessageHeader) GetCollectionId() int64 { + if x != nil { + return x.CollectionId + } + return 0 +} + +func (x *InsertMessageHeader) GetPartitions() []*PartitionSegmentAssignment { + if x != nil { + return x.Partitions + } + return nil +} + +// PartitionSegmentAssignment is the segment assignment of a partition. +type PartitionSegmentAssignment struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + PartitionId int64 `protobuf:"varint,1,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"` + Rows uint64 `protobuf:"varint,2,opt,name=rows,proto3" json:"rows,omitempty"` + BinarySize uint64 `protobuf:"varint,3,opt,name=binary_size,json=binarySize,proto3" json:"binary_size,omitempty"` + SegmentAssignment *SegmentAssignment `protobuf:"bytes,4,opt,name=segment_assignment,json=segmentAssignment,proto3" json:"segment_assignment,omitempty"` +} + +func (x *PartitionSegmentAssignment) Reset() { + *x = PartitionSegmentAssignment{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PartitionSegmentAssignment) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PartitionSegmentAssignment) ProtoMessage() {} + +func (x *PartitionSegmentAssignment) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[13] + 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 PartitionSegmentAssignment.ProtoReflect.Descriptor instead. +func (*PartitionSegmentAssignment) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{13} +} + +func (x *PartitionSegmentAssignment) GetPartitionId() int64 { + if x != nil { + return x.PartitionId + } + return 0 +} + +func (x *PartitionSegmentAssignment) GetRows() uint64 { + if x != nil { + return x.Rows + } + return 0 +} + +func (x *PartitionSegmentAssignment) GetBinarySize() uint64 { + if x != nil { + return x.BinarySize + } + return 0 +} + +func (x *PartitionSegmentAssignment) GetSegmentAssignment() *SegmentAssignment { + if x != nil { + return x.SegmentAssignment + } + return nil +} + +// SegmentAssignment is the assignment of a segment. +type SegmentAssignment struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SegmentId int64 `protobuf:"varint,1,opt,name=segment_id,json=segmentId,proto3" json:"segment_id,omitempty"` +} + +func (x *SegmentAssignment) Reset() { + *x = SegmentAssignment{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentAssignment) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentAssignment) ProtoMessage() {} + +func (x *SegmentAssignment) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[14] + 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 SegmentAssignment.ProtoReflect.Descriptor instead. +func (*SegmentAssignment) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{14} +} + +func (x *SegmentAssignment) GetSegmentId() int64 { + if x != nil { + return x.SegmentId + } + return 0 +} + +// DeleteMessageHeader +type DeleteMessageHeader 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 *DeleteMessageHeader) Reset() { + *x = DeleteMessageHeader{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeleteMessageHeader) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeleteMessageHeader) ProtoMessage() {} + +func (x *DeleteMessageHeader) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[15] + 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 DeleteMessageHeader.ProtoReflect.Descriptor instead. +func (*DeleteMessageHeader) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{15} +} + +func (x *DeleteMessageHeader) GetCollectionId() int64 { + if x != nil { + return x.CollectionId + } + return 0 +} + +// FlushMessageHeader just nothing. +type FlushMessageHeader struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *FlushMessageHeader) Reset() { + *x = FlushMessageHeader{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *FlushMessageHeader) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FlushMessageHeader) ProtoMessage() {} + +func (x *FlushMessageHeader) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[16] + 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 FlushMessageHeader.ProtoReflect.Descriptor instead. +func (*FlushMessageHeader) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{16} +} + +// CreateSegmentMessageHeader just nothing. +type CreateSegmentMessageHeader struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *CreateSegmentMessageHeader) Reset() { + *x = CreateSegmentMessageHeader{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateSegmentMessageHeader) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateSegmentMessageHeader) ProtoMessage() {} + +func (x *CreateSegmentMessageHeader) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[17] + 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 CreateSegmentMessageHeader.ProtoReflect.Descriptor instead. +func (*CreateSegmentMessageHeader) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{17} +} + +type ManualFlushMessageHeader 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"` + FlushTs uint64 `protobuf:"varint,2,opt,name=flush_ts,json=flushTs,proto3" json:"flush_ts,omitempty"` +} + +func (x *ManualFlushMessageHeader) Reset() { + *x = ManualFlushMessageHeader{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ManualFlushMessageHeader) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ManualFlushMessageHeader) ProtoMessage() {} + +func (x *ManualFlushMessageHeader) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[18] + 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 ManualFlushMessageHeader.ProtoReflect.Descriptor instead. +func (*ManualFlushMessageHeader) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{18} +} + +func (x *ManualFlushMessageHeader) GetCollectionId() int64 { + if x != nil { + return x.CollectionId + } + return 0 +} + +func (x *ManualFlushMessageHeader) GetFlushTs() uint64 { + if x != nil { + return x.FlushTs + } + return 0 +} + +// CreateCollectionMessageHeader is the header of create collection message. +type CreateCollectionMessageHeader 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"` + PartitionIds []int64 `protobuf:"varint,2,rep,packed,name=partition_ids,json=partitionIds,proto3" json:"partition_ids,omitempty"` +} + +func (x *CreateCollectionMessageHeader) Reset() { + *x = CreateCollectionMessageHeader{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[19] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateCollectionMessageHeader) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateCollectionMessageHeader) ProtoMessage() {} + +func (x *CreateCollectionMessageHeader) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[19] + 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 CreateCollectionMessageHeader.ProtoReflect.Descriptor instead. +func (*CreateCollectionMessageHeader) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{19} +} + +func (x *CreateCollectionMessageHeader) GetCollectionId() int64 { + if x != nil { + return x.CollectionId + } + return 0 +} + +func (x *CreateCollectionMessageHeader) GetPartitionIds() []int64 { + if x != nil { + return x.PartitionIds + } + return nil +} + +// DropCollectionMessageHeader is the header of drop collection message. +type DropCollectionMessageHeader 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 *DropCollectionMessageHeader) Reset() { + *x = DropCollectionMessageHeader{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[20] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DropCollectionMessageHeader) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DropCollectionMessageHeader) ProtoMessage() {} + +func (x *DropCollectionMessageHeader) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[20] + 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 DropCollectionMessageHeader.ProtoReflect.Descriptor instead. +func (*DropCollectionMessageHeader) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{20} +} + +func (x *DropCollectionMessageHeader) GetCollectionId() int64 { + if x != nil { + return x.CollectionId + } + return 0 +} + +// CreatePartitionMessageHeader is the header of create partition message. +type CreatePartitionMessageHeader 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"` + PartitionId int64 `protobuf:"varint,2,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"` +} + +func (x *CreatePartitionMessageHeader) Reset() { + *x = CreatePartitionMessageHeader{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[21] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreatePartitionMessageHeader) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreatePartitionMessageHeader) ProtoMessage() {} + +func (x *CreatePartitionMessageHeader) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[21] + 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 CreatePartitionMessageHeader.ProtoReflect.Descriptor instead. +func (*CreatePartitionMessageHeader) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{21} +} + +func (x *CreatePartitionMessageHeader) GetCollectionId() int64 { + if x != nil { + return x.CollectionId + } + return 0 +} + +func (x *CreatePartitionMessageHeader) GetPartitionId() int64 { + if x != nil { + return x.PartitionId + } + return 0 +} + +// DropPartitionMessageHeader is the header of drop partition message. +type DropPartitionMessageHeader 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"` + PartitionId int64 `protobuf:"varint,2,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"` +} + +func (x *DropPartitionMessageHeader) Reset() { + *x = DropPartitionMessageHeader{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[22] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DropPartitionMessageHeader) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DropPartitionMessageHeader) ProtoMessage() {} + +func (x *DropPartitionMessageHeader) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[22] + 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 DropPartitionMessageHeader.ProtoReflect.Descriptor instead. +func (*DropPartitionMessageHeader) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{22} +} + +func (x *DropPartitionMessageHeader) GetCollectionId() int64 { + if x != nil { + return x.CollectionId + } + return 0 +} + +func (x *DropPartitionMessageHeader) GetPartitionId() int64 { + if x != nil { + return x.PartitionId + } + return 0 +} + +// BeginTxnMessageHeader is the header of begin transaction message. +// Just do nothing now. +// Add Channel info here to implement cross pchannel transaction. +type BeginTxnMessageHeader struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // the max milliseconds to keep alive of the transaction. + // the keepalive_milliseconds is never changed in a transaction by now, + KeepaliveMilliseconds int64 `protobuf:"varint,1,opt,name=keepalive_milliseconds,json=keepaliveMilliseconds,proto3" json:"keepalive_milliseconds,omitempty"` +} + +func (x *BeginTxnMessageHeader) Reset() { + *x = BeginTxnMessageHeader{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[23] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BeginTxnMessageHeader) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BeginTxnMessageHeader) ProtoMessage() {} + +func (x *BeginTxnMessageHeader) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[23] + 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 BeginTxnMessageHeader.ProtoReflect.Descriptor instead. +func (*BeginTxnMessageHeader) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{23} +} + +func (x *BeginTxnMessageHeader) GetKeepaliveMilliseconds() int64 { + if x != nil { + return x.KeepaliveMilliseconds + } + return 0 +} + +// CommitTxnMessageHeader is the header of commit transaction message. +// Just do nothing now. +type CommitTxnMessageHeader struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *CommitTxnMessageHeader) Reset() { + *x = CommitTxnMessageHeader{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[24] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CommitTxnMessageHeader) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CommitTxnMessageHeader) ProtoMessage() {} + +func (x *CommitTxnMessageHeader) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[24] + 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 CommitTxnMessageHeader.ProtoReflect.Descriptor instead. +func (*CommitTxnMessageHeader) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{24} +} + +// RollbackTxnMessageHeader is the header of rollback transaction +// message. +// Just do nothing now. +type RollbackTxnMessageHeader struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *RollbackTxnMessageHeader) Reset() { + *x = RollbackTxnMessageHeader{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[25] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RollbackTxnMessageHeader) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RollbackTxnMessageHeader) ProtoMessage() {} + +func (x *RollbackTxnMessageHeader) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[25] + 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 RollbackTxnMessageHeader.ProtoReflect.Descriptor instead. +func (*RollbackTxnMessageHeader) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{25} +} + +// TxnMessageHeader is the header of transaction message. +// Just do nothing now. +type TxnMessageHeader struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *TxnMessageHeader) Reset() { + *x = TxnMessageHeader{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[26] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TxnMessageHeader) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TxnMessageHeader) ProtoMessage() {} + +func (x *TxnMessageHeader) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[26] + 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 TxnMessageHeader.ProtoReflect.Descriptor instead. +func (*TxnMessageHeader) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{26} +} + +// ManualFlushExtraResponse is the extra response of manual flush message. +type ManualFlushExtraResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SegmentIds []int64 `protobuf:"varint,1,rep,packed,name=segment_ids,json=segmentIds,proto3" json:"segment_ids,omitempty"` +} + +func (x *ManualFlushExtraResponse) Reset() { + *x = ManualFlushExtraResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[27] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ManualFlushExtraResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ManualFlushExtraResponse) ProtoMessage() {} + +func (x *ManualFlushExtraResponse) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[27] + 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 ManualFlushExtraResponse.ProtoReflect.Descriptor instead. +func (*ManualFlushExtraResponse) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{27} +} + +func (x *ManualFlushExtraResponse) GetSegmentIds() []int64 { + if x != nil { + return x.SegmentIds + } + return nil +} + +// TxnContext is the context of transaction. +// It will be carried by every message in a transaction. +type TxnContext struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // the unique id of the transaction. + // the txn_id is never changed in a transaction. + TxnId int64 `protobuf:"varint,1,opt,name=txn_id,json=txnId,proto3" json:"txn_id,omitempty"` + // the next keep alive timeout of the transaction. + // after the keep alive timeout, the transaction will be expired. + KeepaliveMilliseconds int64 `protobuf:"varint,2,opt,name=keepalive_milliseconds,json=keepaliveMilliseconds,proto3" json:"keepalive_milliseconds,omitempty"` +} + +func (x *TxnContext) Reset() { + *x = TxnContext{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[28] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TxnContext) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TxnContext) ProtoMessage() {} + +func (x *TxnContext) 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 TxnContext.ProtoReflect.Descriptor instead. +func (*TxnContext) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{28} +} + +func (x *TxnContext) GetTxnId() int64 { + if x != nil { + return x.TxnId + } + return 0 +} + +func (x *TxnContext) GetKeepaliveMilliseconds() int64 { + if x != nil { + return x.KeepaliveMilliseconds + } + return 0 +} + +// RMQMessageLayout is the layout of message for RMQ. +type RMQMessageLayout struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Payload []byte `protobuf:"bytes,1,opt,name=payload,proto3" json:"payload,omitempty"` // message body + Properties map[string]string `protobuf:"bytes,2,rep,name=properties,proto3" json:"properties,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // message properties +} + +func (x *RMQMessageLayout) Reset() { + *x = RMQMessageLayout{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[29] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RMQMessageLayout) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RMQMessageLayout) ProtoMessage() {} + +func (x *RMQMessageLayout) 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 RMQMessageLayout.ProtoReflect.Descriptor instead. +func (*RMQMessageLayout) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{29} +} + +func (x *RMQMessageLayout) GetPayload() []byte { + if x != nil { + return x.Payload + } + return nil +} + +func (x *RMQMessageLayout) GetProperties() map[string]string { + if x != nil { + return x.Properties + } + return nil +} + +// VChannels is a layout to represent the virtual channels for broadcast. +type VChannels struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Vchannels []string `protobuf:"bytes,1,rep,name=vchannels,proto3" json:"vchannels,omitempty"` +} + +func (x *VChannels) Reset() { + *x = VChannels{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[30] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *VChannels) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*VChannels) ProtoMessage() {} + +func (x *VChannels) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[30] + 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 VChannels.ProtoReflect.Descriptor instead. +func (*VChannels) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{30} +} + +func (x *VChannels) GetVchannels() []string { + if x != nil { + return x.Vchannels + } + return nil +} + +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, + 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, 0x55, 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, 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, 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, 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, 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, 0x29, 0x0a, 0x09, 0x56, 0x43, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x76, 0x63, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x2a, 0xfc, 0x01, 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, 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, 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, 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 ( + file_messages_proto_rawDescOnce sync.Once + file_messages_proto_rawDescData = file_messages_proto_rawDesc +) + +func file_messages_proto_rawDescGZIP() []byte { + file_messages_proto_rawDescOnce.Do(func() { + file_messages_proto_rawDescData = protoimpl.X.CompressGZIP(file_messages_proto_rawDescData) + }) + return file_messages_proto_rawDescData +} + +var file_messages_proto_enumTypes = make([]protoimpl.EnumInfo, 2) +var file_messages_proto_msgTypes = make([]protoimpl.MessageInfo, 34) +var file_messages_proto_goTypes = []interface{}{ + (MessageType)(0), // 0: milvus.proto.messages.MessageType + (TxnState)(0), // 1: milvus.proto.messages.TxnState + (*MessageID)(nil), // 2: milvus.proto.messages.MessageID + (*Message)(nil), // 3: milvus.proto.messages.Message + (*ImmutableMessage)(nil), // 4: milvus.proto.messages.ImmutableMessage + (*FlushMessageBody)(nil), // 5: milvus.proto.messages.FlushMessageBody + (*ManualFlushMessageBody)(nil), // 6: milvus.proto.messages.ManualFlushMessageBody + (*CreateSegmentMessageBody)(nil), // 7: milvus.proto.messages.CreateSegmentMessageBody + (*CreateSegmentInfo)(nil), // 8: milvus.proto.messages.CreateSegmentInfo + (*BeginTxnMessageBody)(nil), // 9: milvus.proto.messages.BeginTxnMessageBody + (*CommitTxnMessageBody)(nil), // 10: milvus.proto.messages.CommitTxnMessageBody + (*RollbackTxnMessageBody)(nil), // 11: milvus.proto.messages.RollbackTxnMessageBody + (*TxnMessageBody)(nil), // 12: milvus.proto.messages.TxnMessageBody + (*TimeTickMessageHeader)(nil), // 13: milvus.proto.messages.TimeTickMessageHeader + (*InsertMessageHeader)(nil), // 14: milvus.proto.messages.InsertMessageHeader + (*PartitionSegmentAssignment)(nil), // 15: milvus.proto.messages.PartitionSegmentAssignment + (*SegmentAssignment)(nil), // 16: milvus.proto.messages.SegmentAssignment + (*DeleteMessageHeader)(nil), // 17: milvus.proto.messages.DeleteMessageHeader + (*FlushMessageHeader)(nil), // 18: milvus.proto.messages.FlushMessageHeader + (*CreateSegmentMessageHeader)(nil), // 19: milvus.proto.messages.CreateSegmentMessageHeader + (*ManualFlushMessageHeader)(nil), // 20: milvus.proto.messages.ManualFlushMessageHeader + (*CreateCollectionMessageHeader)(nil), // 21: milvus.proto.messages.CreateCollectionMessageHeader + (*DropCollectionMessageHeader)(nil), // 22: milvus.proto.messages.DropCollectionMessageHeader + (*CreatePartitionMessageHeader)(nil), // 23: milvus.proto.messages.CreatePartitionMessageHeader + (*DropPartitionMessageHeader)(nil), // 24: milvus.proto.messages.DropPartitionMessageHeader + (*BeginTxnMessageHeader)(nil), // 25: milvus.proto.messages.BeginTxnMessageHeader + (*CommitTxnMessageHeader)(nil), // 26: milvus.proto.messages.CommitTxnMessageHeader + (*RollbackTxnMessageHeader)(nil), // 27: milvus.proto.messages.RollbackTxnMessageHeader + (*TxnMessageHeader)(nil), // 28: milvus.proto.messages.TxnMessageHeader + (*ManualFlushExtraResponse)(nil), // 29: milvus.proto.messages.ManualFlushExtraResponse + (*TxnContext)(nil), // 30: milvus.proto.messages.TxnContext + (*RMQMessageLayout)(nil), // 31: milvus.proto.messages.RMQMessageLayout + (*VChannels)(nil), // 32: milvus.proto.messages.VChannels + nil, // 33: milvus.proto.messages.Message.PropertiesEntry + nil, // 34: milvus.proto.messages.ImmutableMessage.PropertiesEntry + nil, // 35: milvus.proto.messages.RMQMessageLayout.PropertiesEntry +} +var file_messages_proto_depIdxs = []int32{ + 33, // 0: milvus.proto.messages.Message.properties:type_name -> milvus.proto.messages.Message.PropertiesEntry + 2, // 1: milvus.proto.messages.ImmutableMessage.id:type_name -> milvus.proto.messages.MessageID + 34, // 2: milvus.proto.messages.ImmutableMessage.properties:type_name -> milvus.proto.messages.ImmutableMessage.PropertiesEntry + 8, // 3: milvus.proto.messages.CreateSegmentMessageBody.segments:type_name -> milvus.proto.messages.CreateSegmentInfo + 3, // 4: milvus.proto.messages.TxnMessageBody.messages:type_name -> milvus.proto.messages.Message + 15, // 5: milvus.proto.messages.InsertMessageHeader.partitions:type_name -> milvus.proto.messages.PartitionSegmentAssignment + 16, // 6: milvus.proto.messages.PartitionSegmentAssignment.segment_assignment:type_name -> milvus.proto.messages.SegmentAssignment + 35, // 7: milvus.proto.messages.RMQMessageLayout.properties:type_name -> milvus.proto.messages.RMQMessageLayout.PropertiesEntry + 8, // [8:8] is the sub-list for method output_type + 8, // [8:8] is the sub-list for method input_type + 8, // [8:8] is the sub-list for extension type_name + 8, // [8:8] is the sub-list for extension extendee + 0, // [0:8] is the sub-list for field type_name +} + +func init() { file_messages_proto_init() } +func file_messages_proto_init() { + if File_messages_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_messages_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*MessageID); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Message); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ImmutableMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*FlushMessageBody); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ManualFlushMessageBody); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CreateSegmentMessageBody); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CreateSegmentInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BeginTxnMessageBody); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CommitTxnMessageBody); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RollbackTxnMessageBody); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*TxnMessageBody); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*TimeTickMessageHeader); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*InsertMessageHeader); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PartitionSegmentAssignment); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentAssignment); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DeleteMessageHeader); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*FlushMessageHeader); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CreateSegmentMessageHeader); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ManualFlushMessageHeader); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CreateCollectionMessageHeader); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DropCollectionMessageHeader); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CreatePartitionMessageHeader); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DropPartitionMessageHeader); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BeginTxnMessageHeader); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CommitTxnMessageHeader); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RollbackTxnMessageHeader); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*TxnMessageHeader); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ManualFlushExtraResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*TxnContext); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RMQMessageLayout); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*VChannels); 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{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_messages_proto_rawDesc, + NumEnums: 2, + NumMessages: 34, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_messages_proto_goTypes, + DependencyIndexes: file_messages_proto_depIdxs, + EnumInfos: file_messages_proto_enumTypes, + MessageInfos: file_messages_proto_msgTypes, + }.Build() + File_messages_proto = out.File + file_messages_proto_rawDesc = nil + file_messages_proto_goTypes = nil + file_messages_proto_depIdxs = nil +} diff --git a/internal/proto/plan.proto b/pkg/proto/plan.proto similarity index 98% rename from internal/proto/plan.proto rename to pkg/proto/plan.proto index 4ece80186fb26..43c4494bd9ccb 100644 --- a/internal/proto/plan.proto +++ b/pkg/proto/plan.proto @@ -1,7 +1,7 @@ syntax = "proto3"; package milvus.proto.plan; -option go_package = "github.com/milvus-io/milvus/internal/proto/planpb"; +option go_package = "github.com/milvus-io/milvus/pkg/proto/planpb"; import "schema.proto"; enum OpType { diff --git a/pkg/proto/planpb/plan.pb.go b/pkg/proto/planpb/plan.pb.go new file mode 100644 index 0000000000000..7c8ec954c7bb3 --- /dev/null +++ b/pkg/proto/planpb/plan.pb.go @@ -0,0 +1,3375 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.33.0 +// protoc v3.21.4 +// source: plan.proto + +package planpb + +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" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type OpType int32 + +const ( + OpType_Invalid OpType = 0 + OpType_GreaterThan OpType = 1 + OpType_GreaterEqual OpType = 2 + OpType_LessThan OpType = 3 + OpType_LessEqual OpType = 4 + OpType_Equal OpType = 5 + OpType_NotEqual OpType = 6 + OpType_PrefixMatch OpType = 7 // startsWith + OpType_PostfixMatch OpType = 8 // endsWith + OpType_Match OpType = 9 // like + OpType_Range OpType = 10 // for case 1 < a < b + OpType_In OpType = 11 // TODO:: used for term expr + OpType_NotIn OpType = 12 + OpType_TextMatch OpType = 13 // text match +) + +// Enum value maps for OpType. +var ( + OpType_name = map[int32]string{ + 0: "Invalid", + 1: "GreaterThan", + 2: "GreaterEqual", + 3: "LessThan", + 4: "LessEqual", + 5: "Equal", + 6: "NotEqual", + 7: "PrefixMatch", + 8: "PostfixMatch", + 9: "Match", + 10: "Range", + 11: "In", + 12: "NotIn", + 13: "TextMatch", + } + OpType_value = map[string]int32{ + "Invalid": 0, + "GreaterThan": 1, + "GreaterEqual": 2, + "LessThan": 3, + "LessEqual": 4, + "Equal": 5, + "NotEqual": 6, + "PrefixMatch": 7, + "PostfixMatch": 8, + "Match": 9, + "Range": 10, + "In": 11, + "NotIn": 12, + "TextMatch": 13, + } +) + +func (x OpType) Enum() *OpType { + p := new(OpType) + *p = x + return p +} + +func (x OpType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (OpType) Descriptor() protoreflect.EnumDescriptor { + return file_plan_proto_enumTypes[0].Descriptor() +} + +func (OpType) Type() protoreflect.EnumType { + return &file_plan_proto_enumTypes[0] +} + +func (x OpType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use OpType.Descriptor instead. +func (OpType) EnumDescriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{0} +} + +type ArithOpType int32 + +const ( + ArithOpType_Unknown ArithOpType = 0 + ArithOpType_Add ArithOpType = 1 + ArithOpType_Sub ArithOpType = 2 + ArithOpType_Mul ArithOpType = 3 + ArithOpType_Div ArithOpType = 4 + ArithOpType_Mod ArithOpType = 5 + ArithOpType_ArrayLength ArithOpType = 6 +) + +// Enum value maps for ArithOpType. +var ( + ArithOpType_name = map[int32]string{ + 0: "Unknown", + 1: "Add", + 2: "Sub", + 3: "Mul", + 4: "Div", + 5: "Mod", + 6: "ArrayLength", + } + ArithOpType_value = map[string]int32{ + "Unknown": 0, + "Add": 1, + "Sub": 2, + "Mul": 3, + "Div": 4, + "Mod": 5, + "ArrayLength": 6, + } +) + +func (x ArithOpType) Enum() *ArithOpType { + p := new(ArithOpType) + *p = x + return p +} + +func (x ArithOpType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (ArithOpType) Descriptor() protoreflect.EnumDescriptor { + return file_plan_proto_enumTypes[1].Descriptor() +} + +func (ArithOpType) Type() protoreflect.EnumType { + return &file_plan_proto_enumTypes[1] +} + +func (x ArithOpType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ArithOpType.Descriptor instead. +func (ArithOpType) EnumDescriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{1} +} + +type VectorType int32 + +const ( + VectorType_BinaryVector VectorType = 0 + VectorType_FloatVector VectorType = 1 + VectorType_Float16Vector VectorType = 2 + VectorType_BFloat16Vector VectorType = 3 + VectorType_SparseFloatVector VectorType = 4 +) + +// Enum value maps for VectorType. +var ( + VectorType_name = map[int32]string{ + 0: "BinaryVector", + 1: "FloatVector", + 2: "Float16Vector", + 3: "BFloat16Vector", + 4: "SparseFloatVector", + } + VectorType_value = map[string]int32{ + "BinaryVector": 0, + "FloatVector": 1, + "Float16Vector": 2, + "BFloat16Vector": 3, + "SparseFloatVector": 4, + } +) + +func (x VectorType) Enum() *VectorType { + p := new(VectorType) + *p = x + return p +} + +func (x VectorType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (VectorType) Descriptor() protoreflect.EnumDescriptor { + return file_plan_proto_enumTypes[2].Descriptor() +} + +func (VectorType) Type() protoreflect.EnumType { + return &file_plan_proto_enumTypes[2] +} + +func (x VectorType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use VectorType.Descriptor instead. +func (VectorType) EnumDescriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{2} +} + +// 0: invalid +// 1: json_contains | array_contains +// 2: json_contains_all | array_contains_all +// 3: json_contains_any | array_contains_any +type JSONContainsExpr_JSONOp int32 + +const ( + JSONContainsExpr_Invalid JSONContainsExpr_JSONOp = 0 + JSONContainsExpr_Contains JSONContainsExpr_JSONOp = 1 + JSONContainsExpr_ContainsAll JSONContainsExpr_JSONOp = 2 + JSONContainsExpr_ContainsAny JSONContainsExpr_JSONOp = 3 +) + +// Enum value maps for JSONContainsExpr_JSONOp. +var ( + JSONContainsExpr_JSONOp_name = map[int32]string{ + 0: "Invalid", + 1: "Contains", + 2: "ContainsAll", + 3: "ContainsAny", + } + JSONContainsExpr_JSONOp_value = map[string]int32{ + "Invalid": 0, + "Contains": 1, + "ContainsAll": 2, + "ContainsAny": 3, + } +) + +func (x JSONContainsExpr_JSONOp) Enum() *JSONContainsExpr_JSONOp { + p := new(JSONContainsExpr_JSONOp) + *p = x + return p +} + +func (x JSONContainsExpr_JSONOp) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (JSONContainsExpr_JSONOp) Descriptor() protoreflect.EnumDescriptor { + return file_plan_proto_enumTypes[3].Descriptor() +} + +func (JSONContainsExpr_JSONOp) Type() protoreflect.EnumType { + return &file_plan_proto_enumTypes[3] +} + +func (x JSONContainsExpr_JSONOp) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use JSONContainsExpr_JSONOp.Descriptor instead. +func (JSONContainsExpr_JSONOp) EnumDescriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{13, 0} +} + +type NullExpr_NullOp int32 + +const ( + NullExpr_Invalid NullExpr_NullOp = 0 + NullExpr_IsNull NullExpr_NullOp = 1 + NullExpr_IsNotNull NullExpr_NullOp = 2 +) + +// Enum value maps for NullExpr_NullOp. +var ( + NullExpr_NullOp_name = map[int32]string{ + 0: "Invalid", + 1: "IsNull", + 2: "IsNotNull", + } + NullExpr_NullOp_value = map[string]int32{ + "Invalid": 0, + "IsNull": 1, + "IsNotNull": 2, + } +) + +func (x NullExpr_NullOp) Enum() *NullExpr_NullOp { + p := new(NullExpr_NullOp) + *p = x + return p +} + +func (x NullExpr_NullOp) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (NullExpr_NullOp) Descriptor() protoreflect.EnumDescriptor { + return file_plan_proto_enumTypes[4].Descriptor() +} + +func (NullExpr_NullOp) Type() protoreflect.EnumType { + return &file_plan_proto_enumTypes[4] +} + +func (x NullExpr_NullOp) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use NullExpr_NullOp.Descriptor instead. +func (NullExpr_NullOp) EnumDescriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{14, 0} +} + +type UnaryExpr_UnaryOp int32 + +const ( + UnaryExpr_Invalid UnaryExpr_UnaryOp = 0 + UnaryExpr_Not UnaryExpr_UnaryOp = 1 +) + +// Enum value maps for UnaryExpr_UnaryOp. +var ( + UnaryExpr_UnaryOp_name = map[int32]string{ + 0: "Invalid", + 1: "Not", + } + UnaryExpr_UnaryOp_value = map[string]int32{ + "Invalid": 0, + "Not": 1, + } +) + +func (x UnaryExpr_UnaryOp) Enum() *UnaryExpr_UnaryOp { + p := new(UnaryExpr_UnaryOp) + *p = x + return p +} + +func (x UnaryExpr_UnaryOp) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (UnaryExpr_UnaryOp) Descriptor() protoreflect.EnumDescriptor { + return file_plan_proto_enumTypes[5].Descriptor() +} + +func (UnaryExpr_UnaryOp) Type() protoreflect.EnumType { + return &file_plan_proto_enumTypes[5] +} + +func (x UnaryExpr_UnaryOp) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use UnaryExpr_UnaryOp.Descriptor instead. +func (UnaryExpr_UnaryOp) EnumDescriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{15, 0} +} + +type BinaryExpr_BinaryOp int32 + +const ( + BinaryExpr_Invalid BinaryExpr_BinaryOp = 0 + BinaryExpr_LogicalAnd BinaryExpr_BinaryOp = 1 + BinaryExpr_LogicalOr BinaryExpr_BinaryOp = 2 +) + +// Enum value maps for BinaryExpr_BinaryOp. +var ( + BinaryExpr_BinaryOp_name = map[int32]string{ + 0: "Invalid", + 1: "LogicalAnd", + 2: "LogicalOr", + } + BinaryExpr_BinaryOp_value = map[string]int32{ + "Invalid": 0, + "LogicalAnd": 1, + "LogicalOr": 2, + } +) + +func (x BinaryExpr_BinaryOp) Enum() *BinaryExpr_BinaryOp { + p := new(BinaryExpr_BinaryOp) + *p = x + return p +} + +func (x BinaryExpr_BinaryOp) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (BinaryExpr_BinaryOp) Descriptor() protoreflect.EnumDescriptor { + return file_plan_proto_enumTypes[6].Descriptor() +} + +func (BinaryExpr_BinaryOp) Type() protoreflect.EnumType { + return &file_plan_proto_enumTypes[6] +} + +func (x BinaryExpr_BinaryOp) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use BinaryExpr_BinaryOp.Descriptor instead. +func (BinaryExpr_BinaryOp) EnumDescriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{16, 0} +} + +type GenericValue struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Val: + // + // *GenericValue_BoolVal + // *GenericValue_Int64Val + // *GenericValue_FloatVal + // *GenericValue_StringVal + // *GenericValue_ArrayVal + Val isGenericValue_Val `protobuf_oneof:"val"` +} + +func (x *GenericValue) Reset() { + *x = GenericValue{} + if protoimpl.UnsafeEnabled { + mi := &file_plan_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GenericValue) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GenericValue) ProtoMessage() {} + +func (x *GenericValue) ProtoReflect() protoreflect.Message { + mi := &file_plan_proto_msgTypes[0] + 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 GenericValue.ProtoReflect.Descriptor instead. +func (*GenericValue) Descriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{0} +} + +func (m *GenericValue) GetVal() isGenericValue_Val { + if m != nil { + return m.Val + } + return nil +} + +func (x *GenericValue) GetBoolVal() bool { + if x, ok := x.GetVal().(*GenericValue_BoolVal); ok { + return x.BoolVal + } + return false +} + +func (x *GenericValue) GetInt64Val() int64 { + if x, ok := x.GetVal().(*GenericValue_Int64Val); ok { + return x.Int64Val + } + return 0 +} + +func (x *GenericValue) GetFloatVal() float64 { + if x, ok := x.GetVal().(*GenericValue_FloatVal); ok { + return x.FloatVal + } + return 0 +} + +func (x *GenericValue) GetStringVal() string { + if x, ok := x.GetVal().(*GenericValue_StringVal); ok { + return x.StringVal + } + return "" +} + +func (x *GenericValue) GetArrayVal() *Array { + if x, ok := x.GetVal().(*GenericValue_ArrayVal); ok { + return x.ArrayVal + } + return nil +} + +type isGenericValue_Val interface { + isGenericValue_Val() +} + +type GenericValue_BoolVal struct { + BoolVal bool `protobuf:"varint,1,opt,name=bool_val,json=boolVal,proto3,oneof"` +} + +type GenericValue_Int64Val struct { + Int64Val int64 `protobuf:"varint,2,opt,name=int64_val,json=int64Val,proto3,oneof"` +} + +type GenericValue_FloatVal struct { + FloatVal float64 `protobuf:"fixed64,3,opt,name=float_val,json=floatVal,proto3,oneof"` +} + +type GenericValue_StringVal struct { + StringVal string `protobuf:"bytes,4,opt,name=string_val,json=stringVal,proto3,oneof"` +} + +type GenericValue_ArrayVal struct { + ArrayVal *Array `protobuf:"bytes,5,opt,name=array_val,json=arrayVal,proto3,oneof"` +} + +func (*GenericValue_BoolVal) isGenericValue_Val() {} + +func (*GenericValue_Int64Val) isGenericValue_Val() {} + +func (*GenericValue_FloatVal) isGenericValue_Val() {} + +func (*GenericValue_StringVal) isGenericValue_Val() {} + +func (*GenericValue_ArrayVal) isGenericValue_Val() {} + +type Array struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Array []*GenericValue `protobuf:"bytes,1,rep,name=array,proto3" json:"array,omitempty"` + SameType bool `protobuf:"varint,2,opt,name=same_type,json=sameType,proto3" json:"same_type,omitempty"` + ElementType schemapb.DataType `protobuf:"varint,3,opt,name=element_type,json=elementType,proto3,enum=milvus.proto.schema.DataType" json:"element_type,omitempty"` +} + +func (x *Array) Reset() { + *x = Array{} + if protoimpl.UnsafeEnabled { + mi := &file_plan_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Array) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Array) ProtoMessage() {} + +func (x *Array) ProtoReflect() protoreflect.Message { + mi := &file_plan_proto_msgTypes[1] + 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 Array.ProtoReflect.Descriptor instead. +func (*Array) Descriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{1} +} + +func (x *Array) GetArray() []*GenericValue { + if x != nil { + return x.Array + } + return nil +} + +func (x *Array) GetSameType() bool { + if x != nil { + return x.SameType + } + return false +} + +func (x *Array) GetElementType() schemapb.DataType { + if x != nil { + return x.ElementType + } + return schemapb.DataType(0) +} + +type SearchIteratorV2Info struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Token string `protobuf:"bytes,1,opt,name=token,proto3" json:"token,omitempty"` + BatchSize uint32 `protobuf:"varint,2,opt,name=batch_size,json=batchSize,proto3" json:"batch_size,omitempty"` + LastBound *float32 `protobuf:"fixed32,3,opt,name=last_bound,json=lastBound,proto3,oneof" json:"last_bound,omitempty"` +} + +func (x *SearchIteratorV2Info) Reset() { + *x = SearchIteratorV2Info{} + if protoimpl.UnsafeEnabled { + mi := &file_plan_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SearchIteratorV2Info) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SearchIteratorV2Info) ProtoMessage() {} + +func (x *SearchIteratorV2Info) ProtoReflect() protoreflect.Message { + mi := &file_plan_proto_msgTypes[2] + 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 SearchIteratorV2Info.ProtoReflect.Descriptor instead. +func (*SearchIteratorV2Info) Descriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{2} +} + +func (x *SearchIteratorV2Info) GetToken() string { + if x != nil { + return x.Token + } + return "" +} + +func (x *SearchIteratorV2Info) GetBatchSize() uint32 { + if x != nil { + return x.BatchSize + } + return 0 +} + +func (x *SearchIteratorV2Info) GetLastBound() float32 { + if x != nil && x.LastBound != nil { + return *x.LastBound + } + return 0 +} + +type QueryInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Topk int64 `protobuf:"varint,1,opt,name=topk,proto3" json:"topk,omitempty"` + MetricType string `protobuf:"bytes,3,opt,name=metric_type,json=metricType,proto3" json:"metric_type,omitempty"` + SearchParams string `protobuf:"bytes,4,opt,name=search_params,json=searchParams,proto3" json:"search_params,omitempty"` + RoundDecimal int64 `protobuf:"varint,5,opt,name=round_decimal,json=roundDecimal,proto3" json:"round_decimal,omitempty"` + GroupByFieldId int64 `protobuf:"varint,6,opt,name=group_by_field_id,json=groupByFieldId,proto3" json:"group_by_field_id,omitempty"` + MaterializedViewInvolved bool `protobuf:"varint,7,opt,name=materialized_view_involved,json=materializedViewInvolved,proto3" json:"materialized_view_involved,omitempty"` + GroupSize int64 `protobuf:"varint,8,opt,name=group_size,json=groupSize,proto3" json:"group_size,omitempty"` + StrictGroupSize bool `protobuf:"varint,9,opt,name=strict_group_size,json=strictGroupSize,proto3" json:"strict_group_size,omitempty"` + Bm25Avgdl float64 `protobuf:"fixed64,10,opt,name=bm25_avgdl,json=bm25Avgdl,proto3" json:"bm25_avgdl,omitempty"` + QueryFieldId int64 `protobuf:"varint,11,opt,name=query_field_id,json=queryFieldId,proto3" json:"query_field_id,omitempty"` + Hints string `protobuf:"bytes,12,opt,name=hints,proto3" json:"hints,omitempty"` + SearchIteratorV2Info *SearchIteratorV2Info `protobuf:"bytes,13,opt,name=search_iterator_v2_info,json=searchIteratorV2Info,proto3,oneof" json:"search_iterator_v2_info,omitempty"` +} + +func (x *QueryInfo) Reset() { + *x = QueryInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_plan_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *QueryInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryInfo) ProtoMessage() {} + +func (x *QueryInfo) ProtoReflect() protoreflect.Message { + mi := &file_plan_proto_msgTypes[3] + 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 QueryInfo.ProtoReflect.Descriptor instead. +func (*QueryInfo) Descriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{3} +} + +func (x *QueryInfo) GetTopk() int64 { + if x != nil { + return x.Topk + } + return 0 +} + +func (x *QueryInfo) GetMetricType() string { + if x != nil { + return x.MetricType + } + return "" +} + +func (x *QueryInfo) GetSearchParams() string { + if x != nil { + return x.SearchParams + } + return "" +} + +func (x *QueryInfo) GetRoundDecimal() int64 { + if x != nil { + return x.RoundDecimal + } + return 0 +} + +func (x *QueryInfo) GetGroupByFieldId() int64 { + if x != nil { + return x.GroupByFieldId + } + return 0 +} + +func (x *QueryInfo) GetMaterializedViewInvolved() bool { + if x != nil { + return x.MaterializedViewInvolved + } + return false +} + +func (x *QueryInfo) GetGroupSize() int64 { + if x != nil { + return x.GroupSize + } + return 0 +} + +func (x *QueryInfo) GetStrictGroupSize() bool { + if x != nil { + return x.StrictGroupSize + } + return false +} + +func (x *QueryInfo) GetBm25Avgdl() float64 { + if x != nil { + return x.Bm25Avgdl + } + return 0 +} + +func (x *QueryInfo) GetQueryFieldId() int64 { + if x != nil { + return x.QueryFieldId + } + return 0 +} + +func (x *QueryInfo) GetHints() string { + if x != nil { + return x.Hints + } + return "" +} + +func (x *QueryInfo) GetSearchIteratorV2Info() *SearchIteratorV2Info { + if x != nil { + return x.SearchIteratorV2Info + } + return nil +} + +type ColumnInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + FieldId int64 `protobuf:"varint,1,opt,name=field_id,json=fieldId,proto3" json:"field_id,omitempty"` + DataType schemapb.DataType `protobuf:"varint,2,opt,name=data_type,json=dataType,proto3,enum=milvus.proto.schema.DataType" json:"data_type,omitempty"` + IsPrimaryKey bool `protobuf:"varint,3,opt,name=is_primary_key,json=isPrimaryKey,proto3" json:"is_primary_key,omitempty"` + IsAutoID bool `protobuf:"varint,4,opt,name=is_autoID,json=isAutoID,proto3" json:"is_autoID,omitempty"` + NestedPath []string `protobuf:"bytes,5,rep,name=nested_path,json=nestedPath,proto3" json:"nested_path,omitempty"` + IsPartitionKey bool `protobuf:"varint,6,opt,name=is_partition_key,json=isPartitionKey,proto3" json:"is_partition_key,omitempty"` + ElementType schemapb.DataType `protobuf:"varint,7,opt,name=element_type,json=elementType,proto3,enum=milvus.proto.schema.DataType" json:"element_type,omitempty"` + IsClusteringKey bool `protobuf:"varint,8,opt,name=is_clustering_key,json=isClusteringKey,proto3" json:"is_clustering_key,omitempty"` + Nullable bool `protobuf:"varint,9,opt,name=nullable,proto3" json:"nullable,omitempty"` +} + +func (x *ColumnInfo) Reset() { + *x = ColumnInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_plan_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ColumnInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ColumnInfo) ProtoMessage() {} + +func (x *ColumnInfo) ProtoReflect() protoreflect.Message { + mi := &file_plan_proto_msgTypes[4] + 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 ColumnInfo.ProtoReflect.Descriptor instead. +func (*ColumnInfo) Descriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{4} +} + +func (x *ColumnInfo) GetFieldId() int64 { + if x != nil { + return x.FieldId + } + return 0 +} + +func (x *ColumnInfo) GetDataType() schemapb.DataType { + if x != nil { + return x.DataType + } + return schemapb.DataType(0) +} + +func (x *ColumnInfo) GetIsPrimaryKey() bool { + if x != nil { + return x.IsPrimaryKey + } + return false +} + +func (x *ColumnInfo) GetIsAutoID() bool { + if x != nil { + return x.IsAutoID + } + return false +} + +func (x *ColumnInfo) GetNestedPath() []string { + if x != nil { + return x.NestedPath + } + return nil +} + +func (x *ColumnInfo) GetIsPartitionKey() bool { + if x != nil { + return x.IsPartitionKey + } + return false +} + +func (x *ColumnInfo) GetElementType() schemapb.DataType { + if x != nil { + return x.ElementType + } + return schemapb.DataType(0) +} + +func (x *ColumnInfo) GetIsClusteringKey() bool { + if x != nil { + return x.IsClusteringKey + } + return false +} + +func (x *ColumnInfo) GetNullable() bool { + if x != nil { + return x.Nullable + } + return false +} + +type ColumnExpr struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Info *ColumnInfo `protobuf:"bytes,1,opt,name=info,proto3" json:"info,omitempty"` +} + +func (x *ColumnExpr) Reset() { + *x = ColumnExpr{} + if protoimpl.UnsafeEnabled { + mi := &file_plan_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ColumnExpr) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ColumnExpr) ProtoMessage() {} + +func (x *ColumnExpr) ProtoReflect() protoreflect.Message { + mi := &file_plan_proto_msgTypes[5] + 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 ColumnExpr.ProtoReflect.Descriptor instead. +func (*ColumnExpr) Descriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{5} +} + +func (x *ColumnExpr) GetInfo() *ColumnInfo { + if x != nil { + return x.Info + } + return nil +} + +type ExistsExpr struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Info *ColumnInfo `protobuf:"bytes,1,opt,name=info,proto3" json:"info,omitempty"` +} + +func (x *ExistsExpr) Reset() { + *x = ExistsExpr{} + if protoimpl.UnsafeEnabled { + mi := &file_plan_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ExistsExpr) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ExistsExpr) ProtoMessage() {} + +func (x *ExistsExpr) ProtoReflect() protoreflect.Message { + mi := &file_plan_proto_msgTypes[6] + 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 ExistsExpr.ProtoReflect.Descriptor instead. +func (*ExistsExpr) Descriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{6} +} + +func (x *ExistsExpr) GetInfo() *ColumnInfo { + if x != nil { + return x.Info + } + return nil +} + +type ValueExpr struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Value *GenericValue `protobuf:"bytes,1,opt,name=value,proto3" json:"value,omitempty"` + TemplateVariableName string `protobuf:"bytes,2,opt,name=template_variable_name,json=templateVariableName,proto3" json:"template_variable_name,omitempty"` +} + +func (x *ValueExpr) Reset() { + *x = ValueExpr{} + if protoimpl.UnsafeEnabled { + mi := &file_plan_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ValueExpr) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ValueExpr) ProtoMessage() {} + +func (x *ValueExpr) ProtoReflect() protoreflect.Message { + mi := &file_plan_proto_msgTypes[7] + 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 ValueExpr.ProtoReflect.Descriptor instead. +func (*ValueExpr) Descriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{7} +} + +func (x *ValueExpr) GetValue() *GenericValue { + if x != nil { + return x.Value + } + return nil +} + +func (x *ValueExpr) GetTemplateVariableName() string { + if x != nil { + return x.TemplateVariableName + } + return "" +} + +type UnaryRangeExpr struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ColumnInfo *ColumnInfo `protobuf:"bytes,1,opt,name=column_info,json=columnInfo,proto3" json:"column_info,omitempty"` + Op OpType `protobuf:"varint,2,opt,name=op,proto3,enum=milvus.proto.plan.OpType" json:"op,omitempty"` + Value *GenericValue `protobuf:"bytes,3,opt,name=value,proto3" json:"value,omitempty"` + TemplateVariableName string `protobuf:"bytes,4,opt,name=template_variable_name,json=templateVariableName,proto3" json:"template_variable_name,omitempty"` +} + +func (x *UnaryRangeExpr) Reset() { + *x = UnaryRangeExpr{} + if protoimpl.UnsafeEnabled { + mi := &file_plan_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UnaryRangeExpr) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UnaryRangeExpr) ProtoMessage() {} + +func (x *UnaryRangeExpr) ProtoReflect() protoreflect.Message { + mi := &file_plan_proto_msgTypes[8] + 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 UnaryRangeExpr.ProtoReflect.Descriptor instead. +func (*UnaryRangeExpr) Descriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{8} +} + +func (x *UnaryRangeExpr) GetColumnInfo() *ColumnInfo { + if x != nil { + return x.ColumnInfo + } + return nil +} + +func (x *UnaryRangeExpr) GetOp() OpType { + if x != nil { + return x.Op + } + return OpType_Invalid +} + +func (x *UnaryRangeExpr) GetValue() *GenericValue { + if x != nil { + return x.Value + } + return nil +} + +func (x *UnaryRangeExpr) GetTemplateVariableName() string { + if x != nil { + return x.TemplateVariableName + } + return "" +} + +type BinaryRangeExpr struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ColumnInfo *ColumnInfo `protobuf:"bytes,1,opt,name=column_info,json=columnInfo,proto3" json:"column_info,omitempty"` + LowerInclusive bool `protobuf:"varint,2,opt,name=lower_inclusive,json=lowerInclusive,proto3" json:"lower_inclusive,omitempty"` + UpperInclusive bool `protobuf:"varint,3,opt,name=upper_inclusive,json=upperInclusive,proto3" json:"upper_inclusive,omitempty"` + LowerValue *GenericValue `protobuf:"bytes,4,opt,name=lower_value,json=lowerValue,proto3" json:"lower_value,omitempty"` + UpperValue *GenericValue `protobuf:"bytes,5,opt,name=upper_value,json=upperValue,proto3" json:"upper_value,omitempty"` + LowerTemplateVariableName string `protobuf:"bytes,6,opt,name=lower_template_variable_name,json=lowerTemplateVariableName,proto3" json:"lower_template_variable_name,omitempty"` + UpperTemplateVariableName string `protobuf:"bytes,7,opt,name=upper_template_variable_name,json=upperTemplateVariableName,proto3" json:"upper_template_variable_name,omitempty"` +} + +func (x *BinaryRangeExpr) Reset() { + *x = BinaryRangeExpr{} + if protoimpl.UnsafeEnabled { + mi := &file_plan_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BinaryRangeExpr) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BinaryRangeExpr) ProtoMessage() {} + +func (x *BinaryRangeExpr) ProtoReflect() protoreflect.Message { + mi := &file_plan_proto_msgTypes[9] + 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 BinaryRangeExpr.ProtoReflect.Descriptor instead. +func (*BinaryRangeExpr) Descriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{9} +} + +func (x *BinaryRangeExpr) GetColumnInfo() *ColumnInfo { + if x != nil { + return x.ColumnInfo + } + return nil +} + +func (x *BinaryRangeExpr) GetLowerInclusive() bool { + if x != nil { + return x.LowerInclusive + } + return false +} + +func (x *BinaryRangeExpr) GetUpperInclusive() bool { + if x != nil { + return x.UpperInclusive + } + return false +} + +func (x *BinaryRangeExpr) GetLowerValue() *GenericValue { + if x != nil { + return x.LowerValue + } + return nil +} + +func (x *BinaryRangeExpr) GetUpperValue() *GenericValue { + if x != nil { + return x.UpperValue + } + return nil +} + +func (x *BinaryRangeExpr) GetLowerTemplateVariableName() string { + if x != nil { + return x.LowerTemplateVariableName + } + return "" +} + +func (x *BinaryRangeExpr) GetUpperTemplateVariableName() string { + if x != nil { + return x.UpperTemplateVariableName + } + return "" +} + +type CallExpr struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + FunctionName string `protobuf:"bytes,1,opt,name=function_name,json=functionName,proto3" json:"function_name,omitempty"` + FunctionParameters []*Expr `protobuf:"bytes,2,rep,name=function_parameters,json=functionParameters,proto3" json:"function_parameters,omitempty"` +} + +func (x *CallExpr) Reset() { + *x = CallExpr{} + if protoimpl.UnsafeEnabled { + mi := &file_plan_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CallExpr) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CallExpr) ProtoMessage() {} + +func (x *CallExpr) ProtoReflect() protoreflect.Message { + mi := &file_plan_proto_msgTypes[10] + 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 CallExpr.ProtoReflect.Descriptor instead. +func (*CallExpr) Descriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{10} +} + +func (x *CallExpr) GetFunctionName() string { + if x != nil { + return x.FunctionName + } + return "" +} + +func (x *CallExpr) GetFunctionParameters() []*Expr { + if x != nil { + return x.FunctionParameters + } + return nil +} + +type CompareExpr struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + LeftColumnInfo *ColumnInfo `protobuf:"bytes,1,opt,name=left_column_info,json=leftColumnInfo,proto3" json:"left_column_info,omitempty"` + RightColumnInfo *ColumnInfo `protobuf:"bytes,2,opt,name=right_column_info,json=rightColumnInfo,proto3" json:"right_column_info,omitempty"` + Op OpType `protobuf:"varint,3,opt,name=op,proto3,enum=milvus.proto.plan.OpType" json:"op,omitempty"` +} + +func (x *CompareExpr) Reset() { + *x = CompareExpr{} + if protoimpl.UnsafeEnabled { + mi := &file_plan_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CompareExpr) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CompareExpr) ProtoMessage() {} + +func (x *CompareExpr) ProtoReflect() protoreflect.Message { + mi := &file_plan_proto_msgTypes[11] + 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 CompareExpr.ProtoReflect.Descriptor instead. +func (*CompareExpr) Descriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{11} +} + +func (x *CompareExpr) GetLeftColumnInfo() *ColumnInfo { + if x != nil { + return x.LeftColumnInfo + } + return nil +} + +func (x *CompareExpr) GetRightColumnInfo() *ColumnInfo { + if x != nil { + return x.RightColumnInfo + } + return nil +} + +func (x *CompareExpr) GetOp() OpType { + if x != nil { + return x.Op + } + return OpType_Invalid +} + +type TermExpr struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ColumnInfo *ColumnInfo `protobuf:"bytes,1,opt,name=column_info,json=columnInfo,proto3" json:"column_info,omitempty"` + Values []*GenericValue `protobuf:"bytes,2,rep,name=values,proto3" json:"values,omitempty"` + IsInField bool `protobuf:"varint,3,opt,name=is_in_field,json=isInField,proto3" json:"is_in_field,omitempty"` + TemplateVariableName string `protobuf:"bytes,4,opt,name=template_variable_name,json=templateVariableName,proto3" json:"template_variable_name,omitempty"` +} + +func (x *TermExpr) Reset() { + *x = TermExpr{} + if protoimpl.UnsafeEnabled { + mi := &file_plan_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TermExpr) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TermExpr) ProtoMessage() {} + +func (x *TermExpr) ProtoReflect() protoreflect.Message { + mi := &file_plan_proto_msgTypes[12] + 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 TermExpr.ProtoReflect.Descriptor instead. +func (*TermExpr) Descriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{12} +} + +func (x *TermExpr) GetColumnInfo() *ColumnInfo { + if x != nil { + return x.ColumnInfo + } + return nil +} + +func (x *TermExpr) GetValues() []*GenericValue { + if x != nil { + return x.Values + } + return nil +} + +func (x *TermExpr) GetIsInField() bool { + if x != nil { + return x.IsInField + } + return false +} + +func (x *TermExpr) GetTemplateVariableName() string { + if x != nil { + return x.TemplateVariableName + } + return "" +} + +type JSONContainsExpr struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ColumnInfo *ColumnInfo `protobuf:"bytes,1,opt,name=column_info,json=columnInfo,proto3" json:"column_info,omitempty"` + Elements []*GenericValue `protobuf:"bytes,2,rep,name=elements,proto3" json:"elements,omitempty"` + Op JSONContainsExpr_JSONOp `protobuf:"varint,3,opt,name=op,proto3,enum=milvus.proto.plan.JSONContainsExpr_JSONOp" json:"op,omitempty"` + ElementsSameType bool `protobuf:"varint,4,opt,name=elements_same_type,json=elementsSameType,proto3" json:"elements_same_type,omitempty"` + TemplateVariableName string `protobuf:"bytes,5,opt,name=template_variable_name,json=templateVariableName,proto3" json:"template_variable_name,omitempty"` +} + +func (x *JSONContainsExpr) Reset() { + *x = JSONContainsExpr{} + if protoimpl.UnsafeEnabled { + mi := &file_plan_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *JSONContainsExpr) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*JSONContainsExpr) ProtoMessage() {} + +func (x *JSONContainsExpr) ProtoReflect() protoreflect.Message { + mi := &file_plan_proto_msgTypes[13] + 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 JSONContainsExpr.ProtoReflect.Descriptor instead. +func (*JSONContainsExpr) Descriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{13} +} + +func (x *JSONContainsExpr) GetColumnInfo() *ColumnInfo { + if x != nil { + return x.ColumnInfo + } + return nil +} + +func (x *JSONContainsExpr) GetElements() []*GenericValue { + if x != nil { + return x.Elements + } + return nil +} + +func (x *JSONContainsExpr) GetOp() JSONContainsExpr_JSONOp { + if x != nil { + return x.Op + } + return JSONContainsExpr_Invalid +} + +func (x *JSONContainsExpr) GetElementsSameType() bool { + if x != nil { + return x.ElementsSameType + } + return false +} + +func (x *JSONContainsExpr) GetTemplateVariableName() string { + if x != nil { + return x.TemplateVariableName + } + return "" +} + +type NullExpr struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ColumnInfo *ColumnInfo `protobuf:"bytes,1,opt,name=column_info,json=columnInfo,proto3" json:"column_info,omitempty"` + Op NullExpr_NullOp `protobuf:"varint,2,opt,name=op,proto3,enum=milvus.proto.plan.NullExpr_NullOp" json:"op,omitempty"` +} + +func (x *NullExpr) Reset() { + *x = NullExpr{} + if protoimpl.UnsafeEnabled { + mi := &file_plan_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NullExpr) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NullExpr) ProtoMessage() {} + +func (x *NullExpr) ProtoReflect() protoreflect.Message { + mi := &file_plan_proto_msgTypes[14] + 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 NullExpr.ProtoReflect.Descriptor instead. +func (*NullExpr) Descriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{14} +} + +func (x *NullExpr) GetColumnInfo() *ColumnInfo { + if x != nil { + return x.ColumnInfo + } + return nil +} + +func (x *NullExpr) GetOp() NullExpr_NullOp { + if x != nil { + return x.Op + } + return NullExpr_Invalid +} + +type UnaryExpr struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Op UnaryExpr_UnaryOp `protobuf:"varint,1,opt,name=op,proto3,enum=milvus.proto.plan.UnaryExpr_UnaryOp" json:"op,omitempty"` + Child *Expr `protobuf:"bytes,2,opt,name=child,proto3" json:"child,omitempty"` +} + +func (x *UnaryExpr) Reset() { + *x = UnaryExpr{} + if protoimpl.UnsafeEnabled { + mi := &file_plan_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UnaryExpr) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UnaryExpr) ProtoMessage() {} + +func (x *UnaryExpr) ProtoReflect() protoreflect.Message { + mi := &file_plan_proto_msgTypes[15] + 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 UnaryExpr.ProtoReflect.Descriptor instead. +func (*UnaryExpr) Descriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{15} +} + +func (x *UnaryExpr) GetOp() UnaryExpr_UnaryOp { + if x != nil { + return x.Op + } + return UnaryExpr_Invalid +} + +func (x *UnaryExpr) GetChild() *Expr { + if x != nil { + return x.Child + } + return nil +} + +type BinaryExpr struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Op BinaryExpr_BinaryOp `protobuf:"varint,1,opt,name=op,proto3,enum=milvus.proto.plan.BinaryExpr_BinaryOp" json:"op,omitempty"` + Left *Expr `protobuf:"bytes,2,opt,name=left,proto3" json:"left,omitempty"` + Right *Expr `protobuf:"bytes,3,opt,name=right,proto3" json:"right,omitempty"` +} + +func (x *BinaryExpr) Reset() { + *x = BinaryExpr{} + if protoimpl.UnsafeEnabled { + mi := &file_plan_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BinaryExpr) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BinaryExpr) ProtoMessage() {} + +func (x *BinaryExpr) ProtoReflect() protoreflect.Message { + mi := &file_plan_proto_msgTypes[16] + 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 BinaryExpr.ProtoReflect.Descriptor instead. +func (*BinaryExpr) Descriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{16} +} + +func (x *BinaryExpr) GetOp() BinaryExpr_BinaryOp { + if x != nil { + return x.Op + } + return BinaryExpr_Invalid +} + +func (x *BinaryExpr) GetLeft() *Expr { + if x != nil { + return x.Left + } + return nil +} + +func (x *BinaryExpr) GetRight() *Expr { + if x != nil { + return x.Right + } + return nil +} + +type BinaryArithOp struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ColumnInfo *ColumnInfo `protobuf:"bytes,1,opt,name=column_info,json=columnInfo,proto3" json:"column_info,omitempty"` + ArithOp ArithOpType `protobuf:"varint,2,opt,name=arith_op,json=arithOp,proto3,enum=milvus.proto.plan.ArithOpType" json:"arith_op,omitempty"` + RightOperand *GenericValue `protobuf:"bytes,3,opt,name=right_operand,json=rightOperand,proto3" json:"right_operand,omitempty"` +} + +func (x *BinaryArithOp) Reset() { + *x = BinaryArithOp{} + if protoimpl.UnsafeEnabled { + mi := &file_plan_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BinaryArithOp) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BinaryArithOp) ProtoMessage() {} + +func (x *BinaryArithOp) ProtoReflect() protoreflect.Message { + mi := &file_plan_proto_msgTypes[17] + 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 BinaryArithOp.ProtoReflect.Descriptor instead. +func (*BinaryArithOp) Descriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{17} +} + +func (x *BinaryArithOp) GetColumnInfo() *ColumnInfo { + if x != nil { + return x.ColumnInfo + } + return nil +} + +func (x *BinaryArithOp) GetArithOp() ArithOpType { + if x != nil { + return x.ArithOp + } + return ArithOpType_Unknown +} + +func (x *BinaryArithOp) GetRightOperand() *GenericValue { + if x != nil { + return x.RightOperand + } + return nil +} + +type BinaryArithExpr struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Left *Expr `protobuf:"bytes,1,opt,name=left,proto3" json:"left,omitempty"` + Right *Expr `protobuf:"bytes,2,opt,name=right,proto3" json:"right,omitempty"` + Op ArithOpType `protobuf:"varint,3,opt,name=op,proto3,enum=milvus.proto.plan.ArithOpType" json:"op,omitempty"` +} + +func (x *BinaryArithExpr) Reset() { + *x = BinaryArithExpr{} + if protoimpl.UnsafeEnabled { + mi := &file_plan_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BinaryArithExpr) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BinaryArithExpr) ProtoMessage() {} + +func (x *BinaryArithExpr) ProtoReflect() protoreflect.Message { + mi := &file_plan_proto_msgTypes[18] + 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 BinaryArithExpr.ProtoReflect.Descriptor instead. +func (*BinaryArithExpr) Descriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{18} +} + +func (x *BinaryArithExpr) GetLeft() *Expr { + if x != nil { + return x.Left + } + return nil +} + +func (x *BinaryArithExpr) GetRight() *Expr { + if x != nil { + return x.Right + } + return nil +} + +func (x *BinaryArithExpr) GetOp() ArithOpType { + if x != nil { + return x.Op + } + return ArithOpType_Unknown +} + +type BinaryArithOpEvalRangeExpr struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ColumnInfo *ColumnInfo `protobuf:"bytes,1,opt,name=column_info,json=columnInfo,proto3" json:"column_info,omitempty"` + ArithOp ArithOpType `protobuf:"varint,2,opt,name=arith_op,json=arithOp,proto3,enum=milvus.proto.plan.ArithOpType" json:"arith_op,omitempty"` + RightOperand *GenericValue `protobuf:"bytes,3,opt,name=right_operand,json=rightOperand,proto3" json:"right_operand,omitempty"` + Op OpType `protobuf:"varint,4,opt,name=op,proto3,enum=milvus.proto.plan.OpType" json:"op,omitempty"` + Value *GenericValue `protobuf:"bytes,5,opt,name=value,proto3" json:"value,omitempty"` + OperandTemplateVariableName string `protobuf:"bytes,6,opt,name=operand_template_variable_name,json=operandTemplateVariableName,proto3" json:"operand_template_variable_name,omitempty"` + ValueTemplateVariableName string `protobuf:"bytes,7,opt,name=value_template_variable_name,json=valueTemplateVariableName,proto3" json:"value_template_variable_name,omitempty"` +} + +func (x *BinaryArithOpEvalRangeExpr) Reset() { + *x = BinaryArithOpEvalRangeExpr{} + if protoimpl.UnsafeEnabled { + mi := &file_plan_proto_msgTypes[19] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BinaryArithOpEvalRangeExpr) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BinaryArithOpEvalRangeExpr) ProtoMessage() {} + +func (x *BinaryArithOpEvalRangeExpr) ProtoReflect() protoreflect.Message { + mi := &file_plan_proto_msgTypes[19] + 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 BinaryArithOpEvalRangeExpr.ProtoReflect.Descriptor instead. +func (*BinaryArithOpEvalRangeExpr) Descriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{19} +} + +func (x *BinaryArithOpEvalRangeExpr) GetColumnInfo() *ColumnInfo { + if x != nil { + return x.ColumnInfo + } + return nil +} + +func (x *BinaryArithOpEvalRangeExpr) GetArithOp() ArithOpType { + if x != nil { + return x.ArithOp + } + return ArithOpType_Unknown +} + +func (x *BinaryArithOpEvalRangeExpr) GetRightOperand() *GenericValue { + if x != nil { + return x.RightOperand + } + return nil +} + +func (x *BinaryArithOpEvalRangeExpr) GetOp() OpType { + if x != nil { + return x.Op + } + return OpType_Invalid +} + +func (x *BinaryArithOpEvalRangeExpr) GetValue() *GenericValue { + if x != nil { + return x.Value + } + return nil +} + +func (x *BinaryArithOpEvalRangeExpr) GetOperandTemplateVariableName() string { + if x != nil { + return x.OperandTemplateVariableName + } + return "" +} + +func (x *BinaryArithOpEvalRangeExpr) GetValueTemplateVariableName() string { + if x != nil { + return x.ValueTemplateVariableName + } + return "" +} + +type AlwaysTrueExpr struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *AlwaysTrueExpr) Reset() { + *x = AlwaysTrueExpr{} + if protoimpl.UnsafeEnabled { + mi := &file_plan_proto_msgTypes[20] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AlwaysTrueExpr) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AlwaysTrueExpr) ProtoMessage() {} + +func (x *AlwaysTrueExpr) ProtoReflect() protoreflect.Message { + mi := &file_plan_proto_msgTypes[20] + 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 AlwaysTrueExpr.ProtoReflect.Descriptor instead. +func (*AlwaysTrueExpr) Descriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{20} +} + +type Expr struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Expr: + // + // *Expr_TermExpr + // *Expr_UnaryExpr + // *Expr_BinaryExpr + // *Expr_CompareExpr + // *Expr_UnaryRangeExpr + // *Expr_BinaryRangeExpr + // *Expr_BinaryArithOpEvalRangeExpr + // *Expr_BinaryArithExpr + // *Expr_ValueExpr + // *Expr_ColumnExpr + // *Expr_ExistsExpr + // *Expr_AlwaysTrueExpr + // *Expr_JsonContainsExpr + // *Expr_CallExpr + // *Expr_NullExpr + Expr isExpr_Expr `protobuf_oneof:"expr"` + IsTemplate bool `protobuf:"varint,20,opt,name=is_template,json=isTemplate,proto3" json:"is_template,omitempty"` +} + +func (x *Expr) Reset() { + *x = Expr{} + if protoimpl.UnsafeEnabled { + mi := &file_plan_proto_msgTypes[21] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Expr) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Expr) ProtoMessage() {} + +func (x *Expr) ProtoReflect() protoreflect.Message { + mi := &file_plan_proto_msgTypes[21] + 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 Expr.ProtoReflect.Descriptor instead. +func (*Expr) Descriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{21} +} + +func (m *Expr) GetExpr() isExpr_Expr { + if m != nil { + return m.Expr + } + return nil +} + +func (x *Expr) GetTermExpr() *TermExpr { + if x, ok := x.GetExpr().(*Expr_TermExpr); ok { + return x.TermExpr + } + return nil +} + +func (x *Expr) GetUnaryExpr() *UnaryExpr { + if x, ok := x.GetExpr().(*Expr_UnaryExpr); ok { + return x.UnaryExpr + } + return nil +} + +func (x *Expr) GetBinaryExpr() *BinaryExpr { + if x, ok := x.GetExpr().(*Expr_BinaryExpr); ok { + return x.BinaryExpr + } + return nil +} + +func (x *Expr) GetCompareExpr() *CompareExpr { + if x, ok := x.GetExpr().(*Expr_CompareExpr); ok { + return x.CompareExpr + } + return nil +} + +func (x *Expr) GetUnaryRangeExpr() *UnaryRangeExpr { + if x, ok := x.GetExpr().(*Expr_UnaryRangeExpr); ok { + return x.UnaryRangeExpr + } + return nil +} + +func (x *Expr) GetBinaryRangeExpr() *BinaryRangeExpr { + if x, ok := x.GetExpr().(*Expr_BinaryRangeExpr); ok { + return x.BinaryRangeExpr + } + return nil +} + +func (x *Expr) GetBinaryArithOpEvalRangeExpr() *BinaryArithOpEvalRangeExpr { + if x, ok := x.GetExpr().(*Expr_BinaryArithOpEvalRangeExpr); ok { + return x.BinaryArithOpEvalRangeExpr + } + return nil +} + +func (x *Expr) GetBinaryArithExpr() *BinaryArithExpr { + if x, ok := x.GetExpr().(*Expr_BinaryArithExpr); ok { + return x.BinaryArithExpr + } + return nil +} + +func (x *Expr) GetValueExpr() *ValueExpr { + if x, ok := x.GetExpr().(*Expr_ValueExpr); ok { + return x.ValueExpr + } + return nil +} + +func (x *Expr) GetColumnExpr() *ColumnExpr { + if x, ok := x.GetExpr().(*Expr_ColumnExpr); ok { + return x.ColumnExpr + } + return nil +} + +func (x *Expr) GetExistsExpr() *ExistsExpr { + if x, ok := x.GetExpr().(*Expr_ExistsExpr); ok { + return x.ExistsExpr + } + return nil +} + +func (x *Expr) GetAlwaysTrueExpr() *AlwaysTrueExpr { + if x, ok := x.GetExpr().(*Expr_AlwaysTrueExpr); ok { + return x.AlwaysTrueExpr + } + return nil +} + +func (x *Expr) GetJsonContainsExpr() *JSONContainsExpr { + if x, ok := x.GetExpr().(*Expr_JsonContainsExpr); ok { + return x.JsonContainsExpr + } + return nil +} + +func (x *Expr) GetCallExpr() *CallExpr { + if x, ok := x.GetExpr().(*Expr_CallExpr); ok { + return x.CallExpr + } + return nil +} + +func (x *Expr) GetNullExpr() *NullExpr { + if x, ok := x.GetExpr().(*Expr_NullExpr); ok { + return x.NullExpr + } + return nil +} + +func (x *Expr) GetIsTemplate() bool { + if x != nil { + return x.IsTemplate + } + return false +} + +type isExpr_Expr interface { + isExpr_Expr() +} + +type Expr_TermExpr struct { + TermExpr *TermExpr `protobuf:"bytes,1,opt,name=term_expr,json=termExpr,proto3,oneof"` +} + +type Expr_UnaryExpr struct { + UnaryExpr *UnaryExpr `protobuf:"bytes,2,opt,name=unary_expr,json=unaryExpr,proto3,oneof"` +} + +type Expr_BinaryExpr struct { + BinaryExpr *BinaryExpr `protobuf:"bytes,3,opt,name=binary_expr,json=binaryExpr,proto3,oneof"` +} + +type Expr_CompareExpr struct { + CompareExpr *CompareExpr `protobuf:"bytes,4,opt,name=compare_expr,json=compareExpr,proto3,oneof"` +} + +type Expr_UnaryRangeExpr struct { + UnaryRangeExpr *UnaryRangeExpr `protobuf:"bytes,5,opt,name=unary_range_expr,json=unaryRangeExpr,proto3,oneof"` +} + +type Expr_BinaryRangeExpr struct { + BinaryRangeExpr *BinaryRangeExpr `protobuf:"bytes,6,opt,name=binary_range_expr,json=binaryRangeExpr,proto3,oneof"` +} + +type Expr_BinaryArithOpEvalRangeExpr struct { + BinaryArithOpEvalRangeExpr *BinaryArithOpEvalRangeExpr `protobuf:"bytes,7,opt,name=binary_arith_op_eval_range_expr,json=binaryArithOpEvalRangeExpr,proto3,oneof"` +} + +type Expr_BinaryArithExpr struct { + BinaryArithExpr *BinaryArithExpr `protobuf:"bytes,8,opt,name=binary_arith_expr,json=binaryArithExpr,proto3,oneof"` +} + +type Expr_ValueExpr struct { + ValueExpr *ValueExpr `protobuf:"bytes,9,opt,name=value_expr,json=valueExpr,proto3,oneof"` +} + +type Expr_ColumnExpr struct { + ColumnExpr *ColumnExpr `protobuf:"bytes,10,opt,name=column_expr,json=columnExpr,proto3,oneof"` +} + +type Expr_ExistsExpr struct { + ExistsExpr *ExistsExpr `protobuf:"bytes,11,opt,name=exists_expr,json=existsExpr,proto3,oneof"` +} + +type Expr_AlwaysTrueExpr struct { + AlwaysTrueExpr *AlwaysTrueExpr `protobuf:"bytes,12,opt,name=always_true_expr,json=alwaysTrueExpr,proto3,oneof"` +} + +type Expr_JsonContainsExpr struct { + JsonContainsExpr *JSONContainsExpr `protobuf:"bytes,13,opt,name=json_contains_expr,json=jsonContainsExpr,proto3,oneof"` +} + +type Expr_CallExpr struct { + CallExpr *CallExpr `protobuf:"bytes,14,opt,name=call_expr,json=callExpr,proto3,oneof"` +} + +type Expr_NullExpr struct { + NullExpr *NullExpr `protobuf:"bytes,15,opt,name=null_expr,json=nullExpr,proto3,oneof"` +} + +func (*Expr_TermExpr) isExpr_Expr() {} + +func (*Expr_UnaryExpr) isExpr_Expr() {} + +func (*Expr_BinaryExpr) isExpr_Expr() {} + +func (*Expr_CompareExpr) isExpr_Expr() {} + +func (*Expr_UnaryRangeExpr) isExpr_Expr() {} + +func (*Expr_BinaryRangeExpr) isExpr_Expr() {} + +func (*Expr_BinaryArithOpEvalRangeExpr) isExpr_Expr() {} + +func (*Expr_BinaryArithExpr) isExpr_Expr() {} + +func (*Expr_ValueExpr) isExpr_Expr() {} + +func (*Expr_ColumnExpr) isExpr_Expr() {} + +func (*Expr_ExistsExpr) isExpr_Expr() {} + +func (*Expr_AlwaysTrueExpr) isExpr_Expr() {} + +func (*Expr_JsonContainsExpr) isExpr_Expr() {} + +func (*Expr_CallExpr) isExpr_Expr() {} + +func (*Expr_NullExpr) isExpr_Expr() {} + +type VectorANNS struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + VectorType VectorType `protobuf:"varint,1,opt,name=vector_type,json=vectorType,proto3,enum=milvus.proto.plan.VectorType" json:"vector_type,omitempty"` + FieldId int64 `protobuf:"varint,2,opt,name=field_id,json=fieldId,proto3" json:"field_id,omitempty"` + Predicates *Expr `protobuf:"bytes,3,opt,name=predicates,proto3" json:"predicates,omitempty"` + QueryInfo *QueryInfo `protobuf:"bytes,4,opt,name=query_info,json=queryInfo,proto3" json:"query_info,omitempty"` + PlaceholderTag string `protobuf:"bytes,5,opt,name=placeholder_tag,json=placeholderTag,proto3" json:"placeholder_tag,omitempty"` // always be "$0" +} + +func (x *VectorANNS) Reset() { + *x = VectorANNS{} + if protoimpl.UnsafeEnabled { + mi := &file_plan_proto_msgTypes[22] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *VectorANNS) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*VectorANNS) ProtoMessage() {} + +func (x *VectorANNS) ProtoReflect() protoreflect.Message { + mi := &file_plan_proto_msgTypes[22] + 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 VectorANNS.ProtoReflect.Descriptor instead. +func (*VectorANNS) Descriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{22} +} + +func (x *VectorANNS) GetVectorType() VectorType { + if x != nil { + return x.VectorType + } + return VectorType_BinaryVector +} + +func (x *VectorANNS) GetFieldId() int64 { + if x != nil { + return x.FieldId + } + return 0 +} + +func (x *VectorANNS) GetPredicates() *Expr { + if x != nil { + return x.Predicates + } + return nil +} + +func (x *VectorANNS) GetQueryInfo() *QueryInfo { + if x != nil { + return x.QueryInfo + } + return nil +} + +func (x *VectorANNS) GetPlaceholderTag() string { + if x != nil { + return x.PlaceholderTag + } + return "" +} + +type QueryPlanNode struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Predicates *Expr `protobuf:"bytes,1,opt,name=predicates,proto3" json:"predicates,omitempty"` + IsCount bool `protobuf:"varint,2,opt,name=is_count,json=isCount,proto3" json:"is_count,omitempty"` + Limit int64 `protobuf:"varint,3,opt,name=limit,proto3" json:"limit,omitempty"` +} + +func (x *QueryPlanNode) Reset() { + *x = QueryPlanNode{} + if protoimpl.UnsafeEnabled { + mi := &file_plan_proto_msgTypes[23] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *QueryPlanNode) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryPlanNode) ProtoMessage() {} + +func (x *QueryPlanNode) ProtoReflect() protoreflect.Message { + mi := &file_plan_proto_msgTypes[23] + 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 QueryPlanNode.ProtoReflect.Descriptor instead. +func (*QueryPlanNode) Descriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{23} +} + +func (x *QueryPlanNode) GetPredicates() *Expr { + if x != nil { + return x.Predicates + } + return nil +} + +func (x *QueryPlanNode) GetIsCount() bool { + if x != nil { + return x.IsCount + } + return false +} + +func (x *QueryPlanNode) GetLimit() int64 { + if x != nil { + return x.Limit + } + return 0 +} + +type PlanNode struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Node: + // + // *PlanNode_VectorAnns + // *PlanNode_Predicates + // *PlanNode_Query + Node isPlanNode_Node `protobuf_oneof:"node"` + OutputFieldIds []int64 `protobuf:"varint,3,rep,packed,name=output_field_ids,json=outputFieldIds,proto3" json:"output_field_ids,omitempty"` + DynamicFields []string `protobuf:"bytes,5,rep,name=dynamic_fields,json=dynamicFields,proto3" json:"dynamic_fields,omitempty"` +} + +func (x *PlanNode) Reset() { + *x = PlanNode{} + if protoimpl.UnsafeEnabled { + mi := &file_plan_proto_msgTypes[24] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PlanNode) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PlanNode) ProtoMessage() {} + +func (x *PlanNode) ProtoReflect() protoreflect.Message { + mi := &file_plan_proto_msgTypes[24] + 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 PlanNode.ProtoReflect.Descriptor instead. +func (*PlanNode) Descriptor() ([]byte, []int) { + return file_plan_proto_rawDescGZIP(), []int{24} +} + +func (m *PlanNode) GetNode() isPlanNode_Node { + if m != nil { + return m.Node + } + return nil +} + +func (x *PlanNode) GetVectorAnns() *VectorANNS { + if x, ok := x.GetNode().(*PlanNode_VectorAnns); ok { + return x.VectorAnns + } + return nil +} + +func (x *PlanNode) GetPredicates() *Expr { + if x, ok := x.GetNode().(*PlanNode_Predicates); ok { + return x.Predicates + } + return nil +} + +func (x *PlanNode) GetQuery() *QueryPlanNode { + if x, ok := x.GetNode().(*PlanNode_Query); ok { + return x.Query + } + return nil +} + +func (x *PlanNode) GetOutputFieldIds() []int64 { + if x != nil { + return x.OutputFieldIds + } + return nil +} + +func (x *PlanNode) GetDynamicFields() []string { + if x != nil { + return x.DynamicFields + } + return nil +} + +type isPlanNode_Node interface { + isPlanNode_Node() +} + +type PlanNode_VectorAnns struct { + VectorAnns *VectorANNS `protobuf:"bytes,1,opt,name=vector_anns,json=vectorAnns,proto3,oneof"` +} + +type PlanNode_Predicates struct { + Predicates *Expr `protobuf:"bytes,2,opt,name=predicates,proto3,oneof"` // deprecated, use query instead. +} + +type PlanNode_Query struct { + Query *QueryPlanNode `protobuf:"bytes,4,opt,name=query,proto3,oneof"` +} + +func (*PlanNode_VectorAnns) isPlanNode_Node() {} + +func (*PlanNode_Predicates) isPlanNode_Node() {} + +func (*PlanNode_Query) isPlanNode_Node() {} + +var File_plan_proto protoreflect.FileDescriptor + +var file_plan_proto_rawDesc = []byte{ + 0x0a, 0x0a, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x11, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x1a, + 0x0c, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xca, 0x01, + 0x0a, 0x0c, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x1b, + 0x0a, 0x08, 0x62, 0x6f, 0x6f, 0x6c, 0x5f, 0x76, 0x61, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, + 0x48, 0x00, 0x52, 0x07, 0x62, 0x6f, 0x6f, 0x6c, 0x56, 0x61, 0x6c, 0x12, 0x1d, 0x0a, 0x09, 0x69, + 0x6e, 0x74, 0x36, 0x34, 0x5f, 0x76, 0x61, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x48, 0x00, + 0x52, 0x08, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x56, 0x61, 0x6c, 0x12, 0x1d, 0x0a, 0x09, 0x66, 0x6c, + 0x6f, 0x61, 0x74, 0x5f, 0x76, 0x61, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x01, 0x48, 0x00, 0x52, + 0x08, 0x66, 0x6c, 0x6f, 0x61, 0x74, 0x56, 0x61, 0x6c, 0x12, 0x1f, 0x0a, 0x0a, 0x73, 0x74, 0x72, + 0x69, 0x6e, 0x67, 0x5f, 0x76, 0x61, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, + 0x09, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x12, 0x37, 0x0a, 0x09, 0x61, 0x72, + 0x72, 0x61, 0x79, 0x5f, 0x76, 0x61, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, + 0x6e, 0x2e, 0x41, 0x72, 0x72, 0x61, 0x79, 0x48, 0x00, 0x52, 0x08, 0x61, 0x72, 0x72, 0x61, 0x79, + 0x56, 0x61, 0x6c, 0x42, 0x05, 0x0a, 0x03, 0x76, 0x61, 0x6c, 0x22, 0x9d, 0x01, 0x0a, 0x05, 0x41, + 0x72, 0x72, 0x61, 0x79, 0x12, 0x35, 0x0a, 0x05, 0x61, 0x72, 0x72, 0x61, 0x79, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63, 0x56, + 0x61, 0x6c, 0x75, 0x65, 0x52, 0x05, 0x61, 0x72, 0x72, 0x61, 0x79, 0x12, 0x1b, 0x0a, 0x09, 0x73, + 0x61, 0x6d, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, + 0x73, 0x61, 0x6d, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x40, 0x0a, 0x0c, 0x65, 0x6c, 0x65, 0x6d, + 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1d, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0b, 0x65, + 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x22, 0x7e, 0x0a, 0x14, 0x53, 0x65, + 0x61, 0x72, 0x63, 0x68, 0x49, 0x74, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x56, 0x32, 0x49, 0x6e, + 0x66, 0x6f, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x05, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x61, 0x74, 0x63, + 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x09, 0x62, 0x61, + 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x22, 0x0a, 0x0a, 0x6c, 0x61, 0x73, 0x74, 0x5f, + 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x02, 0x48, 0x00, 0x52, 0x09, 0x6c, + 0x61, 0x73, 0x74, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x88, 0x01, 0x01, 0x42, 0x0d, 0x0a, 0x0b, 0x5f, + 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x22, 0x9a, 0x04, 0x0a, 0x09, 0x51, + 0x75, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x6f, 0x70, 0x6b, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, 0x6f, 0x70, 0x6b, 0x12, 0x1f, 0x0a, 0x0b, + 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x54, 0x79, 0x70, 0x65, 0x12, 0x23, 0x0a, + 0x0d, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x50, 0x61, 0x72, 0x61, + 0x6d, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x6f, 0x75, 0x6e, 0x64, 0x5f, 0x64, 0x65, 0x63, 0x69, + 0x6d, 0x61, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x72, 0x6f, 0x75, 0x6e, 0x64, + 0x44, 0x65, 0x63, 0x69, 0x6d, 0x61, 0x6c, 0x12, 0x29, 0x0a, 0x11, 0x67, 0x72, 0x6f, 0x75, 0x70, + 0x5f, 0x62, 0x79, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x0e, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x42, 0x79, 0x46, 0x69, 0x65, 0x6c, 0x64, + 0x49, 0x64, 0x12, 0x3c, 0x0a, 0x1a, 0x6d, 0x61, 0x74, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, + 0x65, 0x64, 0x5f, 0x76, 0x69, 0x65, 0x77, 0x5f, 0x69, 0x6e, 0x76, 0x6f, 0x6c, 0x76, 0x65, 0x64, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x18, 0x6d, 0x61, 0x74, 0x65, 0x72, 0x69, 0x61, 0x6c, + 0x69, 0x7a, 0x65, 0x64, 0x56, 0x69, 0x65, 0x77, 0x49, 0x6e, 0x76, 0x6f, 0x6c, 0x76, 0x65, 0x64, + 0x12, 0x1d, 0x0a, 0x0a, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x08, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x53, 0x69, 0x7a, 0x65, 0x12, + 0x2a, 0x0a, 0x11, 0x73, 0x74, 0x72, 0x69, 0x63, 0x74, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, + 0x73, 0x69, 0x7a, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x73, 0x74, 0x72, 0x69, + 0x63, 0x74, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x62, + 0x6d, 0x32, 0x35, 0x5f, 0x61, 0x76, 0x67, 0x64, 0x6c, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x01, 0x52, + 0x09, 0x62, 0x6d, 0x32, 0x35, 0x41, 0x76, 0x67, 0x64, 0x6c, 0x12, 0x24, 0x0a, 0x0e, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x0b, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x0c, 0x71, 0x75, 0x65, 0x72, 0x79, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x64, + 0x12, 0x14, 0x0a, 0x05, 0x68, 0x69, 0x6e, 0x74, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x05, 0x68, 0x69, 0x6e, 0x74, 0x73, 0x12, 0x63, 0x0a, 0x17, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, + 0x5f, 0x69, 0x74, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x5f, 0x76, 0x32, 0x5f, 0x69, 0x6e, 0x66, + 0x6f, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x53, 0x65, 0x61, 0x72, + 0x63, 0x68, 0x49, 0x74, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x56, 0x32, 0x49, 0x6e, 0x66, 0x6f, + 0x48, 0x00, 0x52, 0x14, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x49, 0x74, 0x65, 0x72, 0x61, 0x74, + 0x6f, 0x72, 0x56, 0x32, 0x49, 0x6e, 0x66, 0x6f, 0x88, 0x01, 0x01, 0x42, 0x1a, 0x0a, 0x18, 0x5f, + 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x5f, 0x69, 0x74, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x5f, + 0x76, 0x32, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x22, 0xfb, 0x02, 0x0a, 0x0a, 0x43, 0x6f, 0x6c, 0x75, + 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x19, 0x0a, 0x08, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, + 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, + 0x64, 0x12, 0x3a, 0x0a, 0x09, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, + 0x79, 0x70, 0x65, 0x52, 0x08, 0x64, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x12, 0x24, 0x0a, + 0x0e, 0x69, 0x73, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, 0x73, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, + 0x4b, 0x65, 0x79, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x61, 0x75, 0x74, 0x6f, 0x49, 0x44, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x41, 0x75, 0x74, 0x6f, 0x49, 0x44, + 0x12, 0x1f, 0x0a, 0x0b, 0x6e, 0x65, 0x73, 0x74, 0x65, 0x64, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, + 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x6e, 0x65, 0x73, 0x74, 0x65, 0x64, 0x50, 0x61, 0x74, + 0x68, 0x12, 0x28, 0x0a, 0x10, 0x69, 0x73, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x69, 0x73, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x12, 0x40, 0x0a, 0x0c, 0x65, + 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, + 0x0e, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, + 0x52, 0x0b, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x2a, 0x0a, + 0x11, 0x69, 0x73, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x6b, + 0x65, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x69, 0x73, 0x43, 0x6c, 0x75, 0x73, + 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4b, 0x65, 0x79, 0x12, 0x1a, 0x0a, 0x08, 0x6e, 0x75, 0x6c, + 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x6e, 0x75, 0x6c, + 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x22, 0x3f, 0x0a, 0x0a, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x45, + 0x78, 0x70, 0x72, 0x12, 0x31, 0x0a, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, + 0x52, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x22, 0x3f, 0x0a, 0x0a, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, + 0x45, 0x78, 0x70, 0x72, 0x12, 0x31, 0x0a, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, + 0x6f, 0x52, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x22, 0x78, 0x0a, 0x09, 0x56, 0x61, 0x6c, 0x75, 0x65, + 0x45, 0x78, 0x70, 0x72, 0x12, 0x35, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63, 0x56, + 0x61, 0x6c, 0x75, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x34, 0x0a, 0x16, 0x74, + 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x14, 0x74, 0x65, 0x6d, + 0x70, 0x6c, 0x61, 0x74, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, + 0x65, 0x22, 0xe8, 0x01, 0x0a, 0x0e, 0x55, 0x6e, 0x61, 0x72, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, + 0x45, 0x78, 0x70, 0x72, 0x12, 0x3e, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x69, + 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, + 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x29, 0x0a, 0x02, 0x6f, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x19, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x4f, 0x70, 0x54, 0x79, 0x70, 0x65, 0x52, 0x02, 0x6f, 0x70, 0x12, + 0x35, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, + 0x61, 0x6e, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x34, 0x0a, 0x16, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, + 0x74, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x14, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, + 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xa9, 0x03, 0x0a, + 0x0f, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, + 0x12, 0x3e, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, + 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, + 0x12, 0x27, 0x0a, 0x0f, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x73, + 0x69, 0x76, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x6c, 0x6f, 0x77, 0x65, 0x72, + 0x49, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x75, 0x70, 0x70, + 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0e, 0x75, 0x70, 0x70, 0x65, 0x72, 0x49, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, + 0x76, 0x65, 0x12, 0x40, 0x0a, 0x0b, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x5f, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x47, 0x65, 0x6e, 0x65, + 0x72, 0x69, 0x63, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0a, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x56, + 0x61, 0x6c, 0x75, 0x65, 0x12, 0x40, 0x0a, 0x0b, 0x75, 0x70, 0x70, 0x65, 0x72, 0x5f, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x47, 0x65, + 0x6e, 0x65, 0x72, 0x69, 0x63, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0a, 0x75, 0x70, 0x70, 0x65, + 0x72, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x3f, 0x0a, 0x1c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x5f, + 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, + 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x19, 0x6c, 0x6f, + 0x77, 0x65, 0x72, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, + 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3f, 0x0a, 0x1c, 0x75, 0x70, 0x70, 0x65, 0x72, + 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x62, + 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x19, 0x75, + 0x70, 0x70, 0x65, 0x72, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x56, 0x61, 0x72, 0x69, + 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x79, 0x0a, 0x08, 0x43, 0x61, 0x6c, 0x6c, + 0x45, 0x78, 0x70, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x66, 0x75, 0x6e, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x48, 0x0a, 0x13, 0x66, 0x75, 0x6e, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, + 0x12, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, + 0x65, 0x72, 0x73, 0x22, 0xcc, 0x01, 0x0a, 0x0b, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x45, + 0x78, 0x70, 0x72, 0x12, 0x47, 0x0a, 0x10, 0x6c, 0x65, 0x66, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x75, + 0x6d, 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, + 0x6e, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0e, 0x6c, 0x65, + 0x66, 0x74, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x49, 0x0a, 0x11, + 0x72, 0x69, 0x67, 0x68, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x69, 0x6e, 0x66, + 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, 0x6c, 0x75, + 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0f, 0x72, 0x69, 0x67, 0x68, 0x74, 0x43, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x29, 0x0a, 0x02, 0x6f, 0x70, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x4f, 0x70, 0x54, 0x79, 0x70, 0x65, 0x52, 0x02, + 0x6f, 0x70, 0x22, 0xd9, 0x01, 0x0a, 0x08, 0x54, 0x65, 0x72, 0x6d, 0x45, 0x78, 0x70, 0x72, 0x12, + 0x3e, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, + 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, + 0x37, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, + 0x6c, 0x61, 0x6e, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63, 0x56, 0x61, 0x6c, 0x75, 0x65, + 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x12, 0x1e, 0x0a, 0x0b, 0x69, 0x73, 0x5f, 0x69, + 0x6e, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x69, + 0x73, 0x49, 0x6e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x12, 0x34, 0x0a, 0x16, 0x74, 0x65, 0x6d, 0x70, + 0x6c, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x14, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, + 0x74, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xf6, + 0x02, 0x0a, 0x10, 0x4a, 0x53, 0x4f, 0x4e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x45, + 0x78, 0x70, 0x72, 0x12, 0x3e, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x69, 0x6e, + 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, + 0x6e, 0x66, 0x6f, 0x12, 0x3b, 0x0a, 0x08, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, + 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x69, + 0x63, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x08, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, + 0x12, 0x3a, 0x0a, 0x02, 0x6f, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2a, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, + 0x2e, 0x4a, 0x53, 0x4f, 0x4e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x45, 0x78, 0x70, + 0x72, 0x2e, 0x4a, 0x53, 0x4f, 0x4e, 0x4f, 0x70, 0x52, 0x02, 0x6f, 0x70, 0x12, 0x2c, 0x0a, 0x12, + 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x73, 0x61, 0x6d, 0x65, 0x5f, 0x74, 0x79, + 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, + 0x74, 0x73, 0x53, 0x61, 0x6d, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x34, 0x0a, 0x16, 0x74, 0x65, + 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x5f, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x14, 0x74, 0x65, 0x6d, 0x70, + 0x6c, 0x61, 0x74, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, + 0x22, 0x45, 0x0a, 0x06, 0x4a, 0x53, 0x4f, 0x4e, 0x4f, 0x70, 0x12, 0x0b, 0x0a, 0x07, 0x49, 0x6e, + 0x76, 0x61, 0x6c, 0x69, 0x64, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x43, 0x6f, 0x6e, 0x74, 0x61, + 0x69, 0x6e, 0x73, 0x10, 0x01, 0x12, 0x0f, 0x0a, 0x0b, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, + 0x73, 0x41, 0x6c, 0x6c, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, + 0x6e, 0x73, 0x41, 0x6e, 0x79, 0x10, 0x03, 0x22, 0xb0, 0x01, 0x0a, 0x08, 0x4e, 0x75, 0x6c, 0x6c, + 0x45, 0x78, 0x70, 0x72, 0x12, 0x3e, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x69, + 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, + 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x32, 0x0a, 0x02, 0x6f, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x4e, 0x75, 0x6c, 0x6c, 0x45, 0x78, 0x70, 0x72, 0x2e, 0x4e, 0x75, + 0x6c, 0x6c, 0x4f, 0x70, 0x52, 0x02, 0x6f, 0x70, 0x22, 0x30, 0x0a, 0x06, 0x4e, 0x75, 0x6c, 0x6c, + 0x4f, 0x70, 0x12, 0x0b, 0x0a, 0x07, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x10, 0x00, 0x12, + 0x0a, 0x0a, 0x06, 0x49, 0x73, 0x4e, 0x75, 0x6c, 0x6c, 0x10, 0x01, 0x12, 0x0d, 0x0a, 0x09, 0x49, + 0x73, 0x4e, 0x6f, 0x74, 0x4e, 0x75, 0x6c, 0x6c, 0x10, 0x02, 0x22, 0x91, 0x01, 0x0a, 0x09, 0x55, + 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x12, 0x34, 0x0a, 0x02, 0x6f, 0x70, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x55, 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, + 0x70, 0x72, 0x2e, 0x55, 0x6e, 0x61, 0x72, 0x79, 0x4f, 0x70, 0x52, 0x02, 0x6f, 0x70, 0x12, 0x2d, + 0x0a, 0x05, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, + 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x05, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x22, 0x1f, 0x0a, + 0x07, 0x55, 0x6e, 0x61, 0x72, 0x79, 0x4f, 0x70, 0x12, 0x0b, 0x0a, 0x07, 0x49, 0x6e, 0x76, 0x61, + 0x6c, 0x69, 0x64, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x4e, 0x6f, 0x74, 0x10, 0x01, 0x22, 0xd8, + 0x01, 0x0a, 0x0a, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x12, 0x36, 0x0a, + 0x02, 0x6f, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x42, 0x69, + 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x2e, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x4f, + 0x70, 0x52, 0x02, 0x6f, 0x70, 0x12, 0x2b, 0x0a, 0x04, 0x6c, 0x65, 0x66, 0x74, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x04, 0x6c, 0x65, + 0x66, 0x74, 0x12, 0x2d, 0x0a, 0x05, 0x72, 0x69, 0x67, 0x68, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x05, 0x72, 0x69, 0x67, 0x68, + 0x74, 0x22, 0x36, 0x0a, 0x08, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x4f, 0x70, 0x12, 0x0b, 0x0a, + 0x07, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x10, 0x00, 0x12, 0x0e, 0x0a, 0x0a, 0x4c, 0x6f, + 0x67, 0x69, 0x63, 0x61, 0x6c, 0x41, 0x6e, 0x64, 0x10, 0x01, 0x12, 0x0d, 0x0a, 0x09, 0x4c, 0x6f, + 0x67, 0x69, 0x63, 0x61, 0x6c, 0x4f, 0x72, 0x10, 0x02, 0x22, 0xd0, 0x01, 0x0a, 0x0d, 0x42, 0x69, + 0x6e, 0x61, 0x72, 0x79, 0x41, 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x12, 0x3e, 0x0a, 0x0b, 0x63, + 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, + 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x39, 0x0a, 0x08, 0x61, + 0x72, 0x69, 0x74, 0x68, 0x5f, 0x6f, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1e, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, + 0x6e, 0x2e, 0x41, 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x54, 0x79, 0x70, 0x65, 0x52, 0x07, 0x61, + 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x12, 0x44, 0x0a, 0x0d, 0x72, 0x69, 0x67, 0x68, 0x74, 0x5f, + 0x6f, 0x70, 0x65, 0x72, 0x61, 0x6e, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, + 0x6e, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0c, + 0x72, 0x69, 0x67, 0x68, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x6e, 0x64, 0x22, 0x9d, 0x01, 0x0a, + 0x0f, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x41, 0x72, 0x69, 0x74, 0x68, 0x45, 0x78, 0x70, 0x72, + 0x12, 0x2b, 0x0a, 0x04, 0x6c, 0x65, 0x66, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, + 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x04, 0x6c, 0x65, 0x66, 0x74, 0x12, 0x2d, 0x0a, + 0x05, 0x72, 0x69, 0x67, 0x68, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, + 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x05, 0x72, 0x69, 0x67, 0x68, 0x74, 0x12, 0x2e, 0x0a, 0x02, + 0x6f, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x41, 0x72, 0x69, + 0x74, 0x68, 0x4f, 0x70, 0x54, 0x79, 0x70, 0x65, 0x52, 0x02, 0x6f, 0x70, 0x22, 0xc5, 0x03, 0x0a, + 0x1a, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x41, 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x45, 0x76, + 0x61, 0x6c, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x3e, 0x0a, 0x0b, 0x63, + 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, + 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x39, 0x0a, 0x08, 0x61, + 0x72, 0x69, 0x74, 0x68, 0x5f, 0x6f, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1e, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, + 0x6e, 0x2e, 0x41, 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x54, 0x79, 0x70, 0x65, 0x52, 0x07, 0x61, + 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x12, 0x44, 0x0a, 0x0d, 0x72, 0x69, 0x67, 0x68, 0x74, 0x5f, + 0x6f, 0x70, 0x65, 0x72, 0x61, 0x6e, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, + 0x6e, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0c, + 0x72, 0x69, 0x67, 0x68, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x6e, 0x64, 0x12, 0x29, 0x0a, 0x02, + 0x6f, 0x70, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x4f, 0x70, 0x54, + 0x79, 0x70, 0x65, 0x52, 0x02, 0x6f, 0x70, 0x12, 0x35, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, + 0x69, 0x63, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x43, + 0x0a, 0x1e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x6e, 0x64, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, + 0x74, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1b, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x6e, 0x64, 0x54, + 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x4e, + 0x61, 0x6d, 0x65, 0x12, 0x3f, 0x0a, 0x1c, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x5f, 0x74, 0x65, 0x6d, + 0x70, 0x6c, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x19, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, + 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x10, 0x0a, 0x0e, 0x41, 0x6c, 0x77, 0x61, 0x79, 0x73, 0x54, 0x72, + 0x75, 0x65, 0x45, 0x78, 0x70, 0x72, 0x22, 0xf9, 0x08, 0x0a, 0x04, 0x45, 0x78, 0x70, 0x72, 0x12, + 0x3a, 0x0a, 0x09, 0x74, 0x65, 0x72, 0x6d, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x54, 0x65, 0x72, 0x6d, 0x45, 0x78, 0x70, 0x72, 0x48, + 0x00, 0x52, 0x08, 0x74, 0x65, 0x72, 0x6d, 0x45, 0x78, 0x70, 0x72, 0x12, 0x3d, 0x0a, 0x0a, 0x75, + 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, + 0x6c, 0x61, 0x6e, 0x2e, 0x55, 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, + 0x09, 0x75, 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x12, 0x40, 0x0a, 0x0b, 0x62, 0x69, + 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, + 0x6c, 0x61, 0x6e, 0x2e, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, + 0x52, 0x0a, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x12, 0x43, 0x0a, 0x0c, + 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x45, 0x78, + 0x70, 0x72, 0x48, 0x00, 0x52, 0x0b, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x45, 0x78, 0x70, + 0x72, 0x12, 0x4d, 0x0a, 0x10, 0x75, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, + 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, + 0x55, 0x6e, 0x61, 0x72, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, + 0x52, 0x0e, 0x75, 0x6e, 0x61, 0x72, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, + 0x12, 0x50, 0x0a, 0x11, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, + 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, + 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, 0x48, + 0x00, 0x52, 0x0f, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, + 0x70, 0x72, 0x12, 0x74, 0x0a, 0x1f, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x61, 0x72, 0x69, + 0x74, 0x68, 0x5f, 0x6f, 0x70, 0x5f, 0x65, 0x76, 0x61, 0x6c, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, + 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, + 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x41, 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x45, 0x76, 0x61, + 0x6c, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x1a, 0x62, 0x69, + 0x6e, 0x61, 0x72, 0x79, 0x41, 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x45, 0x76, 0x61, 0x6c, 0x52, + 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x50, 0x0a, 0x11, 0x62, 0x69, 0x6e, 0x61, + 0x72, 0x79, 0x5f, 0x61, 0x72, 0x69, 0x74, 0x68, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x08, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x41, 0x72, + 0x69, 0x74, 0x68, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x0f, 0x62, 0x69, 0x6e, 0x61, 0x72, + 0x79, 0x41, 0x72, 0x69, 0x74, 0x68, 0x45, 0x78, 0x70, 0x72, 0x12, 0x3d, 0x0a, 0x0a, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, + 0x61, 0x6e, 0x2e, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x09, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x40, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, + 0x61, 0x6e, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, + 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x45, 0x78, 0x70, 0x72, 0x12, 0x40, 0x0a, 0x0b, 0x65, + 0x78, 0x69, 0x73, 0x74, 0x73, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x45, 0x78, 0x70, 0x72, 0x48, + 0x00, 0x52, 0x0a, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x45, 0x78, 0x70, 0x72, 0x12, 0x4d, 0x0a, + 0x10, 0x61, 0x6c, 0x77, 0x61, 0x79, 0x73, 0x5f, 0x74, 0x72, 0x75, 0x65, 0x5f, 0x65, 0x78, 0x70, + 0x72, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x41, 0x6c, 0x77, 0x61, + 0x79, 0x73, 0x54, 0x72, 0x75, 0x65, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x0e, 0x61, 0x6c, + 0x77, 0x61, 0x79, 0x73, 0x54, 0x72, 0x75, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x53, 0x0a, 0x12, + 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x5f, 0x65, 0x78, + 0x70, 0x72, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x4a, 0x53, 0x4f, + 0x4e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, + 0x10, 0x6a, 0x73, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x45, 0x78, 0x70, + 0x72, 0x12, 0x3a, 0x0a, 0x09, 0x63, 0x61, 0x6c, 0x6c, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x0e, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x45, 0x78, 0x70, + 0x72, 0x48, 0x00, 0x52, 0x08, 0x63, 0x61, 0x6c, 0x6c, 0x45, 0x78, 0x70, 0x72, 0x12, 0x3a, 0x0a, + 0x09, 0x6e, 0x75, 0x6c, 0x6c, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x4e, 0x75, 0x6c, 0x6c, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, + 0x08, 0x6e, 0x75, 0x6c, 0x6c, 0x45, 0x78, 0x70, 0x72, 0x12, 0x1f, 0x0a, 0x0b, 0x69, 0x73, 0x5f, + 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x18, 0x14, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, + 0x69, 0x73, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x42, 0x06, 0x0a, 0x04, 0x65, 0x78, + 0x70, 0x72, 0x22, 0x86, 0x02, 0x0a, 0x0a, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x41, 0x4e, 0x4e, + 0x53, 0x12, 0x3e, 0x0a, 0x0b, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x5f, 0x74, 0x79, 0x70, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x56, 0x65, 0x63, 0x74, 0x6f, + 0x72, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0a, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x54, 0x79, 0x70, + 0x65, 0x12, 0x19, 0x0a, 0x08, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x64, 0x12, 0x37, 0x0a, 0x0a, + 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, + 0x63, 0x61, 0x74, 0x65, 0x73, 0x12, 0x3b, 0x0a, 0x0a, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x69, + 0x6e, 0x66, 0x6f, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x51, 0x75, + 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x71, 0x75, 0x65, 0x72, 0x79, 0x49, 0x6e, + 0x66, 0x6f, 0x12, 0x27, 0x0a, 0x0f, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x68, 0x6f, 0x6c, 0x64, 0x65, + 0x72, 0x5f, 0x74, 0x61, 0x67, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x70, 0x6c, 0x61, + 0x63, 0x65, 0x68, 0x6f, 0x6c, 0x64, 0x65, 0x72, 0x54, 0x61, 0x67, 0x22, 0x79, 0x0a, 0x0d, 0x51, + 0x75, 0x65, 0x72, 0x79, 0x50, 0x6c, 0x61, 0x6e, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x37, 0x0a, 0x0a, + 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, + 0x63, 0x61, 0x74, 0x65, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x69, 0x73, 0x5f, 0x63, 0x6f, 0x75, 0x6e, + 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x69, 0x73, 0x43, 0x6f, 0x75, 0x6e, 0x74, + 0x12, 0x14, 0x0a, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x22, 0x9a, 0x02, 0x0a, 0x08, 0x50, 0x6c, 0x61, 0x6e, 0x4e, + 0x6f, 0x64, 0x65, 0x12, 0x40, 0x0a, 0x0b, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x5f, 0x61, 0x6e, + 0x6e, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x56, 0x65, 0x63, + 0x74, 0x6f, 0x72, 0x41, 0x4e, 0x4e, 0x53, 0x48, 0x00, 0x52, 0x0a, 0x76, 0x65, 0x63, 0x74, 0x6f, + 0x72, 0x41, 0x6e, 0x6e, 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, + 0x74, 0x65, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, + 0x70, 0x72, 0x48, 0x00, 0x52, 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, + 0x12, 0x38, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, + 0x6c, 0x61, 0x6e, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x50, 0x6c, 0x61, 0x6e, 0x4e, 0x6f, 0x64, + 0x65, 0x48, 0x00, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x28, 0x0a, 0x10, 0x6f, 0x75, + 0x74, 0x70, 0x75, 0x74, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x03, + 0x20, 0x03, 0x28, 0x03, 0x52, 0x0e, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x46, 0x69, 0x65, 0x6c, + 0x64, 0x49, 0x64, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x64, 0x79, 0x6e, 0x61, 0x6d, 0x69, 0x63, 0x5f, + 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x64, 0x79, + 0x6e, 0x61, 0x6d, 0x69, 0x63, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x42, 0x06, 0x0a, 0x04, 0x6e, + 0x6f, 0x64, 0x65, 0x2a, 0xc9, 0x01, 0x0a, 0x06, 0x4f, 0x70, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, + 0x0a, 0x07, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x10, 0x00, 0x12, 0x0f, 0x0a, 0x0b, 0x47, + 0x72, 0x65, 0x61, 0x74, 0x65, 0x72, 0x54, 0x68, 0x61, 0x6e, 0x10, 0x01, 0x12, 0x10, 0x0a, 0x0c, + 0x47, 0x72, 0x65, 0x61, 0x74, 0x65, 0x72, 0x45, 0x71, 0x75, 0x61, 0x6c, 0x10, 0x02, 0x12, 0x0c, + 0x0a, 0x08, 0x4c, 0x65, 0x73, 0x73, 0x54, 0x68, 0x61, 0x6e, 0x10, 0x03, 0x12, 0x0d, 0x0a, 0x09, + 0x4c, 0x65, 0x73, 0x73, 0x45, 0x71, 0x75, 0x61, 0x6c, 0x10, 0x04, 0x12, 0x09, 0x0a, 0x05, 0x45, + 0x71, 0x75, 0x61, 0x6c, 0x10, 0x05, 0x12, 0x0c, 0x0a, 0x08, 0x4e, 0x6f, 0x74, 0x45, 0x71, 0x75, + 0x61, 0x6c, 0x10, 0x06, 0x12, 0x0f, 0x0a, 0x0b, 0x50, 0x72, 0x65, 0x66, 0x69, 0x78, 0x4d, 0x61, + 0x74, 0x63, 0x68, 0x10, 0x07, 0x12, 0x10, 0x0a, 0x0c, 0x50, 0x6f, 0x73, 0x74, 0x66, 0x69, 0x78, + 0x4d, 0x61, 0x74, 0x63, 0x68, 0x10, 0x08, 0x12, 0x09, 0x0a, 0x05, 0x4d, 0x61, 0x74, 0x63, 0x68, + 0x10, 0x09, 0x12, 0x09, 0x0a, 0x05, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x10, 0x0a, 0x12, 0x06, 0x0a, + 0x02, 0x49, 0x6e, 0x10, 0x0b, 0x12, 0x09, 0x0a, 0x05, 0x4e, 0x6f, 0x74, 0x49, 0x6e, 0x10, 0x0c, + 0x12, 0x0d, 0x0a, 0x09, 0x54, 0x65, 0x78, 0x74, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x10, 0x0d, 0x2a, + 0x58, 0x0a, 0x0b, 0x41, 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, + 0x0a, 0x07, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x41, + 0x64, 0x64, 0x10, 0x01, 0x12, 0x07, 0x0a, 0x03, 0x53, 0x75, 0x62, 0x10, 0x02, 0x12, 0x07, 0x0a, + 0x03, 0x4d, 0x75, 0x6c, 0x10, 0x03, 0x12, 0x07, 0x0a, 0x03, 0x44, 0x69, 0x76, 0x10, 0x04, 0x12, + 0x07, 0x0a, 0x03, 0x4d, 0x6f, 0x64, 0x10, 0x05, 0x12, 0x0f, 0x0a, 0x0b, 0x41, 0x72, 0x72, 0x61, + 0x79, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x10, 0x06, 0x2a, 0x6d, 0x0a, 0x0a, 0x56, 0x65, 0x63, + 0x74, 0x6f, 0x72, 0x54, 0x79, 0x70, 0x65, 0x12, 0x10, 0x0a, 0x0c, 0x42, 0x69, 0x6e, 0x61, 0x72, + 0x79, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, 0x00, 0x12, 0x0f, 0x0a, 0x0b, 0x46, 0x6c, 0x6f, + 0x61, 0x74, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, 0x01, 0x12, 0x11, 0x0a, 0x0d, 0x46, 0x6c, + 0x6f, 0x61, 0x74, 0x31, 0x36, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, 0x02, 0x12, 0x12, 0x0a, + 0x0e, 0x42, 0x46, 0x6c, 0x6f, 0x61, 0x74, 0x31, 0x36, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, + 0x03, 0x12, 0x15, 0x0a, 0x11, 0x53, 0x70, 0x61, 0x72, 0x73, 0x65, 0x46, 0x6c, 0x6f, 0x61, 0x74, + 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, 0x04, 0x42, 0x2e, 0x5a, 0x2c, 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, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2f, 0x70, 0x6c, 0x61, 0x6e, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_plan_proto_rawDescOnce sync.Once + file_plan_proto_rawDescData = file_plan_proto_rawDesc +) + +func file_plan_proto_rawDescGZIP() []byte { + file_plan_proto_rawDescOnce.Do(func() { + file_plan_proto_rawDescData = protoimpl.X.CompressGZIP(file_plan_proto_rawDescData) + }) + return file_plan_proto_rawDescData +} + +var file_plan_proto_enumTypes = make([]protoimpl.EnumInfo, 7) +var file_plan_proto_msgTypes = make([]protoimpl.MessageInfo, 25) +var file_plan_proto_goTypes = []interface{}{ + (OpType)(0), // 0: milvus.proto.plan.OpType + (ArithOpType)(0), // 1: milvus.proto.plan.ArithOpType + (VectorType)(0), // 2: milvus.proto.plan.VectorType + (JSONContainsExpr_JSONOp)(0), // 3: milvus.proto.plan.JSONContainsExpr.JSONOp + (NullExpr_NullOp)(0), // 4: milvus.proto.plan.NullExpr.NullOp + (UnaryExpr_UnaryOp)(0), // 5: milvus.proto.plan.UnaryExpr.UnaryOp + (BinaryExpr_BinaryOp)(0), // 6: milvus.proto.plan.BinaryExpr.BinaryOp + (*GenericValue)(nil), // 7: milvus.proto.plan.GenericValue + (*Array)(nil), // 8: milvus.proto.plan.Array + (*SearchIteratorV2Info)(nil), // 9: milvus.proto.plan.SearchIteratorV2Info + (*QueryInfo)(nil), // 10: milvus.proto.plan.QueryInfo + (*ColumnInfo)(nil), // 11: milvus.proto.plan.ColumnInfo + (*ColumnExpr)(nil), // 12: milvus.proto.plan.ColumnExpr + (*ExistsExpr)(nil), // 13: milvus.proto.plan.ExistsExpr + (*ValueExpr)(nil), // 14: milvus.proto.plan.ValueExpr + (*UnaryRangeExpr)(nil), // 15: milvus.proto.plan.UnaryRangeExpr + (*BinaryRangeExpr)(nil), // 16: milvus.proto.plan.BinaryRangeExpr + (*CallExpr)(nil), // 17: milvus.proto.plan.CallExpr + (*CompareExpr)(nil), // 18: milvus.proto.plan.CompareExpr + (*TermExpr)(nil), // 19: milvus.proto.plan.TermExpr + (*JSONContainsExpr)(nil), // 20: milvus.proto.plan.JSONContainsExpr + (*NullExpr)(nil), // 21: milvus.proto.plan.NullExpr + (*UnaryExpr)(nil), // 22: milvus.proto.plan.UnaryExpr + (*BinaryExpr)(nil), // 23: milvus.proto.plan.BinaryExpr + (*BinaryArithOp)(nil), // 24: milvus.proto.plan.BinaryArithOp + (*BinaryArithExpr)(nil), // 25: milvus.proto.plan.BinaryArithExpr + (*BinaryArithOpEvalRangeExpr)(nil), // 26: milvus.proto.plan.BinaryArithOpEvalRangeExpr + (*AlwaysTrueExpr)(nil), // 27: milvus.proto.plan.AlwaysTrueExpr + (*Expr)(nil), // 28: milvus.proto.plan.Expr + (*VectorANNS)(nil), // 29: milvus.proto.plan.VectorANNS + (*QueryPlanNode)(nil), // 30: milvus.proto.plan.QueryPlanNode + (*PlanNode)(nil), // 31: milvus.proto.plan.PlanNode + (schemapb.DataType)(0), // 32: milvus.proto.schema.DataType +} +var file_plan_proto_depIdxs = []int32{ + 8, // 0: milvus.proto.plan.GenericValue.array_val:type_name -> milvus.proto.plan.Array + 7, // 1: milvus.proto.plan.Array.array:type_name -> milvus.proto.plan.GenericValue + 32, // 2: milvus.proto.plan.Array.element_type:type_name -> milvus.proto.schema.DataType + 9, // 3: milvus.proto.plan.QueryInfo.search_iterator_v2_info:type_name -> milvus.proto.plan.SearchIteratorV2Info + 32, // 4: milvus.proto.plan.ColumnInfo.data_type:type_name -> milvus.proto.schema.DataType + 32, // 5: milvus.proto.plan.ColumnInfo.element_type:type_name -> milvus.proto.schema.DataType + 11, // 6: milvus.proto.plan.ColumnExpr.info:type_name -> milvus.proto.plan.ColumnInfo + 11, // 7: milvus.proto.plan.ExistsExpr.info:type_name -> milvus.proto.plan.ColumnInfo + 7, // 8: milvus.proto.plan.ValueExpr.value:type_name -> milvus.proto.plan.GenericValue + 11, // 9: milvus.proto.plan.UnaryRangeExpr.column_info:type_name -> milvus.proto.plan.ColumnInfo + 0, // 10: milvus.proto.plan.UnaryRangeExpr.op:type_name -> milvus.proto.plan.OpType + 7, // 11: milvus.proto.plan.UnaryRangeExpr.value:type_name -> milvus.proto.plan.GenericValue + 11, // 12: milvus.proto.plan.BinaryRangeExpr.column_info:type_name -> milvus.proto.plan.ColumnInfo + 7, // 13: milvus.proto.plan.BinaryRangeExpr.lower_value:type_name -> milvus.proto.plan.GenericValue + 7, // 14: milvus.proto.plan.BinaryRangeExpr.upper_value:type_name -> milvus.proto.plan.GenericValue + 28, // 15: milvus.proto.plan.CallExpr.function_parameters:type_name -> milvus.proto.plan.Expr + 11, // 16: milvus.proto.plan.CompareExpr.left_column_info:type_name -> milvus.proto.plan.ColumnInfo + 11, // 17: milvus.proto.plan.CompareExpr.right_column_info:type_name -> milvus.proto.plan.ColumnInfo + 0, // 18: milvus.proto.plan.CompareExpr.op:type_name -> milvus.proto.plan.OpType + 11, // 19: milvus.proto.plan.TermExpr.column_info:type_name -> milvus.proto.plan.ColumnInfo + 7, // 20: milvus.proto.plan.TermExpr.values:type_name -> milvus.proto.plan.GenericValue + 11, // 21: milvus.proto.plan.JSONContainsExpr.column_info:type_name -> milvus.proto.plan.ColumnInfo + 7, // 22: milvus.proto.plan.JSONContainsExpr.elements:type_name -> milvus.proto.plan.GenericValue + 3, // 23: milvus.proto.plan.JSONContainsExpr.op:type_name -> milvus.proto.plan.JSONContainsExpr.JSONOp + 11, // 24: milvus.proto.plan.NullExpr.column_info:type_name -> milvus.proto.plan.ColumnInfo + 4, // 25: milvus.proto.plan.NullExpr.op:type_name -> milvus.proto.plan.NullExpr.NullOp + 5, // 26: milvus.proto.plan.UnaryExpr.op:type_name -> milvus.proto.plan.UnaryExpr.UnaryOp + 28, // 27: milvus.proto.plan.UnaryExpr.child:type_name -> milvus.proto.plan.Expr + 6, // 28: milvus.proto.plan.BinaryExpr.op:type_name -> milvus.proto.plan.BinaryExpr.BinaryOp + 28, // 29: milvus.proto.plan.BinaryExpr.left:type_name -> milvus.proto.plan.Expr + 28, // 30: milvus.proto.plan.BinaryExpr.right:type_name -> milvus.proto.plan.Expr + 11, // 31: milvus.proto.plan.BinaryArithOp.column_info:type_name -> milvus.proto.plan.ColumnInfo + 1, // 32: milvus.proto.plan.BinaryArithOp.arith_op:type_name -> milvus.proto.plan.ArithOpType + 7, // 33: milvus.proto.plan.BinaryArithOp.right_operand:type_name -> milvus.proto.plan.GenericValue + 28, // 34: milvus.proto.plan.BinaryArithExpr.left:type_name -> milvus.proto.plan.Expr + 28, // 35: milvus.proto.plan.BinaryArithExpr.right:type_name -> milvus.proto.plan.Expr + 1, // 36: milvus.proto.plan.BinaryArithExpr.op:type_name -> milvus.proto.plan.ArithOpType + 11, // 37: milvus.proto.plan.BinaryArithOpEvalRangeExpr.column_info:type_name -> milvus.proto.plan.ColumnInfo + 1, // 38: milvus.proto.plan.BinaryArithOpEvalRangeExpr.arith_op:type_name -> milvus.proto.plan.ArithOpType + 7, // 39: milvus.proto.plan.BinaryArithOpEvalRangeExpr.right_operand:type_name -> milvus.proto.plan.GenericValue + 0, // 40: milvus.proto.plan.BinaryArithOpEvalRangeExpr.op:type_name -> milvus.proto.plan.OpType + 7, // 41: milvus.proto.plan.BinaryArithOpEvalRangeExpr.value:type_name -> milvus.proto.plan.GenericValue + 19, // 42: milvus.proto.plan.Expr.term_expr:type_name -> milvus.proto.plan.TermExpr + 22, // 43: milvus.proto.plan.Expr.unary_expr:type_name -> milvus.proto.plan.UnaryExpr + 23, // 44: milvus.proto.plan.Expr.binary_expr:type_name -> milvus.proto.plan.BinaryExpr + 18, // 45: milvus.proto.plan.Expr.compare_expr:type_name -> milvus.proto.plan.CompareExpr + 15, // 46: milvus.proto.plan.Expr.unary_range_expr:type_name -> milvus.proto.plan.UnaryRangeExpr + 16, // 47: milvus.proto.plan.Expr.binary_range_expr:type_name -> milvus.proto.plan.BinaryRangeExpr + 26, // 48: milvus.proto.plan.Expr.binary_arith_op_eval_range_expr:type_name -> milvus.proto.plan.BinaryArithOpEvalRangeExpr + 25, // 49: milvus.proto.plan.Expr.binary_arith_expr:type_name -> milvus.proto.plan.BinaryArithExpr + 14, // 50: milvus.proto.plan.Expr.value_expr:type_name -> milvus.proto.plan.ValueExpr + 12, // 51: milvus.proto.plan.Expr.column_expr:type_name -> milvus.proto.plan.ColumnExpr + 13, // 52: milvus.proto.plan.Expr.exists_expr:type_name -> milvus.proto.plan.ExistsExpr + 27, // 53: milvus.proto.plan.Expr.always_true_expr:type_name -> milvus.proto.plan.AlwaysTrueExpr + 20, // 54: milvus.proto.plan.Expr.json_contains_expr:type_name -> milvus.proto.plan.JSONContainsExpr + 17, // 55: milvus.proto.plan.Expr.call_expr:type_name -> milvus.proto.plan.CallExpr + 21, // 56: milvus.proto.plan.Expr.null_expr:type_name -> milvus.proto.plan.NullExpr + 2, // 57: milvus.proto.plan.VectorANNS.vector_type:type_name -> milvus.proto.plan.VectorType + 28, // 58: milvus.proto.plan.VectorANNS.predicates:type_name -> milvus.proto.plan.Expr + 10, // 59: milvus.proto.plan.VectorANNS.query_info:type_name -> milvus.proto.plan.QueryInfo + 28, // 60: milvus.proto.plan.QueryPlanNode.predicates:type_name -> milvus.proto.plan.Expr + 29, // 61: milvus.proto.plan.PlanNode.vector_anns:type_name -> milvus.proto.plan.VectorANNS + 28, // 62: milvus.proto.plan.PlanNode.predicates:type_name -> milvus.proto.plan.Expr + 30, // 63: milvus.proto.plan.PlanNode.query:type_name -> milvus.proto.plan.QueryPlanNode + 64, // [64:64] is the sub-list for method output_type + 64, // [64:64] is the sub-list for method input_type + 64, // [64:64] is the sub-list for extension type_name + 64, // [64:64] is the sub-list for extension extendee + 0, // [0:64] is the sub-list for field type_name +} + +func init() { file_plan_proto_init() } +func file_plan_proto_init() { + if File_plan_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_plan_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GenericValue); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_plan_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Array); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_plan_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SearchIteratorV2Info); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_plan_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QueryInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_plan_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ColumnInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_plan_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ColumnExpr); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_plan_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ExistsExpr); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_plan_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ValueExpr); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_plan_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*UnaryRangeExpr); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_plan_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BinaryRangeExpr); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_plan_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CallExpr); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_plan_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CompareExpr); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_plan_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*TermExpr); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_plan_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*JSONContainsExpr); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_plan_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*NullExpr); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_plan_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*UnaryExpr); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_plan_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BinaryExpr); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_plan_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BinaryArithOp); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_plan_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BinaryArithExpr); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_plan_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BinaryArithOpEvalRangeExpr); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_plan_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AlwaysTrueExpr); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_plan_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Expr); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_plan_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*VectorANNS); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_plan_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QueryPlanNode); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_plan_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PlanNode); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + file_plan_proto_msgTypes[0].OneofWrappers = []interface{}{ + (*GenericValue_BoolVal)(nil), + (*GenericValue_Int64Val)(nil), + (*GenericValue_FloatVal)(nil), + (*GenericValue_StringVal)(nil), + (*GenericValue_ArrayVal)(nil), + } + file_plan_proto_msgTypes[2].OneofWrappers = []interface{}{} + file_plan_proto_msgTypes[3].OneofWrappers = []interface{}{} + file_plan_proto_msgTypes[21].OneofWrappers = []interface{}{ + (*Expr_TermExpr)(nil), + (*Expr_UnaryExpr)(nil), + (*Expr_BinaryExpr)(nil), + (*Expr_CompareExpr)(nil), + (*Expr_UnaryRangeExpr)(nil), + (*Expr_BinaryRangeExpr)(nil), + (*Expr_BinaryArithOpEvalRangeExpr)(nil), + (*Expr_BinaryArithExpr)(nil), + (*Expr_ValueExpr)(nil), + (*Expr_ColumnExpr)(nil), + (*Expr_ExistsExpr)(nil), + (*Expr_AlwaysTrueExpr)(nil), + (*Expr_JsonContainsExpr)(nil), + (*Expr_CallExpr)(nil), + (*Expr_NullExpr)(nil), + } + file_plan_proto_msgTypes[24].OneofWrappers = []interface{}{ + (*PlanNode_VectorAnns)(nil), + (*PlanNode_Predicates)(nil), + (*PlanNode_Query)(nil), + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_plan_proto_rawDesc, + NumEnums: 7, + NumMessages: 25, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_plan_proto_goTypes, + DependencyIndexes: file_plan_proto_depIdxs, + EnumInfos: file_plan_proto_enumTypes, + MessageInfos: file_plan_proto_msgTypes, + }.Build() + File_plan_proto = out.File + file_plan_proto_rawDesc = nil + file_plan_proto_goTypes = nil + file_plan_proto_depIdxs = nil +} diff --git a/internal/proto/proxy.proto b/pkg/proto/proxy.proto similarity index 97% rename from internal/proto/proxy.proto rename to pkg/proto/proxy.proto index 7fd4f59da7f6b..d493ae62c5edf 100644 --- a/internal/proto/proxy.proto +++ b/pkg/proto/proxy.proto @@ -1,7 +1,7 @@ syntax = "proto3"; package milvus.proto.proxy; -option go_package = "github.com/milvus-io/milvus/internal/proto/proxypb"; +option go_package = "github.com/milvus-io/milvus/pkg/proto/proxypb"; import "common.proto"; import "internal.proto"; diff --git a/pkg/proto/proxypb/proxy.pb.go b/pkg/proto/proxypb/proxy.pb.go new file mode 100644 index 0000000000000..36a0e3dcae011 --- /dev/null +++ b/pkg/proto/proxypb/proxy.pb.go @@ -0,0 +1,1184 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.33.0 +// protoc v3.21.4 +// source: proxy.proto + +package proxypb + +import ( + commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type InvalidateCollMetaCacheRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // MsgType: + // + // DropCollection -> {meta cache, dml channels} + // Other -> {meta cache} + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + DbName string `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` + CollectionName string `protobuf:"bytes,3,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` + CollectionID int64 `protobuf:"varint,4,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionName string `protobuf:"bytes,5,opt,name=partition_name,json=partitionName,proto3" json:"partition_name,omitempty"` +} + +func (x *InvalidateCollMetaCacheRequest) Reset() { + *x = InvalidateCollMetaCacheRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_proxy_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *InvalidateCollMetaCacheRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*InvalidateCollMetaCacheRequest) ProtoMessage() {} + +func (x *InvalidateCollMetaCacheRequest) ProtoReflect() protoreflect.Message { + mi := &file_proxy_proto_msgTypes[0] + 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 InvalidateCollMetaCacheRequest.ProtoReflect.Descriptor instead. +func (*InvalidateCollMetaCacheRequest) Descriptor() ([]byte, []int) { + return file_proxy_proto_rawDescGZIP(), []int{0} +} + +func (x *InvalidateCollMetaCacheRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *InvalidateCollMetaCacheRequest) GetDbName() string { + if x != nil { + return x.DbName + } + return "" +} + +func (x *InvalidateCollMetaCacheRequest) GetCollectionName() string { + if x != nil { + return x.CollectionName + } + return "" +} + +func (x *InvalidateCollMetaCacheRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *InvalidateCollMetaCacheRequest) GetPartitionName() string { + if x != nil { + return x.PartitionName + } + return "" +} + +type InvalidateShardLeaderCacheRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + CollectionIDs []int64 `protobuf:"varint,2,rep,packed,name=collectionIDs,proto3" json:"collectionIDs,omitempty"` +} + +func (x *InvalidateShardLeaderCacheRequest) Reset() { + *x = InvalidateShardLeaderCacheRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_proxy_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *InvalidateShardLeaderCacheRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*InvalidateShardLeaderCacheRequest) ProtoMessage() {} + +func (x *InvalidateShardLeaderCacheRequest) ProtoReflect() protoreflect.Message { + mi := &file_proxy_proto_msgTypes[1] + 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 InvalidateShardLeaderCacheRequest.ProtoReflect.Descriptor instead. +func (*InvalidateShardLeaderCacheRequest) Descriptor() ([]byte, []int) { + return file_proxy_proto_rawDescGZIP(), []int{1} +} + +func (x *InvalidateShardLeaderCacheRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *InvalidateShardLeaderCacheRequest) GetCollectionIDs() []int64 { + if x != nil { + return x.CollectionIDs + } + return nil +} + +type InvalidateCredCacheRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + Username string `protobuf:"bytes,2,opt,name=username,proto3" json:"username,omitempty"` +} + +func (x *InvalidateCredCacheRequest) Reset() { + *x = InvalidateCredCacheRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_proxy_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *InvalidateCredCacheRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*InvalidateCredCacheRequest) ProtoMessage() {} + +func (x *InvalidateCredCacheRequest) ProtoReflect() protoreflect.Message { + mi := &file_proxy_proto_msgTypes[2] + 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 InvalidateCredCacheRequest.ProtoReflect.Descriptor instead. +func (*InvalidateCredCacheRequest) Descriptor() ([]byte, []int) { + return file_proxy_proto_rawDescGZIP(), []int{2} +} + +func (x *InvalidateCredCacheRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *InvalidateCredCacheRequest) GetUsername() string { + if x != nil { + return x.Username + } + return "" +} + +type UpdateCredCacheRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + Username string `protobuf:"bytes,2,opt,name=username,proto3" json:"username,omitempty"` + // password stored in cache + Password string `protobuf:"bytes,3,opt,name=password,proto3" json:"password,omitempty"` +} + +func (x *UpdateCredCacheRequest) Reset() { + *x = UpdateCredCacheRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_proxy_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateCredCacheRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateCredCacheRequest) ProtoMessage() {} + +func (x *UpdateCredCacheRequest) ProtoReflect() protoreflect.Message { + mi := &file_proxy_proto_msgTypes[3] + 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 UpdateCredCacheRequest.ProtoReflect.Descriptor instead. +func (*UpdateCredCacheRequest) Descriptor() ([]byte, []int) { + return file_proxy_proto_rawDescGZIP(), []int{3} +} + +func (x *UpdateCredCacheRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *UpdateCredCacheRequest) GetUsername() string { + if x != nil { + return x.Username + } + return "" +} + +func (x *UpdateCredCacheRequest) GetPassword() string { + if x != nil { + return x.Password + } + return "" +} + +type RefreshPolicyInfoCacheRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + OpType int32 `protobuf:"varint,2,opt,name=opType,proto3" json:"opType,omitempty"` + OpKey string `protobuf:"bytes,3,opt,name=opKey,proto3" json:"opKey,omitempty"` +} + +func (x *RefreshPolicyInfoCacheRequest) Reset() { + *x = RefreshPolicyInfoCacheRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_proxy_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RefreshPolicyInfoCacheRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RefreshPolicyInfoCacheRequest) ProtoMessage() {} + +func (x *RefreshPolicyInfoCacheRequest) ProtoReflect() protoreflect.Message { + mi := &file_proxy_proto_msgTypes[4] + 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 RefreshPolicyInfoCacheRequest.ProtoReflect.Descriptor instead. +func (*RefreshPolicyInfoCacheRequest) Descriptor() ([]byte, []int) { + return file_proxy_proto_rawDescGZIP(), []int{4} +} + +func (x *RefreshPolicyInfoCacheRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *RefreshPolicyInfoCacheRequest) GetOpType() int32 { + if x != nil { + return x.OpType + } + return 0 +} + +func (x *RefreshPolicyInfoCacheRequest) GetOpKey() string { + if x != nil { + return x.OpKey + } + return "" +} + +// Deprecated: use ClusterLimiter instead it +type CollectionRate struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Collection int64 `protobuf:"varint,1,opt,name=collection,proto3" json:"collection,omitempty"` + Rates []*internalpb.Rate `protobuf:"bytes,2,rep,name=rates,proto3" json:"rates,omitempty"` + States []milvuspb.QuotaState `protobuf:"varint,3,rep,packed,name=states,proto3,enum=milvus.proto.milvus.QuotaState" json:"states,omitempty"` + Codes []commonpb.ErrorCode `protobuf:"varint,4,rep,packed,name=codes,proto3,enum=milvus.proto.common.ErrorCode" json:"codes,omitempty"` +} + +func (x *CollectionRate) Reset() { + *x = CollectionRate{} + if protoimpl.UnsafeEnabled { + mi := &file_proxy_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CollectionRate) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CollectionRate) ProtoMessage() {} + +func (x *CollectionRate) ProtoReflect() protoreflect.Message { + mi := &file_proxy_proto_msgTypes[5] + 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 CollectionRate.ProtoReflect.Descriptor instead. +func (*CollectionRate) Descriptor() ([]byte, []int) { + return file_proxy_proto_rawDescGZIP(), []int{5} +} + +func (x *CollectionRate) GetCollection() int64 { + if x != nil { + return x.Collection + } + return 0 +} + +func (x *CollectionRate) GetRates() []*internalpb.Rate { + if x != nil { + return x.Rates + } + return nil +} + +func (x *CollectionRate) GetStates() []milvuspb.QuotaState { + if x != nil { + return x.States + } + return nil +} + +func (x *CollectionRate) GetCodes() []commonpb.ErrorCode { + if x != nil { + return x.Codes + } + return nil +} + +type LimiterNode struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // self limiter information + Limiter *Limiter `protobuf:"bytes,1,opt,name=limiter,proto3" json:"limiter,omitempty"` + // db id -> db limiter + // collection id -> collection limiter + // partition id -> partition limiter + Children map[int64]*LimiterNode `protobuf:"bytes,2,rep,name=children,proto3" json:"children,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *LimiterNode) Reset() { + *x = LimiterNode{} + if protoimpl.UnsafeEnabled { + mi := &file_proxy_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *LimiterNode) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LimiterNode) ProtoMessage() {} + +func (x *LimiterNode) ProtoReflect() protoreflect.Message { + mi := &file_proxy_proto_msgTypes[6] + 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 LimiterNode.ProtoReflect.Descriptor instead. +func (*LimiterNode) Descriptor() ([]byte, []int) { + return file_proxy_proto_rawDescGZIP(), []int{6} +} + +func (x *LimiterNode) GetLimiter() *Limiter { + if x != nil { + return x.Limiter + } + return nil +} + +func (x *LimiterNode) GetChildren() map[int64]*LimiterNode { + if x != nil { + return x.Children + } + return nil +} + +type Limiter struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Rates []*internalpb.Rate `protobuf:"bytes,1,rep,name=rates,proto3" json:"rates,omitempty"` + // we can use map to store quota states and error code, because key in map fields cannot be enum types + States []milvuspb.QuotaState `protobuf:"varint,2,rep,packed,name=states,proto3,enum=milvus.proto.milvus.QuotaState" json:"states,omitempty"` + Codes []commonpb.ErrorCode `protobuf:"varint,3,rep,packed,name=codes,proto3,enum=milvus.proto.common.ErrorCode" json:"codes,omitempty"` +} + +func (x *Limiter) Reset() { + *x = Limiter{} + if protoimpl.UnsafeEnabled { + mi := &file_proxy_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Limiter) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Limiter) ProtoMessage() {} + +func (x *Limiter) ProtoReflect() protoreflect.Message { + mi := &file_proxy_proto_msgTypes[7] + 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 Limiter.ProtoReflect.Descriptor instead. +func (*Limiter) Descriptor() ([]byte, []int) { + return file_proxy_proto_rawDescGZIP(), []int{7} +} + +func (x *Limiter) GetRates() []*internalpb.Rate { + if x != nil { + return x.Rates + } + return nil +} + +func (x *Limiter) GetStates() []milvuspb.QuotaState { + if x != nil { + return x.States + } + return nil +} + +func (x *Limiter) GetCodes() []commonpb.ErrorCode { + if x != nil { + return x.Codes + } + return nil +} + +type SetRatesRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + // deprecated + Rates []*CollectionRate `protobuf:"bytes,2,rep,name=rates,proto3" json:"rates,omitempty"` + RootLimiter *LimiterNode `protobuf:"bytes,3,opt,name=rootLimiter,proto3" json:"rootLimiter,omitempty"` +} + +func (x *SetRatesRequest) Reset() { + *x = SetRatesRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_proxy_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SetRatesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SetRatesRequest) ProtoMessage() {} + +func (x *SetRatesRequest) ProtoReflect() protoreflect.Message { + mi := &file_proxy_proto_msgTypes[8] + 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 SetRatesRequest.ProtoReflect.Descriptor instead. +func (*SetRatesRequest) Descriptor() ([]byte, []int) { + return file_proxy_proto_rawDescGZIP(), []int{8} +} + +func (x *SetRatesRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *SetRatesRequest) GetRates() []*CollectionRate { + if x != nil { + return x.Rates + } + return nil +} + +func (x *SetRatesRequest) GetRootLimiter() *LimiterNode { + if x != nil { + return x.RootLimiter + } + return nil +} + +type ListClientInfosRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` +} + +func (x *ListClientInfosRequest) Reset() { + *x = ListClientInfosRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_proxy_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListClientInfosRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListClientInfosRequest) ProtoMessage() {} + +func (x *ListClientInfosRequest) ProtoReflect() protoreflect.Message { + mi := &file_proxy_proto_msgTypes[9] + 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 ListClientInfosRequest.ProtoReflect.Descriptor instead. +func (*ListClientInfosRequest) Descriptor() ([]byte, []int) { + return file_proxy_proto_rawDescGZIP(), []int{9} +} + +func (x *ListClientInfosRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +type ListClientInfosResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + ClientInfos []*commonpb.ClientInfo `protobuf:"bytes,2,rep,name=client_infos,json=clientInfos,proto3" json:"client_infos,omitempty"` +} + +func (x *ListClientInfosResponse) Reset() { + *x = ListClientInfosResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_proxy_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListClientInfosResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListClientInfosResponse) ProtoMessage() {} + +func (x *ListClientInfosResponse) ProtoReflect() protoreflect.Message { + mi := &file_proxy_proto_msgTypes[10] + 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 ListClientInfosResponse.ProtoReflect.Descriptor instead. +func (*ListClientInfosResponse) Descriptor() ([]byte, []int) { + return file_proxy_proto_rawDescGZIP(), []int{10} +} + +func (x *ListClientInfosResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *ListClientInfosResponse) GetClientInfos() []*commonpb.ClientInfo { + if x != nil { + return x.ClientInfos + } + return nil +} + +var File_proxy_proto protoreflect.FileDescriptor + +var file_proxy_proto_rawDesc = []byte{ + 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x12, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x72, 0x6f, 0x78, + 0x79, 0x1a, 0x0c, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, + 0x0e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, + 0x0c, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xdf, 0x01, + 0x0a, 0x1e, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, + 0x4d, 0x65, 0x74, 0x61, 0x43, 0x61, 0x63, 0x68, 0x65, 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, 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x63, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0d, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x22, + 0x7b, 0x0a, 0x21, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, + 0x72, 0x64, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x43, 0x61, 0x63, 0x68, 0x65, 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, 0x24, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0d, 0x63, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x22, 0x6a, 0x0a, 0x1a, + 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x43, 0x72, 0x65, 0x64, 0x43, 0x61, + 0x63, 0x68, 0x65, 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, 0x1a, 0x0a, 0x08, + 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, + 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x82, 0x01, 0x0a, 0x16, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x43, 0x72, 0x65, 0x64, 0x43, 0x61, 0x63, 0x68, 0x65, 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, 0x1a, 0x0a, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, + 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x22, 0x7f, 0x0a, + 0x1d, 0x52, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x49, 0x6e, + 0x66, 0x6f, 0x43, 0x61, 0x63, 0x68, 0x65, 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, 0x16, 0x0a, 0x06, 0x6f, 0x70, 0x54, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, + 0x52, 0x06, 0x6f, 0x70, 0x54, 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x6f, 0x70, 0x4b, 0x65, + 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6f, 0x70, 0x4b, 0x65, 0x79, 0x22, 0xd2, + 0x01, 0x0a, 0x0e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x74, + 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x12, 0x31, 0x0a, 0x05, 0x72, 0x61, 0x74, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x52, 0x61, 0x74, 0x65, 0x52, 0x05, 0x72, + 0x61, 0x74, 0x65, 0x73, 0x12, 0x37, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x18, 0x03, + 0x20, 0x03, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x51, 0x75, 0x6f, 0x74, 0x61, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x34, 0x0a, + 0x05, 0x63, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0e, 0x32, 0x1e, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x05, 0x63, 0x6f, + 0x64, 0x65, 0x73, 0x22, 0xed, 0x01, 0x0a, 0x0b, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x72, 0x4e, + 0x6f, 0x64, 0x65, 0x12, 0x35, 0x0a, 0x07, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x72, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x2e, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x65, + 0x72, 0x52, 0x07, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x72, 0x12, 0x49, 0x0a, 0x08, 0x63, 0x68, + 0x69, 0x6c, 0x64, 0x72, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x72, 0x6f, 0x78, + 0x79, 0x2e, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, 0x2e, 0x43, 0x68, + 0x69, 0x6c, 0x64, 0x72, 0x65, 0x6e, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, 0x63, 0x68, 0x69, + 0x6c, 0x64, 0x72, 0x65, 0x6e, 0x1a, 0x5c, 0x0a, 0x0d, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x72, 0x65, + 0x6e, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x35, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x2e, 0x4c, 0x69, 0x6d, + 0x69, 0x74, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, + 0x02, 0x38, 0x01, 0x22, 0xab, 0x01, 0x0a, 0x07, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x72, 0x12, + 0x31, 0x0a, 0x05, 0x72, 0x61, 0x74, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, + 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x52, 0x61, 0x74, 0x65, 0x52, 0x05, 0x72, 0x61, 0x74, + 0x65, 0x73, 0x12, 0x37, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, + 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x51, 0x75, 0x6f, 0x74, 0x61, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x34, 0x0a, 0x05, 0x63, + 0x6f, 0x64, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0e, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x05, 0x63, 0x6f, 0x64, 0x65, + 0x73, 0x22, 0xc0, 0x01, 0x0a, 0x0f, 0x53, 0x65, 0x74, 0x52, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, + 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x38, 0x0a, 0x05, 0x72, 0x61, 0x74, 0x65, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x74, 0x65, 0x52, 0x05, 0x72, 0x61, 0x74, 0x65, + 0x73, 0x12, 0x41, 0x0a, 0x0b, 0x72, 0x6f, 0x6f, 0x74, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x72, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x2e, 0x4c, 0x69, 0x6d, 0x69, + 0x74, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x0b, 0x72, 0x6f, 0x6f, 0x74, 0x4c, 0x69, 0x6d, + 0x69, 0x74, 0x65, 0x72, 0x22, 0x4a, 0x0a, 0x16, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x69, 0x65, + 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, + 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, + 0x22, 0x92, 0x01, 0x0a, 0x17, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x49, + 0x6e, 0x66, 0x6f, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, + 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x12, 0x42, 0x0a, 0x0c, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x6e, 0x66, 0x6f, + 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x43, 0x6c, + 0x69, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0b, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, + 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x32, 0xd0, 0x0b, 0x0a, 0x05, 0x50, 0x72, 0x6f, 0x78, 0x79, 0x12, + 0x6c, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x43, + 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x6f, 0x6d, 0x70, + 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x22, 0x00, 0x12, 0x71, 0x0a, + 0x14, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x43, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x32, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x47, 0x65, + 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x43, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x72, 0x0a, 0x1d, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x43, 0x61, 0x63, 0x68, + 0x65, 0x12, 0x32, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x2e, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, + 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x43, 0x61, 0x63, 0x68, 0x65, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x22, 0x00, 0x12, 0x61, 0x0a, 0x0c, 0x47, 0x65, 0x74, 0x44, 0x64, 0x43, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x2a, 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, + 0x44, 0x64, 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, 0x6a, 0x0a, 0x19, 0x49, 0x6e, 0x76, 0x61, 0x6c, + 0x69, 0x64, 0x61, 0x74, 0x65, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x43, + 0x61, 0x63, 0x68, 0x65, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x2e, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, + 0x64, 0x61, 0x74, 0x65, 0x43, 0x72, 0x65, 0x64, 0x43, 0x61, 0x63, 0x68, 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, 0x62, 0x0a, 0x15, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x72, 0x65, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x61, 0x63, 0x68, 0x65, 0x12, 0x2a, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x72, 0x6f, 0x78, + 0x79, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x72, 0x65, 0x64, 0x43, 0x61, 0x63, 0x68, + 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, 0x6a, 0x0a, 0x16, 0x52, 0x65, 0x66, 0x72, 0x65, + 0x73, 0x68, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x43, 0x61, 0x63, 0x68, + 0x65, 0x12, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x2e, 0x52, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x50, 0x6f, + 0x6c, 0x69, 0x63, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x43, 0x61, 0x63, 0x68, 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, 0x64, 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x50, 0x72, 0x6f, 0x78, 0x79, 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, 0x4e, 0x0a, 0x08, 0x53, 0x65, 0x74, + 0x52, 0x61, 0x74, 0x65, 0x73, 0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x2e, 0x53, 0x65, 0x74, 0x52, 0x61, + 0x74, 0x65, 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, 0x6c, 0x0a, 0x0f, 0x4c, 0x69, 0x73, + 0x74, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x2a, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x72, 0x6f, 0x78, + 0x79, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x2e, 0x4c, 0x69, + 0x73, 0x74, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x59, 0x0a, 0x08, 0x49, 0x6d, 0x70, 0x6f, 0x72, + 0x74, 0x56, 0x32, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, + 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, + 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x78, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x50, + 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, + 0x47, 0x65, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, + 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, + 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x66, 0x0a, 0x0b, + 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x12, 0x29, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, + 0x6e, 0x61, 0x6c, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x4c, + 0x69, 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x72, 0x0a, 0x1a, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, + 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x43, 0x61, 0x63, + 0x68, 0x65, 0x12, 0x35, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x2e, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, + 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x43, 0x61, 0x63, + 0x68, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x42, 0x2f, 0x5a, 0x2d, 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, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2f, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x33, +} + +var ( + file_proxy_proto_rawDescOnce sync.Once + file_proxy_proto_rawDescData = file_proxy_proto_rawDesc +) + +func file_proxy_proto_rawDescGZIP() []byte { + file_proxy_proto_rawDescOnce.Do(func() { + file_proxy_proto_rawDescData = protoimpl.X.CompressGZIP(file_proxy_proto_rawDescData) + }) + return file_proxy_proto_rawDescData +} + +var file_proxy_proto_msgTypes = make([]protoimpl.MessageInfo, 12) +var file_proxy_proto_goTypes = []interface{}{ + (*InvalidateCollMetaCacheRequest)(nil), // 0: milvus.proto.proxy.InvalidateCollMetaCacheRequest + (*InvalidateShardLeaderCacheRequest)(nil), // 1: milvus.proto.proxy.InvalidateShardLeaderCacheRequest + (*InvalidateCredCacheRequest)(nil), // 2: milvus.proto.proxy.InvalidateCredCacheRequest + (*UpdateCredCacheRequest)(nil), // 3: milvus.proto.proxy.UpdateCredCacheRequest + (*RefreshPolicyInfoCacheRequest)(nil), // 4: milvus.proto.proxy.RefreshPolicyInfoCacheRequest + (*CollectionRate)(nil), // 5: milvus.proto.proxy.CollectionRate + (*LimiterNode)(nil), // 6: milvus.proto.proxy.LimiterNode + (*Limiter)(nil), // 7: milvus.proto.proxy.Limiter + (*SetRatesRequest)(nil), // 8: milvus.proto.proxy.SetRatesRequest + (*ListClientInfosRequest)(nil), // 9: milvus.proto.proxy.ListClientInfosRequest + (*ListClientInfosResponse)(nil), // 10: milvus.proto.proxy.ListClientInfosResponse + nil, // 11: milvus.proto.proxy.LimiterNode.ChildrenEntry + (*commonpb.MsgBase)(nil), // 12: milvus.proto.common.MsgBase + (*internalpb.Rate)(nil), // 13: milvus.proto.internal.Rate + (milvuspb.QuotaState)(0), // 14: milvus.proto.milvus.QuotaState + (commonpb.ErrorCode)(0), // 15: milvus.proto.common.ErrorCode + (*commonpb.Status)(nil), // 16: milvus.proto.common.Status + (*commonpb.ClientInfo)(nil), // 17: milvus.proto.common.ClientInfo + (*milvuspb.GetComponentStatesRequest)(nil), // 18: milvus.proto.milvus.GetComponentStatesRequest + (*internalpb.GetStatisticsChannelRequest)(nil), // 19: milvus.proto.internal.GetStatisticsChannelRequest + (*internalpb.GetDdChannelRequest)(nil), // 20: milvus.proto.internal.GetDdChannelRequest + (*milvuspb.GetMetricsRequest)(nil), // 21: milvus.proto.milvus.GetMetricsRequest + (*internalpb.ImportRequest)(nil), // 22: milvus.proto.internal.ImportRequest + (*internalpb.GetImportProgressRequest)(nil), // 23: milvus.proto.internal.GetImportProgressRequest + (*internalpb.ListImportsRequest)(nil), // 24: milvus.proto.internal.ListImportsRequest + (*milvuspb.ComponentStates)(nil), // 25: milvus.proto.milvus.ComponentStates + (*milvuspb.StringResponse)(nil), // 26: milvus.proto.milvus.StringResponse + (*milvuspb.GetMetricsResponse)(nil), // 27: milvus.proto.milvus.GetMetricsResponse + (*internalpb.ImportResponse)(nil), // 28: milvus.proto.internal.ImportResponse + (*internalpb.GetImportProgressResponse)(nil), // 29: milvus.proto.internal.GetImportProgressResponse + (*internalpb.ListImportsResponse)(nil), // 30: milvus.proto.internal.ListImportsResponse +} +var file_proxy_proto_depIdxs = []int32{ + 12, // 0: milvus.proto.proxy.InvalidateCollMetaCacheRequest.base:type_name -> milvus.proto.common.MsgBase + 12, // 1: milvus.proto.proxy.InvalidateShardLeaderCacheRequest.base:type_name -> milvus.proto.common.MsgBase + 12, // 2: milvus.proto.proxy.InvalidateCredCacheRequest.base:type_name -> milvus.proto.common.MsgBase + 12, // 3: milvus.proto.proxy.UpdateCredCacheRequest.base:type_name -> milvus.proto.common.MsgBase + 12, // 4: milvus.proto.proxy.RefreshPolicyInfoCacheRequest.base:type_name -> milvus.proto.common.MsgBase + 13, // 5: milvus.proto.proxy.CollectionRate.rates:type_name -> milvus.proto.internal.Rate + 14, // 6: milvus.proto.proxy.CollectionRate.states:type_name -> milvus.proto.milvus.QuotaState + 15, // 7: milvus.proto.proxy.CollectionRate.codes:type_name -> milvus.proto.common.ErrorCode + 7, // 8: milvus.proto.proxy.LimiterNode.limiter:type_name -> milvus.proto.proxy.Limiter + 11, // 9: milvus.proto.proxy.LimiterNode.children:type_name -> milvus.proto.proxy.LimiterNode.ChildrenEntry + 13, // 10: milvus.proto.proxy.Limiter.rates:type_name -> milvus.proto.internal.Rate + 14, // 11: milvus.proto.proxy.Limiter.states:type_name -> milvus.proto.milvus.QuotaState + 15, // 12: milvus.proto.proxy.Limiter.codes:type_name -> milvus.proto.common.ErrorCode + 12, // 13: milvus.proto.proxy.SetRatesRequest.base:type_name -> milvus.proto.common.MsgBase + 5, // 14: milvus.proto.proxy.SetRatesRequest.rates:type_name -> milvus.proto.proxy.CollectionRate + 6, // 15: milvus.proto.proxy.SetRatesRequest.rootLimiter:type_name -> milvus.proto.proxy.LimiterNode + 12, // 16: milvus.proto.proxy.ListClientInfosRequest.base:type_name -> milvus.proto.common.MsgBase + 16, // 17: milvus.proto.proxy.ListClientInfosResponse.status:type_name -> milvus.proto.common.Status + 17, // 18: milvus.proto.proxy.ListClientInfosResponse.client_infos:type_name -> milvus.proto.common.ClientInfo + 6, // 19: milvus.proto.proxy.LimiterNode.ChildrenEntry.value:type_name -> milvus.proto.proxy.LimiterNode + 18, // 20: milvus.proto.proxy.Proxy.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest + 19, // 21: milvus.proto.proxy.Proxy.GetStatisticsChannel:input_type -> milvus.proto.internal.GetStatisticsChannelRequest + 0, // 22: milvus.proto.proxy.Proxy.InvalidateCollectionMetaCache:input_type -> milvus.proto.proxy.InvalidateCollMetaCacheRequest + 20, // 23: milvus.proto.proxy.Proxy.GetDdChannel:input_type -> milvus.proto.internal.GetDdChannelRequest + 2, // 24: milvus.proto.proxy.Proxy.InvalidateCredentialCache:input_type -> milvus.proto.proxy.InvalidateCredCacheRequest + 3, // 25: milvus.proto.proxy.Proxy.UpdateCredentialCache:input_type -> milvus.proto.proxy.UpdateCredCacheRequest + 4, // 26: milvus.proto.proxy.Proxy.RefreshPolicyInfoCache:input_type -> milvus.proto.proxy.RefreshPolicyInfoCacheRequest + 21, // 27: milvus.proto.proxy.Proxy.GetProxyMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest + 8, // 28: milvus.proto.proxy.Proxy.SetRates:input_type -> milvus.proto.proxy.SetRatesRequest + 9, // 29: milvus.proto.proxy.Proxy.ListClientInfos:input_type -> milvus.proto.proxy.ListClientInfosRequest + 22, // 30: milvus.proto.proxy.Proxy.ImportV2:input_type -> milvus.proto.internal.ImportRequest + 23, // 31: milvus.proto.proxy.Proxy.GetImportProgress:input_type -> milvus.proto.internal.GetImportProgressRequest + 24, // 32: milvus.proto.proxy.Proxy.ListImports:input_type -> milvus.proto.internal.ListImportsRequest + 1, // 33: milvus.proto.proxy.Proxy.InvalidateShardLeaderCache:input_type -> milvus.proto.proxy.InvalidateShardLeaderCacheRequest + 25, // 34: milvus.proto.proxy.Proxy.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates + 26, // 35: milvus.proto.proxy.Proxy.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse + 16, // 36: milvus.proto.proxy.Proxy.InvalidateCollectionMetaCache:output_type -> milvus.proto.common.Status + 26, // 37: milvus.proto.proxy.Proxy.GetDdChannel:output_type -> milvus.proto.milvus.StringResponse + 16, // 38: milvus.proto.proxy.Proxy.InvalidateCredentialCache:output_type -> milvus.proto.common.Status + 16, // 39: milvus.proto.proxy.Proxy.UpdateCredentialCache:output_type -> milvus.proto.common.Status + 16, // 40: milvus.proto.proxy.Proxy.RefreshPolicyInfoCache:output_type -> milvus.proto.common.Status + 27, // 41: milvus.proto.proxy.Proxy.GetProxyMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse + 16, // 42: milvus.proto.proxy.Proxy.SetRates:output_type -> milvus.proto.common.Status + 10, // 43: milvus.proto.proxy.Proxy.ListClientInfos:output_type -> milvus.proto.proxy.ListClientInfosResponse + 28, // 44: milvus.proto.proxy.Proxy.ImportV2:output_type -> milvus.proto.internal.ImportResponse + 29, // 45: milvus.proto.proxy.Proxy.GetImportProgress:output_type -> milvus.proto.internal.GetImportProgressResponse + 30, // 46: milvus.proto.proxy.Proxy.ListImports:output_type -> milvus.proto.internal.ListImportsResponse + 16, // 47: milvus.proto.proxy.Proxy.InvalidateShardLeaderCache:output_type -> milvus.proto.common.Status + 34, // [34:48] is the sub-list for method output_type + 20, // [20:34] is the sub-list for method input_type + 20, // [20:20] is the sub-list for extension type_name + 20, // [20:20] is the sub-list for extension extendee + 0, // [0:20] is the sub-list for field type_name +} + +func init() { file_proxy_proto_init() } +func file_proxy_proto_init() { + if File_proxy_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_proxy_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*InvalidateCollMetaCacheRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proxy_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*InvalidateShardLeaderCacheRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proxy_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*InvalidateCredCacheRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proxy_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*UpdateCredCacheRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proxy_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RefreshPolicyInfoCacheRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proxy_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CollectionRate); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proxy_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*LimiterNode); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proxy_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Limiter); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proxy_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SetRatesRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proxy_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListClientInfosRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proxy_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListClientInfosResponse); 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{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_proxy_proto_rawDesc, + NumEnums: 0, + NumMessages: 12, + NumExtensions: 0, + NumServices: 1, + }, + GoTypes: file_proxy_proto_goTypes, + DependencyIndexes: file_proxy_proto_depIdxs, + MessageInfos: file_proxy_proto_msgTypes, + }.Build() + File_proxy_proto = out.File + file_proxy_proto_rawDesc = nil + file_proxy_proto_goTypes = nil + file_proxy_proto_depIdxs = nil +} diff --git a/pkg/proto/proxypb/proxy_grpc.pb.go b/pkg/proto/proxypb/proxy_grpc.pb.go new file mode 100644 index 0000000000000..8f155d508bd56 --- /dev/null +++ b/pkg/proto/proxypb/proxy_grpc.pb.go @@ -0,0 +1,593 @@ +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. +// versions: +// - protoc-gen-go-grpc v1.3.0 +// - protoc v3.21.4 +// source: proxy.proto + +package proxypb + +import ( + context "context" + commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" +) + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +// Requires gRPC-Go v1.32.0 or later. +const _ = grpc.SupportPackageIsVersion7 + +const ( + Proxy_GetComponentStates_FullMethodName = "/milvus.proto.proxy.Proxy/GetComponentStates" + Proxy_GetStatisticsChannel_FullMethodName = "/milvus.proto.proxy.Proxy/GetStatisticsChannel" + Proxy_InvalidateCollectionMetaCache_FullMethodName = "/milvus.proto.proxy.Proxy/InvalidateCollectionMetaCache" + Proxy_GetDdChannel_FullMethodName = "/milvus.proto.proxy.Proxy/GetDdChannel" + Proxy_InvalidateCredentialCache_FullMethodName = "/milvus.proto.proxy.Proxy/InvalidateCredentialCache" + Proxy_UpdateCredentialCache_FullMethodName = "/milvus.proto.proxy.Proxy/UpdateCredentialCache" + Proxy_RefreshPolicyInfoCache_FullMethodName = "/milvus.proto.proxy.Proxy/RefreshPolicyInfoCache" + Proxy_GetProxyMetrics_FullMethodName = "/milvus.proto.proxy.Proxy/GetProxyMetrics" + Proxy_SetRates_FullMethodName = "/milvus.proto.proxy.Proxy/SetRates" + Proxy_ListClientInfos_FullMethodName = "/milvus.proto.proxy.Proxy/ListClientInfos" + Proxy_ImportV2_FullMethodName = "/milvus.proto.proxy.Proxy/ImportV2" + Proxy_GetImportProgress_FullMethodName = "/milvus.proto.proxy.Proxy/GetImportProgress" + Proxy_ListImports_FullMethodName = "/milvus.proto.proxy.Proxy/ListImports" + Proxy_InvalidateShardLeaderCache_FullMethodName = "/milvus.proto.proxy.Proxy/InvalidateShardLeaderCache" +) + +// ProxyClient is the client API for Proxy service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +type ProxyClient interface { + GetComponentStates(ctx context.Context, in *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) + GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) + InvalidateCollectionMetaCache(ctx context.Context, in *InvalidateCollMetaCacheRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + GetDdChannel(ctx context.Context, in *internalpb.GetDdChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) + InvalidateCredentialCache(ctx context.Context, in *InvalidateCredCacheRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + UpdateCredentialCache(ctx context.Context, in *UpdateCredCacheRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + RefreshPolicyInfoCache(ctx context.Context, in *RefreshPolicyInfoCacheRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + GetProxyMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) + SetRates(ctx context.Context, in *SetRatesRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + ListClientInfos(ctx context.Context, in *ListClientInfosRequest, opts ...grpc.CallOption) (*ListClientInfosResponse, error) + // importV2 + ImportV2(ctx context.Context, in *internalpb.ImportRequest, opts ...grpc.CallOption) (*internalpb.ImportResponse, error) + GetImportProgress(ctx context.Context, in *internalpb.GetImportProgressRequest, opts ...grpc.CallOption) (*internalpb.GetImportProgressResponse, error) + ListImports(ctx context.Context, in *internalpb.ListImportsRequest, opts ...grpc.CallOption) (*internalpb.ListImportsResponse, error) + InvalidateShardLeaderCache(ctx context.Context, in *InvalidateShardLeaderCacheRequest, opts ...grpc.CallOption) (*commonpb.Status, error) +} + +type proxyClient struct { + cc grpc.ClientConnInterface +} + +func NewProxyClient(cc grpc.ClientConnInterface) ProxyClient { + return &proxyClient{cc} +} + +func (c *proxyClient) GetComponentStates(ctx context.Context, in *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) { + out := new(milvuspb.ComponentStates) + err := c.cc.Invoke(ctx, Proxy_GetComponentStates_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *proxyClient) GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) { + out := new(milvuspb.StringResponse) + err := c.cc.Invoke(ctx, Proxy_GetStatisticsChannel_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *proxyClient) InvalidateCollectionMetaCache(ctx context.Context, in *InvalidateCollMetaCacheRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, Proxy_InvalidateCollectionMetaCache_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *proxyClient) GetDdChannel(ctx context.Context, in *internalpb.GetDdChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) { + out := new(milvuspb.StringResponse) + err := c.cc.Invoke(ctx, Proxy_GetDdChannel_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *proxyClient) InvalidateCredentialCache(ctx context.Context, in *InvalidateCredCacheRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, Proxy_InvalidateCredentialCache_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *proxyClient) UpdateCredentialCache(ctx context.Context, in *UpdateCredCacheRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, Proxy_UpdateCredentialCache_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *proxyClient) RefreshPolicyInfoCache(ctx context.Context, in *RefreshPolicyInfoCacheRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, Proxy_RefreshPolicyInfoCache_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *proxyClient) GetProxyMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) { + out := new(milvuspb.GetMetricsResponse) + err := c.cc.Invoke(ctx, Proxy_GetProxyMetrics_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *proxyClient) SetRates(ctx context.Context, in *SetRatesRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, Proxy_SetRates_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *proxyClient) ListClientInfos(ctx context.Context, in *ListClientInfosRequest, opts ...grpc.CallOption) (*ListClientInfosResponse, error) { + out := new(ListClientInfosResponse) + err := c.cc.Invoke(ctx, Proxy_ListClientInfos_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *proxyClient) ImportV2(ctx context.Context, in *internalpb.ImportRequest, opts ...grpc.CallOption) (*internalpb.ImportResponse, error) { + out := new(internalpb.ImportResponse) + err := c.cc.Invoke(ctx, Proxy_ImportV2_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *proxyClient) GetImportProgress(ctx context.Context, in *internalpb.GetImportProgressRequest, opts ...grpc.CallOption) (*internalpb.GetImportProgressResponse, error) { + out := new(internalpb.GetImportProgressResponse) + err := c.cc.Invoke(ctx, Proxy_GetImportProgress_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *proxyClient) ListImports(ctx context.Context, in *internalpb.ListImportsRequest, opts ...grpc.CallOption) (*internalpb.ListImportsResponse, error) { + out := new(internalpb.ListImportsResponse) + err := c.cc.Invoke(ctx, Proxy_ListImports_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *proxyClient) InvalidateShardLeaderCache(ctx context.Context, in *InvalidateShardLeaderCacheRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, Proxy_InvalidateShardLeaderCache_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// ProxyServer is the server API for Proxy service. +// All implementations should embed UnimplementedProxyServer +// for forward compatibility +type ProxyServer interface { + GetComponentStates(context.Context, *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) + GetStatisticsChannel(context.Context, *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) + InvalidateCollectionMetaCache(context.Context, *InvalidateCollMetaCacheRequest) (*commonpb.Status, error) + GetDdChannel(context.Context, *internalpb.GetDdChannelRequest) (*milvuspb.StringResponse, error) + InvalidateCredentialCache(context.Context, *InvalidateCredCacheRequest) (*commonpb.Status, error) + UpdateCredentialCache(context.Context, *UpdateCredCacheRequest) (*commonpb.Status, error) + RefreshPolicyInfoCache(context.Context, *RefreshPolicyInfoCacheRequest) (*commonpb.Status, error) + GetProxyMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) + SetRates(context.Context, *SetRatesRequest) (*commonpb.Status, error) + ListClientInfos(context.Context, *ListClientInfosRequest) (*ListClientInfosResponse, error) + // importV2 + ImportV2(context.Context, *internalpb.ImportRequest) (*internalpb.ImportResponse, error) + GetImportProgress(context.Context, *internalpb.GetImportProgressRequest) (*internalpb.GetImportProgressResponse, error) + ListImports(context.Context, *internalpb.ListImportsRequest) (*internalpb.ListImportsResponse, error) + InvalidateShardLeaderCache(context.Context, *InvalidateShardLeaderCacheRequest) (*commonpb.Status, error) +} + +// UnimplementedProxyServer should be embedded to have forward compatible implementations. +type UnimplementedProxyServer struct { +} + +func (UnimplementedProxyServer) GetComponentStates(context.Context, *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetComponentStates not implemented") +} +func (UnimplementedProxyServer) GetStatisticsChannel(context.Context, *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetStatisticsChannel not implemented") +} +func (UnimplementedProxyServer) InvalidateCollectionMetaCache(context.Context, *InvalidateCollMetaCacheRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method InvalidateCollectionMetaCache not implemented") +} +func (UnimplementedProxyServer) GetDdChannel(context.Context, *internalpb.GetDdChannelRequest) (*milvuspb.StringResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetDdChannel not implemented") +} +func (UnimplementedProxyServer) InvalidateCredentialCache(context.Context, *InvalidateCredCacheRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method InvalidateCredentialCache not implemented") +} +func (UnimplementedProxyServer) UpdateCredentialCache(context.Context, *UpdateCredCacheRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateCredentialCache not implemented") +} +func (UnimplementedProxyServer) RefreshPolicyInfoCache(context.Context, *RefreshPolicyInfoCacheRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method RefreshPolicyInfoCache not implemented") +} +func (UnimplementedProxyServer) GetProxyMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetProxyMetrics not implemented") +} +func (UnimplementedProxyServer) SetRates(context.Context, *SetRatesRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method SetRates not implemented") +} +func (UnimplementedProxyServer) ListClientInfos(context.Context, *ListClientInfosRequest) (*ListClientInfosResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListClientInfos not implemented") +} +func (UnimplementedProxyServer) ImportV2(context.Context, *internalpb.ImportRequest) (*internalpb.ImportResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ImportV2 not implemented") +} +func (UnimplementedProxyServer) GetImportProgress(context.Context, *internalpb.GetImportProgressRequest) (*internalpb.GetImportProgressResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetImportProgress not implemented") +} +func (UnimplementedProxyServer) ListImports(context.Context, *internalpb.ListImportsRequest) (*internalpb.ListImportsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListImports not implemented") +} +func (UnimplementedProxyServer) InvalidateShardLeaderCache(context.Context, *InvalidateShardLeaderCacheRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method InvalidateShardLeaderCache not implemented") +} + +// UnsafeProxyServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to ProxyServer will +// result in compilation errors. +type UnsafeProxyServer interface { + mustEmbedUnimplementedProxyServer() +} + +func RegisterProxyServer(s grpc.ServiceRegistrar, srv ProxyServer) { + s.RegisterService(&Proxy_ServiceDesc, srv) +} + +func _Proxy_GetComponentStates_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.GetComponentStatesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProxyServer).GetComponentStates(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: Proxy_GetComponentStates_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProxyServer).GetComponentStates(ctx, req.(*milvuspb.GetComponentStatesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Proxy_GetStatisticsChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.GetStatisticsChannelRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProxyServer).GetStatisticsChannel(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: Proxy_GetStatisticsChannel_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProxyServer).GetStatisticsChannel(ctx, req.(*internalpb.GetStatisticsChannelRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Proxy_InvalidateCollectionMetaCache_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(InvalidateCollMetaCacheRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProxyServer).InvalidateCollectionMetaCache(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: Proxy_InvalidateCollectionMetaCache_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProxyServer).InvalidateCollectionMetaCache(ctx, req.(*InvalidateCollMetaCacheRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Proxy_GetDdChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.GetDdChannelRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProxyServer).GetDdChannel(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: Proxy_GetDdChannel_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProxyServer).GetDdChannel(ctx, req.(*internalpb.GetDdChannelRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Proxy_InvalidateCredentialCache_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(InvalidateCredCacheRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProxyServer).InvalidateCredentialCache(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: Proxy_InvalidateCredentialCache_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProxyServer).InvalidateCredentialCache(ctx, req.(*InvalidateCredCacheRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Proxy_UpdateCredentialCache_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(UpdateCredCacheRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProxyServer).UpdateCredentialCache(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: Proxy_UpdateCredentialCache_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProxyServer).UpdateCredentialCache(ctx, req.(*UpdateCredCacheRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Proxy_RefreshPolicyInfoCache_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RefreshPolicyInfoCacheRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProxyServer).RefreshPolicyInfoCache(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: Proxy_RefreshPolicyInfoCache_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProxyServer).RefreshPolicyInfoCache(ctx, req.(*RefreshPolicyInfoCacheRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Proxy_GetProxyMetrics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.GetMetricsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProxyServer).GetProxyMetrics(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: Proxy_GetProxyMetrics_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProxyServer).GetProxyMetrics(ctx, req.(*milvuspb.GetMetricsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Proxy_SetRates_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(SetRatesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProxyServer).SetRates(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: Proxy_SetRates_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProxyServer).SetRates(ctx, req.(*SetRatesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Proxy_ListClientInfos_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ListClientInfosRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProxyServer).ListClientInfos(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: Proxy_ListClientInfos_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProxyServer).ListClientInfos(ctx, req.(*ListClientInfosRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Proxy_ImportV2_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.ImportRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProxyServer).ImportV2(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: Proxy_ImportV2_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProxyServer).ImportV2(ctx, req.(*internalpb.ImportRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Proxy_GetImportProgress_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.GetImportProgressRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProxyServer).GetImportProgress(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: Proxy_GetImportProgress_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProxyServer).GetImportProgress(ctx, req.(*internalpb.GetImportProgressRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Proxy_ListImports_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.ListImportsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProxyServer).ListImports(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: Proxy_ListImports_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProxyServer).ListImports(ctx, req.(*internalpb.ListImportsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Proxy_InvalidateShardLeaderCache_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(InvalidateShardLeaderCacheRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProxyServer).InvalidateShardLeaderCache(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: Proxy_InvalidateShardLeaderCache_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProxyServer).InvalidateShardLeaderCache(ctx, req.(*InvalidateShardLeaderCacheRequest)) + } + return interceptor(ctx, in, info, handler) +} + +// Proxy_ServiceDesc is the grpc.ServiceDesc for Proxy service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var Proxy_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "milvus.proto.proxy.Proxy", + HandlerType: (*ProxyServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "GetComponentStates", + Handler: _Proxy_GetComponentStates_Handler, + }, + { + MethodName: "GetStatisticsChannel", + Handler: _Proxy_GetStatisticsChannel_Handler, + }, + { + MethodName: "InvalidateCollectionMetaCache", + Handler: _Proxy_InvalidateCollectionMetaCache_Handler, + }, + { + MethodName: "GetDdChannel", + Handler: _Proxy_GetDdChannel_Handler, + }, + { + MethodName: "InvalidateCredentialCache", + Handler: _Proxy_InvalidateCredentialCache_Handler, + }, + { + MethodName: "UpdateCredentialCache", + Handler: _Proxy_UpdateCredentialCache_Handler, + }, + { + MethodName: "RefreshPolicyInfoCache", + Handler: _Proxy_RefreshPolicyInfoCache_Handler, + }, + { + MethodName: "GetProxyMetrics", + Handler: _Proxy_GetProxyMetrics_Handler, + }, + { + MethodName: "SetRates", + Handler: _Proxy_SetRates_Handler, + }, + { + MethodName: "ListClientInfos", + Handler: _Proxy_ListClientInfos_Handler, + }, + { + MethodName: "ImportV2", + Handler: _Proxy_ImportV2_Handler, + }, + { + MethodName: "GetImportProgress", + Handler: _Proxy_GetImportProgress_Handler, + }, + { + MethodName: "ListImports", + Handler: _Proxy_ListImports_Handler, + }, + { + MethodName: "InvalidateShardLeaderCache", + Handler: _Proxy_InvalidateShardLeaderCache_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "proxy.proto", +} diff --git a/internal/proto/query_coord.proto b/pkg/proto/query_coord.proto similarity index 99% rename from internal/proto/query_coord.proto rename to pkg/proto/query_coord.proto index 9b44e6f5c9560..e84ecf51584b6 100644 --- a/internal/proto/query_coord.proto +++ b/pkg/proto/query_coord.proto @@ -2,7 +2,7 @@ syntax = "proto3"; package milvus.proto.query; -option go_package = "github.com/milvus-io/milvus/internal/proto/querypb"; +option go_package = "github.com/milvus-io/milvus/pkg/proto/querypb"; import "common.proto"; import "milvus.proto"; diff --git a/pkg/proto/querypb/query_coord.pb.go b/pkg/proto/querypb/query_coord.pb.go new file mode 100644 index 0000000000000..8ce9c2d41034b --- /dev/null +++ b/pkg/proto/querypb/query_coord.pb.go @@ -0,0 +1,10390 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.33.0 +// protoc v3.21.4 +// source: query_coord.proto + +package querypb + +import ( + commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + msgpb "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" + rgpb "github.com/milvus-io/milvus-proto/go-api/v2/rgpb" + schemapb "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" + datapb "github.com/milvus-io/milvus/pkg/proto/datapb" + indexpb "github.com/milvus-io/milvus/pkg/proto/indexpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type LoadScope int32 + +const ( + LoadScope_Full LoadScope = 0 + LoadScope_Delta LoadScope = 1 + LoadScope_Index LoadScope = 2 +) + +// Enum value maps for LoadScope. +var ( + LoadScope_name = map[int32]string{ + 0: "Full", + 1: "Delta", + 2: "Index", + } + LoadScope_value = map[string]int32{ + "Full": 0, + "Delta": 1, + "Index": 2, + } +) + +func (x LoadScope) Enum() *LoadScope { + p := new(LoadScope) + *p = x + return p +} + +func (x LoadScope) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (LoadScope) Descriptor() protoreflect.EnumDescriptor { + return file_query_coord_proto_enumTypes[0].Descriptor() +} + +func (LoadScope) Type() protoreflect.EnumType { + return &file_query_coord_proto_enumTypes[0] +} + +func (x LoadScope) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use LoadScope.Descriptor instead. +func (LoadScope) EnumDescriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{0} +} + +type DataScope int32 + +const ( + DataScope_UnKnown DataScope = 0 + DataScope_All DataScope = 1 + DataScope_Streaming DataScope = 2 + DataScope_Historical DataScope = 3 +) + +// Enum value maps for DataScope. +var ( + DataScope_name = map[int32]string{ + 0: "UnKnown", + 1: "All", + 2: "Streaming", + 3: "Historical", + } + DataScope_value = map[string]int32{ + "UnKnown": 0, + "All": 1, + "Streaming": 2, + "Historical": 3, + } +) + +func (x DataScope) Enum() *DataScope { + p := new(DataScope) + *p = x + return p +} + +func (x DataScope) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (DataScope) Descriptor() protoreflect.EnumDescriptor { + return file_query_coord_proto_enumTypes[1].Descriptor() +} + +func (DataScope) Type() protoreflect.EnumType { + return &file_query_coord_proto_enumTypes[1] +} + +func (x DataScope) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use DataScope.Descriptor instead. +func (DataScope) EnumDescriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{1} +} + +type PartitionState int32 + +const ( + PartitionState_NotExist PartitionState = 0 + PartitionState_NotPresent PartitionState = 1 + PartitionState_OnDisk PartitionState = 2 + PartitionState_PartialInMemory PartitionState = 3 + PartitionState_InMemory PartitionState = 4 + PartitionState_PartialInGPU PartitionState = 5 + PartitionState_InGPU PartitionState = 6 +) + +// Enum value maps for PartitionState. +var ( + PartitionState_name = map[int32]string{ + 0: "NotExist", + 1: "NotPresent", + 2: "OnDisk", + 3: "PartialInMemory", + 4: "InMemory", + 5: "PartialInGPU", + 6: "InGPU", + } + PartitionState_value = map[string]int32{ + "NotExist": 0, + "NotPresent": 1, + "OnDisk": 2, + "PartialInMemory": 3, + "InMemory": 4, + "PartialInGPU": 5, + "InGPU": 6, + } +) + +func (x PartitionState) Enum() *PartitionState { + p := new(PartitionState) + *p = x + return p +} + +func (x PartitionState) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (PartitionState) Descriptor() protoreflect.EnumDescriptor { + return file_query_coord_proto_enumTypes[2].Descriptor() +} + +func (PartitionState) Type() protoreflect.EnumType { + return &file_query_coord_proto_enumTypes[2] +} + +func (x PartitionState) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use PartitionState.Descriptor instead. +func (PartitionState) EnumDescriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{2} +} + +type TriggerCondition int32 + +const ( + TriggerCondition_UnKnowCondition TriggerCondition = 0 + TriggerCondition_Handoff TriggerCondition = 1 + TriggerCondition_LoadBalance TriggerCondition = 2 + TriggerCondition_GrpcRequest TriggerCondition = 3 + TriggerCondition_NodeDown TriggerCondition = 4 +) + +// Enum value maps for TriggerCondition. +var ( + TriggerCondition_name = map[int32]string{ + 0: "UnKnowCondition", + 1: "Handoff", + 2: "LoadBalance", + 3: "GrpcRequest", + 4: "NodeDown", + } + TriggerCondition_value = map[string]int32{ + "UnKnowCondition": 0, + "Handoff": 1, + "LoadBalance": 2, + "GrpcRequest": 3, + "NodeDown": 4, + } +) + +func (x TriggerCondition) Enum() *TriggerCondition { + p := new(TriggerCondition) + *p = x + return p +} + +func (x TriggerCondition) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (TriggerCondition) Descriptor() protoreflect.EnumDescriptor { + return file_query_coord_proto_enumTypes[3].Descriptor() +} + +func (TriggerCondition) Type() protoreflect.EnumType { + return &file_query_coord_proto_enumTypes[3] +} + +func (x TriggerCondition) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use TriggerCondition.Descriptor instead. +func (TriggerCondition) EnumDescriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{3} +} + +type LoadType int32 + +const ( + LoadType_UnKnownType LoadType = 0 + LoadType_LoadPartition LoadType = 1 + LoadType_LoadCollection LoadType = 2 +) + +// Enum value maps for LoadType. +var ( + LoadType_name = map[int32]string{ + 0: "UnKnownType", + 1: "LoadPartition", + 2: "LoadCollection", + } + LoadType_value = map[string]int32{ + "UnKnownType": 0, + "LoadPartition": 1, + "LoadCollection": 2, + } +) + +func (x LoadType) Enum() *LoadType { + p := new(LoadType) + *p = x + return p +} + +func (x LoadType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (LoadType) Descriptor() protoreflect.EnumDescriptor { + return file_query_coord_proto_enumTypes[4].Descriptor() +} + +func (LoadType) Type() protoreflect.EnumType { + return &file_query_coord_proto_enumTypes[4] +} + +func (x LoadType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use LoadType.Descriptor instead. +func (LoadType) EnumDescriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{4} +} + +type LoadStatus int32 + +const ( + LoadStatus_Invalid LoadStatus = 0 + LoadStatus_Loading LoadStatus = 1 + LoadStatus_Loaded LoadStatus = 2 +) + +// Enum value maps for LoadStatus. +var ( + LoadStatus_name = map[int32]string{ + 0: "Invalid", + 1: "Loading", + 2: "Loaded", + } + LoadStatus_value = map[string]int32{ + "Invalid": 0, + "Loading": 1, + "Loaded": 2, + } +) + +func (x LoadStatus) Enum() *LoadStatus { + p := new(LoadStatus) + *p = x + return p +} + +func (x LoadStatus) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (LoadStatus) Descriptor() protoreflect.EnumDescriptor { + return file_query_coord_proto_enumTypes[5].Descriptor() +} + +func (LoadStatus) Type() protoreflect.EnumType { + return &file_query_coord_proto_enumTypes[5] +} + +func (x LoadStatus) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use LoadStatus.Descriptor instead. +func (LoadStatus) EnumDescriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{5} +} + +type SyncType int32 + +const ( + SyncType_Remove SyncType = 0 + SyncType_Set SyncType = 1 + SyncType_Amend SyncType = 2 + SyncType_UpdateVersion SyncType = 3 + SyncType_UpdatePartitionStats SyncType = 4 +) + +// Enum value maps for SyncType. +var ( + SyncType_name = map[int32]string{ + 0: "Remove", + 1: "Set", + 2: "Amend", + 3: "UpdateVersion", + 4: "UpdatePartitionStats", + } + SyncType_value = map[string]int32{ + "Remove": 0, + "Set": 1, + "Amend": 2, + "UpdateVersion": 3, + "UpdatePartitionStats": 4, + } +) + +func (x SyncType) Enum() *SyncType { + p := new(SyncType) + *p = x + return p +} + +func (x SyncType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SyncType) Descriptor() protoreflect.EnumDescriptor { + return file_query_coord_proto_enumTypes[6].Descriptor() +} + +func (SyncType) Type() protoreflect.EnumType { + return &file_query_coord_proto_enumTypes[6] +} + +func (x SyncType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use SyncType.Descriptor instead. +func (SyncType) EnumDescriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{6} +} + +type ShowCollectionsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + // Not useful for now + DbID int64 `protobuf:"varint,2,opt,name=dbID,proto3" json:"dbID,omitempty"` + CollectionIDs []int64 `protobuf:"varint,3,rep,packed,name=collectionIDs,proto3" json:"collectionIDs,omitempty"` +} + +func (x *ShowCollectionsRequest) Reset() { + *x = ShowCollectionsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ShowCollectionsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ShowCollectionsRequest) ProtoMessage() {} + +func (x *ShowCollectionsRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[0] + 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 ShowCollectionsRequest.ProtoReflect.Descriptor instead. +func (*ShowCollectionsRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{0} +} + +func (x *ShowCollectionsRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *ShowCollectionsRequest) GetDbID() int64 { + if x != nil { + return x.DbID + } + return 0 +} + +func (x *ShowCollectionsRequest) GetCollectionIDs() []int64 { + if x != nil { + return x.CollectionIDs + } + return nil +} + +type ShowCollectionsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + CollectionIDs []int64 `protobuf:"varint,2,rep,packed,name=collectionIDs,proto3" json:"collectionIDs,omitempty"` + InMemoryPercentages []int64 `protobuf:"varint,3,rep,packed,name=inMemory_percentages,json=inMemoryPercentages,proto3" json:"inMemory_percentages,omitempty"` + QueryServiceAvailable []bool `protobuf:"varint,4,rep,packed,name=query_service_available,json=queryServiceAvailable,proto3" json:"query_service_available,omitempty"` + RefreshProgress []int64 `protobuf:"varint,5,rep,packed,name=refresh_progress,json=refreshProgress,proto3" json:"refresh_progress,omitempty"` + LoadFields []*schemapb.LongArray `protobuf:"bytes,6,rep,name=load_fields,json=loadFields,proto3" json:"load_fields,omitempty"` +} + +func (x *ShowCollectionsResponse) Reset() { + *x = ShowCollectionsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ShowCollectionsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ShowCollectionsResponse) ProtoMessage() {} + +func (x *ShowCollectionsResponse) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[1] + 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 ShowCollectionsResponse.ProtoReflect.Descriptor instead. +func (*ShowCollectionsResponse) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{1} +} + +func (x *ShowCollectionsResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *ShowCollectionsResponse) GetCollectionIDs() []int64 { + if x != nil { + return x.CollectionIDs + } + return nil +} + +func (x *ShowCollectionsResponse) GetInMemoryPercentages() []int64 { + if x != nil { + return x.InMemoryPercentages + } + return nil +} + +func (x *ShowCollectionsResponse) GetQueryServiceAvailable() []bool { + if x != nil { + return x.QueryServiceAvailable + } + return nil +} + +func (x *ShowCollectionsResponse) GetRefreshProgress() []int64 { + if x != nil { + return x.RefreshProgress + } + return nil +} + +func (x *ShowCollectionsResponse) GetLoadFields() []*schemapb.LongArray { + if x != nil { + return x.LoadFields + } + return nil +} + +type ShowPartitionsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + DbID int64 `protobuf:"varint,2,opt,name=dbID,proto3" json:"dbID,omitempty"` + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionIDs []int64 `protobuf:"varint,4,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` +} + +func (x *ShowPartitionsRequest) Reset() { + *x = ShowPartitionsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ShowPartitionsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ShowPartitionsRequest) ProtoMessage() {} + +func (x *ShowPartitionsRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[2] + 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 ShowPartitionsRequest.ProtoReflect.Descriptor instead. +func (*ShowPartitionsRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{2} +} + +func (x *ShowPartitionsRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *ShowPartitionsRequest) GetDbID() int64 { + if x != nil { + return x.DbID + } + return 0 +} + +func (x *ShowPartitionsRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *ShowPartitionsRequest) GetPartitionIDs() []int64 { + if x != nil { + return x.PartitionIDs + } + return nil +} + +type ShowPartitionsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + PartitionIDs []int64 `protobuf:"varint,2,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` + InMemoryPercentages []int64 `protobuf:"varint,3,rep,packed,name=inMemory_percentages,json=inMemoryPercentages,proto3" json:"inMemory_percentages,omitempty"` + RefreshProgress []int64 `protobuf:"varint,4,rep,packed,name=refresh_progress,json=refreshProgress,proto3" json:"refresh_progress,omitempty"` +} + +func (x *ShowPartitionsResponse) Reset() { + *x = ShowPartitionsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ShowPartitionsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ShowPartitionsResponse) ProtoMessage() {} + +func (x *ShowPartitionsResponse) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[3] + 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 ShowPartitionsResponse.ProtoReflect.Descriptor instead. +func (*ShowPartitionsResponse) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{3} +} + +func (x *ShowPartitionsResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *ShowPartitionsResponse) GetPartitionIDs() []int64 { + if x != nil { + return x.PartitionIDs + } + return nil +} + +func (x *ShowPartitionsResponse) GetInMemoryPercentages() []int64 { + if x != nil { + return x.InMemoryPercentages + } + return nil +} + +func (x *ShowPartitionsResponse) GetRefreshProgress() []int64 { + if x != nil { + return x.RefreshProgress + } + return nil +} + +type LoadCollectionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + DbID int64 `protobuf:"varint,2,opt,name=dbID,proto3" json:"dbID,omitempty"` + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + Schema *schemapb.CollectionSchema `protobuf:"bytes,4,opt,name=schema,proto3" json:"schema,omitempty"` + ReplicaNumber int32 `protobuf:"varint,5,opt,name=replica_number,json=replicaNumber,proto3" json:"replica_number,omitempty"` + // fieldID -> indexID + FieldIndexID map[int64]int64 `protobuf:"bytes,6,rep,name=field_indexID,json=fieldIndexID,proto3" json:"field_indexID,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + Refresh bool `protobuf:"varint,7,opt,name=refresh,proto3" json:"refresh,omitempty"` + // resource group names + ResourceGroups []string `protobuf:"bytes,8,rep,name=resource_groups,json=resourceGroups,proto3" json:"resource_groups,omitempty"` + LoadFields []int64 `protobuf:"varint,9,rep,packed,name=load_fields,json=loadFields,proto3" json:"load_fields,omitempty"` +} + +func (x *LoadCollectionRequest) Reset() { + *x = LoadCollectionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *LoadCollectionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LoadCollectionRequest) ProtoMessage() {} + +func (x *LoadCollectionRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[4] + 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 LoadCollectionRequest.ProtoReflect.Descriptor instead. +func (*LoadCollectionRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{4} +} + +func (x *LoadCollectionRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *LoadCollectionRequest) GetDbID() int64 { + if x != nil { + return x.DbID + } + return 0 +} + +func (x *LoadCollectionRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *LoadCollectionRequest) GetSchema() *schemapb.CollectionSchema { + if x != nil { + return x.Schema + } + return nil +} + +func (x *LoadCollectionRequest) GetReplicaNumber() int32 { + if x != nil { + return x.ReplicaNumber + } + return 0 +} + +func (x *LoadCollectionRequest) GetFieldIndexID() map[int64]int64 { + if x != nil { + return x.FieldIndexID + } + return nil +} + +func (x *LoadCollectionRequest) GetRefresh() bool { + if x != nil { + return x.Refresh + } + return false +} + +func (x *LoadCollectionRequest) GetResourceGroups() []string { + if x != nil { + return x.ResourceGroups + } + return nil +} + +func (x *LoadCollectionRequest) GetLoadFields() []int64 { + if x != nil { + return x.LoadFields + } + return nil +} + +type ReleaseCollectionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + DbID int64 `protobuf:"varint,2,opt,name=dbID,proto3" json:"dbID,omitempty"` + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + NodeID int64 `protobuf:"varint,4,opt,name=nodeID,proto3" json:"nodeID,omitempty"` +} + +func (x *ReleaseCollectionRequest) Reset() { + *x = ReleaseCollectionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ReleaseCollectionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ReleaseCollectionRequest) ProtoMessage() {} + +func (x *ReleaseCollectionRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[5] + 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 ReleaseCollectionRequest.ProtoReflect.Descriptor instead. +func (*ReleaseCollectionRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{5} +} + +func (x *ReleaseCollectionRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *ReleaseCollectionRequest) GetDbID() int64 { + if x != nil { + return x.DbID + } + return 0 +} + +func (x *ReleaseCollectionRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *ReleaseCollectionRequest) GetNodeID() int64 { + if x != nil { + return x.NodeID + } + return 0 +} + +type GetStatisticsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Req *internalpb.GetStatisticsRequest `protobuf:"bytes,1,opt,name=req,proto3" json:"req,omitempty"` + DmlChannels []string `protobuf:"bytes,2,rep,name=dml_channels,json=dmlChannels,proto3" json:"dml_channels,omitempty"` + SegmentIDs []int64 `protobuf:"varint,3,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,omitempty"` + FromShardLeader bool `protobuf:"varint,4,opt,name=from_shard_leader,json=fromShardLeader,proto3" json:"from_shard_leader,omitempty"` + Scope DataScope `protobuf:"varint,5,opt,name=scope,proto3,enum=milvus.proto.query.DataScope" json:"scope,omitempty"` // All, Streaming, Historical +} + +func (x *GetStatisticsRequest) Reset() { + *x = GetStatisticsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetStatisticsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetStatisticsRequest) ProtoMessage() {} + +func (x *GetStatisticsRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[6] + 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 GetStatisticsRequest.ProtoReflect.Descriptor instead. +func (*GetStatisticsRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{6} +} + +func (x *GetStatisticsRequest) GetReq() *internalpb.GetStatisticsRequest { + if x != nil { + return x.Req + } + return nil +} + +func (x *GetStatisticsRequest) GetDmlChannels() []string { + if x != nil { + return x.DmlChannels + } + return nil +} + +func (x *GetStatisticsRequest) GetSegmentIDs() []int64 { + if x != nil { + return x.SegmentIDs + } + return nil +} + +func (x *GetStatisticsRequest) GetFromShardLeader() bool { + if x != nil { + return x.FromShardLeader + } + return false +} + +func (x *GetStatisticsRequest) GetScope() DataScope { + if x != nil { + return x.Scope + } + return DataScope_UnKnown +} + +type LoadPartitionsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + DbID int64 `protobuf:"varint,2,opt,name=dbID,proto3" json:"dbID,omitempty"` + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionIDs []int64 `protobuf:"varint,4,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` + Schema *schemapb.CollectionSchema `protobuf:"bytes,5,opt,name=schema,proto3" json:"schema,omitempty"` + ReplicaNumber int32 `protobuf:"varint,6,opt,name=replica_number,json=replicaNumber,proto3" json:"replica_number,omitempty"` + // fieldID -> indexID + FieldIndexID map[int64]int64 `protobuf:"bytes,7,rep,name=field_indexID,json=fieldIndexID,proto3" json:"field_indexID,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + Refresh bool `protobuf:"varint,8,opt,name=refresh,proto3" json:"refresh,omitempty"` + // resource group names + ResourceGroups []string `protobuf:"bytes,9,rep,name=resource_groups,json=resourceGroups,proto3" json:"resource_groups,omitempty"` + IndexInfoList []*indexpb.IndexInfo `protobuf:"bytes,10,rep,name=index_info_list,json=indexInfoList,proto3" json:"index_info_list,omitempty"` + LoadFields []int64 `protobuf:"varint,11,rep,packed,name=load_fields,json=loadFields,proto3" json:"load_fields,omitempty"` +} + +func (x *LoadPartitionsRequest) Reset() { + *x = LoadPartitionsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *LoadPartitionsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LoadPartitionsRequest) ProtoMessage() {} + +func (x *LoadPartitionsRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[7] + 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 LoadPartitionsRequest.ProtoReflect.Descriptor instead. +func (*LoadPartitionsRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{7} +} + +func (x *LoadPartitionsRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *LoadPartitionsRequest) GetDbID() int64 { + if x != nil { + return x.DbID + } + return 0 +} + +func (x *LoadPartitionsRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *LoadPartitionsRequest) GetPartitionIDs() []int64 { + if x != nil { + return x.PartitionIDs + } + return nil +} + +func (x *LoadPartitionsRequest) GetSchema() *schemapb.CollectionSchema { + if x != nil { + return x.Schema + } + return nil +} + +func (x *LoadPartitionsRequest) GetReplicaNumber() int32 { + if x != nil { + return x.ReplicaNumber + } + return 0 +} + +func (x *LoadPartitionsRequest) GetFieldIndexID() map[int64]int64 { + if x != nil { + return x.FieldIndexID + } + return nil +} + +func (x *LoadPartitionsRequest) GetRefresh() bool { + if x != nil { + return x.Refresh + } + return false +} + +func (x *LoadPartitionsRequest) GetResourceGroups() []string { + if x != nil { + return x.ResourceGroups + } + return nil +} + +func (x *LoadPartitionsRequest) GetIndexInfoList() []*indexpb.IndexInfo { + if x != nil { + return x.IndexInfoList + } + return nil +} + +func (x *LoadPartitionsRequest) GetLoadFields() []int64 { + if x != nil { + return x.LoadFields + } + return nil +} + +type ReleasePartitionsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + DbID int64 `protobuf:"varint,2,opt,name=dbID,proto3" json:"dbID,omitempty"` + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionIDs []int64 `protobuf:"varint,4,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` + NodeID int64 `protobuf:"varint,5,opt,name=nodeID,proto3" json:"nodeID,omitempty"` +} + +func (x *ReleasePartitionsRequest) Reset() { + *x = ReleasePartitionsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ReleasePartitionsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ReleasePartitionsRequest) ProtoMessage() {} + +func (x *ReleasePartitionsRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[8] + 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 ReleasePartitionsRequest.ProtoReflect.Descriptor instead. +func (*ReleasePartitionsRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{8} +} + +func (x *ReleasePartitionsRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *ReleasePartitionsRequest) GetDbID() int64 { + if x != nil { + return x.DbID + } + return 0 +} + +func (x *ReleasePartitionsRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *ReleasePartitionsRequest) GetPartitionIDs() []int64 { + if x != nil { + return x.PartitionIDs + } + return nil +} + +func (x *ReleasePartitionsRequest) GetNodeID() int64 { + if x != nil { + return x.NodeID + } + return 0 +} + +type GetPartitionStatesRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + DbID int64 `protobuf:"varint,2,opt,name=dbID,proto3" json:"dbID,omitempty"` + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionIDs []int64 `protobuf:"varint,4,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` +} + +func (x *GetPartitionStatesRequest) Reset() { + *x = GetPartitionStatesRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetPartitionStatesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetPartitionStatesRequest) ProtoMessage() {} + +func (x *GetPartitionStatesRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[9] + 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 GetPartitionStatesRequest.ProtoReflect.Descriptor instead. +func (*GetPartitionStatesRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{9} +} + +func (x *GetPartitionStatesRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *GetPartitionStatesRequest) GetDbID() int64 { + if x != nil { + return x.DbID + } + return 0 +} + +func (x *GetPartitionStatesRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *GetPartitionStatesRequest) GetPartitionIDs() []int64 { + if x != nil { + return x.PartitionIDs + } + return nil +} + +type GetPartitionStatesResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + PartitionDescriptions []*PartitionStates `protobuf:"bytes,2,rep,name=partition_descriptions,json=partitionDescriptions,proto3" json:"partition_descriptions,omitempty"` +} + +func (x *GetPartitionStatesResponse) Reset() { + *x = GetPartitionStatesResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetPartitionStatesResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetPartitionStatesResponse) ProtoMessage() {} + +func (x *GetPartitionStatesResponse) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[10] + 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 GetPartitionStatesResponse.ProtoReflect.Descriptor instead. +func (*GetPartitionStatesResponse) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{10} +} + +func (x *GetPartitionStatesResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *GetPartitionStatesResponse) GetPartitionDescriptions() []*PartitionStates { + if x != nil { + return x.PartitionDescriptions + } + return nil +} + +type GetSegmentInfoRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + SegmentIDs []int64 `protobuf:"varint,2,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,omitempty"` // deprecated + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` +} + +func (x *GetSegmentInfoRequest) Reset() { + *x = GetSegmentInfoRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetSegmentInfoRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetSegmentInfoRequest) ProtoMessage() {} + +func (x *GetSegmentInfoRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[11] + 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 GetSegmentInfoRequest.ProtoReflect.Descriptor instead. +func (*GetSegmentInfoRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{11} +} + +func (x *GetSegmentInfoRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *GetSegmentInfoRequest) GetSegmentIDs() []int64 { + if x != nil { + return x.SegmentIDs + } + return nil +} + +func (x *GetSegmentInfoRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +type GetSegmentInfoResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + Infos []*SegmentInfo `protobuf:"bytes,2,rep,name=infos,proto3" json:"infos,omitempty"` +} + +func (x *GetSegmentInfoResponse) Reset() { + *x = GetSegmentInfoResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetSegmentInfoResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetSegmentInfoResponse) ProtoMessage() {} + +func (x *GetSegmentInfoResponse) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[12] + 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 GetSegmentInfoResponse.ProtoReflect.Descriptor instead. +func (*GetSegmentInfoResponse) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{12} +} + +func (x *GetSegmentInfoResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *GetSegmentInfoResponse) GetInfos() []*SegmentInfo { + if x != nil { + return x.Infos + } + return nil +} + +type GetShardLeadersRequest 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"` +} + +func (x *GetShardLeadersRequest) Reset() { + *x = GetShardLeadersRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetShardLeadersRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetShardLeadersRequest) ProtoMessage() {} + +func (x *GetShardLeadersRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[13] + 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 GetShardLeadersRequest.ProtoReflect.Descriptor instead. +func (*GetShardLeadersRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{13} +} + +func (x *GetShardLeadersRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *GetShardLeadersRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +type GetShardLeadersResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + Shards []*ShardLeadersList `protobuf:"bytes,2,rep,name=shards,proto3" json:"shards,omitempty"` +} + +func (x *GetShardLeadersResponse) Reset() { + *x = GetShardLeadersResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetShardLeadersResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetShardLeadersResponse) ProtoMessage() {} + +func (x *GetShardLeadersResponse) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[14] + 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 GetShardLeadersResponse.ProtoReflect.Descriptor instead. +func (*GetShardLeadersResponse) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{14} +} + +func (x *GetShardLeadersResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *GetShardLeadersResponse) GetShards() []*ShardLeadersList { + if x != nil { + return x.Shards + } + return nil +} + +type UpdateResourceGroupsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + ResourceGroups map[string]*rgpb.ResourceGroupConfig `protobuf:"bytes,2,rep,name=resource_groups,json=resourceGroups,proto3" json:"resource_groups,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *UpdateResourceGroupsRequest) Reset() { + *x = UpdateResourceGroupsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateResourceGroupsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateResourceGroupsRequest) ProtoMessage() {} + +func (x *UpdateResourceGroupsRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[15] + 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 UpdateResourceGroupsRequest.ProtoReflect.Descriptor instead. +func (*UpdateResourceGroupsRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{15} +} + +func (x *UpdateResourceGroupsRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *UpdateResourceGroupsRequest) GetResourceGroups() map[string]*rgpb.ResourceGroupConfig { + if x != nil { + return x.ResourceGroups + } + return nil +} + +type ShardLeadersList struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ChannelName string `protobuf:"bytes,1,opt,name=channel_name,json=channelName,proto3" json:"channel_name,omitempty"` + NodeIds []int64 `protobuf:"varint,2,rep,packed,name=node_ids,json=nodeIds,proto3" json:"node_ids,omitempty"` + NodeAddrs []string `protobuf:"bytes,3,rep,name=node_addrs,json=nodeAddrs,proto3" json:"node_addrs,omitempty"` +} + +func (x *ShardLeadersList) Reset() { + *x = ShardLeadersList{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ShardLeadersList) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ShardLeadersList) ProtoMessage() {} + +func (x *ShardLeadersList) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[16] + 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 ShardLeadersList.ProtoReflect.Descriptor instead. +func (*ShardLeadersList) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{16} +} + +func (x *ShardLeadersList) GetChannelName() string { + if x != nil { + return x.ChannelName + } + return "" +} + +func (x *ShardLeadersList) GetNodeIds() []int64 { + if x != nil { + return x.NodeIds + } + return nil +} + +func (x *ShardLeadersList) GetNodeAddrs() []string { + if x != nil { + return x.NodeAddrs + } + return nil +} + +type SyncNewCreatedPartitionRequest 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"` + PartitionID int64 `protobuf:"varint,3,opt,name=partitionID,proto3" json:"partitionID,omitempty"` +} + +func (x *SyncNewCreatedPartitionRequest) Reset() { + *x = SyncNewCreatedPartitionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SyncNewCreatedPartitionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SyncNewCreatedPartitionRequest) ProtoMessage() {} + +func (x *SyncNewCreatedPartitionRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[17] + 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 SyncNewCreatedPartitionRequest.ProtoReflect.Descriptor instead. +func (*SyncNewCreatedPartitionRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{17} +} + +func (x *SyncNewCreatedPartitionRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *SyncNewCreatedPartitionRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *SyncNewCreatedPartitionRequest) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +type LoadMetaInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + LoadType LoadType `protobuf:"varint,1,opt,name=load_type,json=loadType,proto3,enum=milvus.proto.query.LoadType" json:"load_type,omitempty"` + CollectionID int64 `protobuf:"varint,2,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionIDs []int64 `protobuf:"varint,3,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` + // Deprecated: Marked as deprecated in query_coord.proto. + MetricType string `protobuf:"bytes,4,opt,name=metric_type,json=metricType,proto3" json:"metric_type,omitempty"` + DbName string `protobuf:"bytes,5,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` // Only used for metrics label. + ResourceGroup string `protobuf:"bytes,6,opt,name=resource_group,json=resourceGroup,proto3" json:"resource_group,omitempty"` // Only used for metrics label. + LoadFields []int64 `protobuf:"varint,7,rep,packed,name=load_fields,json=loadFields,proto3" json:"load_fields,omitempty"` + DbProperties []*commonpb.KeyValuePair `protobuf:"bytes,8,rep,name=db_properties,json=dbProperties,proto3" json:"db_properties,omitempty"` +} + +func (x *LoadMetaInfo) Reset() { + *x = LoadMetaInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *LoadMetaInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LoadMetaInfo) ProtoMessage() {} + +func (x *LoadMetaInfo) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[18] + 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 LoadMetaInfo.ProtoReflect.Descriptor instead. +func (*LoadMetaInfo) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{18} +} + +func (x *LoadMetaInfo) GetLoadType() LoadType { + if x != nil { + return x.LoadType + } + return LoadType_UnKnownType +} + +func (x *LoadMetaInfo) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *LoadMetaInfo) GetPartitionIDs() []int64 { + if x != nil { + return x.PartitionIDs + } + return nil +} + +// Deprecated: Marked as deprecated in query_coord.proto. +func (x *LoadMetaInfo) GetMetricType() string { + if x != nil { + return x.MetricType + } + return "" +} + +func (x *LoadMetaInfo) GetDbName() string { + if x != nil { + return x.DbName + } + return "" +} + +func (x *LoadMetaInfo) GetResourceGroup() string { + if x != nil { + return x.ResourceGroup + } + return "" +} + +func (x *LoadMetaInfo) GetLoadFields() []int64 { + if x != nil { + return x.LoadFields + } + return nil +} + +func (x *LoadMetaInfo) GetDbProperties() []*commonpb.KeyValuePair { + if x != nil { + return x.DbProperties + } + return nil +} + +type WatchDmChannelsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + NodeID int64 `protobuf:"varint,2,opt,name=nodeID,proto3" json:"nodeID,omitempty"` + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionIDs []int64 `protobuf:"varint,4,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` + Infos []*datapb.VchannelInfo `protobuf:"bytes,5,rep,name=infos,proto3" json:"infos,omitempty"` + Schema *schemapb.CollectionSchema `protobuf:"bytes,6,opt,name=schema,proto3" json:"schema,omitempty"` + ExcludeInfos []*datapb.SegmentInfo `protobuf:"bytes,7,rep,name=exclude_infos,json=excludeInfos,proto3" json:"exclude_infos,omitempty"` + LoadMeta *LoadMetaInfo `protobuf:"bytes,8,opt,name=load_meta,json=loadMeta,proto3" json:"load_meta,omitempty"` + ReplicaID int64 `protobuf:"varint,9,opt,name=replicaID,proto3" json:"replicaID,omitempty"` + SegmentInfos map[int64]*datapb.SegmentInfo `protobuf:"bytes,10,rep,name=segment_infos,json=segmentInfos,proto3" json:"segment_infos,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + // Deprecated + // for node down load balance, need to remove offline node in time after every watchDmChannel finish. + OfflineNodeID int64 `protobuf:"varint,11,opt,name=offlineNodeID,proto3" json:"offlineNodeID,omitempty"` + Version int64 `protobuf:"varint,12,opt,name=version,proto3" json:"version,omitempty"` + IndexInfoList []*indexpb.IndexInfo `protobuf:"bytes,13,rep,name=index_info_list,json=indexInfoList,proto3" json:"index_info_list,omitempty"` +} + +func (x *WatchDmChannelsRequest) Reset() { + *x = WatchDmChannelsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[19] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WatchDmChannelsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WatchDmChannelsRequest) ProtoMessage() {} + +func (x *WatchDmChannelsRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[19] + 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 WatchDmChannelsRequest.ProtoReflect.Descriptor instead. +func (*WatchDmChannelsRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{19} +} + +func (x *WatchDmChannelsRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *WatchDmChannelsRequest) GetNodeID() int64 { + if x != nil { + return x.NodeID + } + return 0 +} + +func (x *WatchDmChannelsRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *WatchDmChannelsRequest) GetPartitionIDs() []int64 { + if x != nil { + return x.PartitionIDs + } + return nil +} + +func (x *WatchDmChannelsRequest) GetInfos() []*datapb.VchannelInfo { + if x != nil { + return x.Infos + } + return nil +} + +func (x *WatchDmChannelsRequest) GetSchema() *schemapb.CollectionSchema { + if x != nil { + return x.Schema + } + return nil +} + +func (x *WatchDmChannelsRequest) GetExcludeInfos() []*datapb.SegmentInfo { + if x != nil { + return x.ExcludeInfos + } + return nil +} + +func (x *WatchDmChannelsRequest) GetLoadMeta() *LoadMetaInfo { + if x != nil { + return x.LoadMeta + } + return nil +} + +func (x *WatchDmChannelsRequest) GetReplicaID() int64 { + if x != nil { + return x.ReplicaID + } + return 0 +} + +func (x *WatchDmChannelsRequest) GetSegmentInfos() map[int64]*datapb.SegmentInfo { + if x != nil { + return x.SegmentInfos + } + return nil +} + +func (x *WatchDmChannelsRequest) GetOfflineNodeID() int64 { + if x != nil { + return x.OfflineNodeID + } + return 0 +} + +func (x *WatchDmChannelsRequest) GetVersion() int64 { + if x != nil { + return x.Version + } + return 0 +} + +func (x *WatchDmChannelsRequest) GetIndexInfoList() []*indexpb.IndexInfo { + if x != nil { + return x.IndexInfoList + } + return nil +} + +type UnsubDmChannelRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + NodeID int64 `protobuf:"varint,2,opt,name=nodeID,proto3" json:"nodeID,omitempty"` + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + ChannelName string `protobuf:"bytes,4,opt,name=channel_name,json=channelName,proto3" json:"channel_name,omitempty"` +} + +func (x *UnsubDmChannelRequest) Reset() { + *x = UnsubDmChannelRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[20] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UnsubDmChannelRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UnsubDmChannelRequest) ProtoMessage() {} + +func (x *UnsubDmChannelRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[20] + 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 UnsubDmChannelRequest.ProtoReflect.Descriptor instead. +func (*UnsubDmChannelRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{20} +} + +func (x *UnsubDmChannelRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *UnsubDmChannelRequest) GetNodeID() int64 { + if x != nil { + return x.NodeID + } + return 0 +} + +func (x *UnsubDmChannelRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *UnsubDmChannelRequest) GetChannelName() string { + if x != nil { + return x.ChannelName + } + return "" +} + +type SegmentLoadInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SegmentID int64 `protobuf:"varint,1,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + PartitionID int64 `protobuf:"varint,2,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + DbID int64 `protobuf:"varint,4,opt,name=dbID,proto3" json:"dbID,omitempty"` + FlushTime int64 `protobuf:"varint,5,opt,name=flush_time,json=flushTime,proto3" json:"flush_time,omitempty"` + BinlogPaths []*datapb.FieldBinlog `protobuf:"bytes,6,rep,name=binlog_paths,json=binlogPaths,proto3" json:"binlog_paths,omitempty"` + NumOfRows int64 `protobuf:"varint,7,opt,name=num_of_rows,json=numOfRows,proto3" json:"num_of_rows,omitempty"` + Statslogs []*datapb.FieldBinlog `protobuf:"bytes,8,rep,name=statslogs,proto3" json:"statslogs,omitempty"` + Deltalogs []*datapb.FieldBinlog `protobuf:"bytes,9,rep,name=deltalogs,proto3" json:"deltalogs,omitempty"` + CompactionFrom []int64 `protobuf:"varint,10,rep,packed,name=compactionFrom,proto3" json:"compactionFrom,omitempty"` // segmentIDs compacted from + IndexInfos []*FieldIndexInfo `protobuf:"bytes,11,rep,name=index_infos,json=indexInfos,proto3" json:"index_infos,omitempty"` + // Deprecated: Marked as deprecated in query_coord.proto. + SegmentSize int64 `protobuf:"varint,12,opt,name=segment_size,json=segmentSize,proto3" json:"segment_size,omitempty"` + InsertChannel string `protobuf:"bytes,13,opt,name=insert_channel,json=insertChannel,proto3" json:"insert_channel,omitempty"` + StartPosition *msgpb.MsgPosition `protobuf:"bytes,14,opt,name=start_position,json=startPosition,proto3" json:"start_position,omitempty"` + DeltaPosition *msgpb.MsgPosition `protobuf:"bytes,15,opt,name=delta_position,json=deltaPosition,proto3" json:"delta_position,omitempty"` + ReadableVersion int64 `protobuf:"varint,16,opt,name=readableVersion,proto3" json:"readableVersion,omitempty"` + Level datapb.SegmentLevel `protobuf:"varint,17,opt,name=level,proto3,enum=milvus.proto.data.SegmentLevel" json:"level,omitempty"` + StorageVersion int64 `protobuf:"varint,18,opt,name=storageVersion,proto3" json:"storageVersion,omitempty"` + IsSorted bool `protobuf:"varint,19,opt,name=is_sorted,json=isSorted,proto3" json:"is_sorted,omitempty"` + TextStatsLogs map[int64]*datapb.TextIndexStats `protobuf:"bytes,20,rep,name=textStatsLogs,proto3" json:"textStatsLogs,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Bm25Logs []*datapb.FieldBinlog `protobuf:"bytes,21,rep,name=bm25logs,proto3" json:"bm25logs,omitempty"` +} + +func (x *SegmentLoadInfo) Reset() { + *x = SegmentLoadInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[21] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentLoadInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentLoadInfo) ProtoMessage() {} + +func (x *SegmentLoadInfo) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[21] + 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 SegmentLoadInfo.ProtoReflect.Descriptor instead. +func (*SegmentLoadInfo) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{21} +} + +func (x *SegmentLoadInfo) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *SegmentLoadInfo) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *SegmentLoadInfo) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *SegmentLoadInfo) GetDbID() int64 { + if x != nil { + return x.DbID + } + return 0 +} + +func (x *SegmentLoadInfo) GetFlushTime() int64 { + if x != nil { + return x.FlushTime + } + return 0 +} + +func (x *SegmentLoadInfo) GetBinlogPaths() []*datapb.FieldBinlog { + if x != nil { + return x.BinlogPaths + } + return nil +} + +func (x *SegmentLoadInfo) GetNumOfRows() int64 { + if x != nil { + return x.NumOfRows + } + return 0 +} + +func (x *SegmentLoadInfo) GetStatslogs() []*datapb.FieldBinlog { + if x != nil { + return x.Statslogs + } + return nil +} + +func (x *SegmentLoadInfo) GetDeltalogs() []*datapb.FieldBinlog { + if x != nil { + return x.Deltalogs + } + return nil +} + +func (x *SegmentLoadInfo) GetCompactionFrom() []int64 { + if x != nil { + return x.CompactionFrom + } + return nil +} + +func (x *SegmentLoadInfo) GetIndexInfos() []*FieldIndexInfo { + if x != nil { + return x.IndexInfos + } + return nil +} + +// Deprecated: Marked as deprecated in query_coord.proto. +func (x *SegmentLoadInfo) GetSegmentSize() int64 { + if x != nil { + return x.SegmentSize + } + return 0 +} + +func (x *SegmentLoadInfo) GetInsertChannel() string { + if x != nil { + return x.InsertChannel + } + return "" +} + +func (x *SegmentLoadInfo) GetStartPosition() *msgpb.MsgPosition { + if x != nil { + return x.StartPosition + } + return nil +} + +func (x *SegmentLoadInfo) GetDeltaPosition() *msgpb.MsgPosition { + if x != nil { + return x.DeltaPosition + } + return nil +} + +func (x *SegmentLoadInfo) GetReadableVersion() int64 { + if x != nil { + return x.ReadableVersion + } + return 0 +} + +func (x *SegmentLoadInfo) GetLevel() datapb.SegmentLevel { + if x != nil { + return x.Level + } + return datapb.SegmentLevel(0) +} + +func (x *SegmentLoadInfo) GetStorageVersion() int64 { + if x != nil { + return x.StorageVersion + } + return 0 +} + +func (x *SegmentLoadInfo) GetIsSorted() bool { + if x != nil { + return x.IsSorted + } + return false +} + +func (x *SegmentLoadInfo) GetTextStatsLogs() map[int64]*datapb.TextIndexStats { + if x != nil { + return x.TextStatsLogs + } + return nil +} + +func (x *SegmentLoadInfo) GetBm25Logs() []*datapb.FieldBinlog { + if x != nil { + return x.Bm25Logs + } + return nil +} + +type FieldIndexInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + FieldID int64 `protobuf:"varint,1,opt,name=fieldID,proto3" json:"fieldID,omitempty"` + // deprecated + EnableIndex bool `protobuf:"varint,2,opt,name=enable_index,json=enableIndex,proto3" json:"enable_index,omitempty"` + IndexName string `protobuf:"bytes,3,opt,name=index_name,json=indexName,proto3" json:"index_name,omitempty"` + IndexID int64 `protobuf:"varint,4,opt,name=indexID,proto3" json:"indexID,omitempty"` + BuildID int64 `protobuf:"varint,5,opt,name=buildID,proto3" json:"buildID,omitempty"` + IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,6,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"` + IndexFilePaths []string `protobuf:"bytes,7,rep,name=index_file_paths,json=indexFilePaths,proto3" json:"index_file_paths,omitempty"` + IndexSize int64 `protobuf:"varint,8,opt,name=index_size,json=indexSize,proto3" json:"index_size,omitempty"` + IndexVersion int64 `protobuf:"varint,9,opt,name=index_version,json=indexVersion,proto3" json:"index_version,omitempty"` + NumRows int64 `protobuf:"varint,10,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"` + CurrentIndexVersion int32 `protobuf:"varint,11,opt,name=current_index_version,json=currentIndexVersion,proto3" json:"current_index_version,omitempty"` + IndexStoreVersion int64 `protobuf:"varint,12,opt,name=index_store_version,json=indexStoreVersion,proto3" json:"index_store_version,omitempty"` +} + +func (x *FieldIndexInfo) Reset() { + *x = FieldIndexInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[22] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *FieldIndexInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FieldIndexInfo) ProtoMessage() {} + +func (x *FieldIndexInfo) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[22] + 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 FieldIndexInfo.ProtoReflect.Descriptor instead. +func (*FieldIndexInfo) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{22} +} + +func (x *FieldIndexInfo) GetFieldID() int64 { + if x != nil { + return x.FieldID + } + return 0 +} + +func (x *FieldIndexInfo) GetEnableIndex() bool { + if x != nil { + return x.EnableIndex + } + return false +} + +func (x *FieldIndexInfo) GetIndexName() string { + if x != nil { + return x.IndexName + } + return "" +} + +func (x *FieldIndexInfo) GetIndexID() int64 { + if x != nil { + return x.IndexID + } + return 0 +} + +func (x *FieldIndexInfo) GetBuildID() int64 { + if x != nil { + return x.BuildID + } + return 0 +} + +func (x *FieldIndexInfo) GetIndexParams() []*commonpb.KeyValuePair { + if x != nil { + return x.IndexParams + } + return nil +} + +func (x *FieldIndexInfo) GetIndexFilePaths() []string { + if x != nil { + return x.IndexFilePaths + } + return nil +} + +func (x *FieldIndexInfo) GetIndexSize() int64 { + if x != nil { + return x.IndexSize + } + return 0 +} + +func (x *FieldIndexInfo) GetIndexVersion() int64 { + if x != nil { + return x.IndexVersion + } + return 0 +} + +func (x *FieldIndexInfo) GetNumRows() int64 { + if x != nil { + return x.NumRows + } + return 0 +} + +func (x *FieldIndexInfo) GetCurrentIndexVersion() int32 { + if x != nil { + return x.CurrentIndexVersion + } + return 0 +} + +func (x *FieldIndexInfo) GetIndexStoreVersion() int64 { + if x != nil { + return x.IndexStoreVersion + } + return 0 +} + +type LoadSegmentsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + DstNodeID int64 `protobuf:"varint,2,opt,name=dst_nodeID,json=dstNodeID,proto3" json:"dst_nodeID,omitempty"` + Infos []*SegmentLoadInfo `protobuf:"bytes,3,rep,name=infos,proto3" json:"infos,omitempty"` + Schema *schemapb.CollectionSchema `protobuf:"bytes,4,opt,name=schema,proto3" json:"schema,omitempty"` + SourceNodeID int64 `protobuf:"varint,5,opt,name=source_nodeID,json=sourceNodeID,proto3" json:"source_nodeID,omitempty"` + CollectionID int64 `protobuf:"varint,6,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + LoadMeta *LoadMetaInfo `protobuf:"bytes,7,opt,name=load_meta,json=loadMeta,proto3" json:"load_meta,omitempty"` + ReplicaID int64 `protobuf:"varint,8,opt,name=replicaID,proto3" json:"replicaID,omitempty"` + DeltaPositions []*msgpb.MsgPosition `protobuf:"bytes,9,rep,name=delta_positions,json=deltaPositions,proto3" json:"delta_positions,omitempty"` // keep it for compatibility of rolling upgrade from 2.2.x to 2.3 + Version int64 `protobuf:"varint,10,opt,name=version,proto3" json:"version,omitempty"` + NeedTransfer bool `protobuf:"varint,11,opt,name=need_transfer,json=needTransfer,proto3" json:"need_transfer,omitempty"` + LoadScope LoadScope `protobuf:"varint,12,opt,name=load_scope,json=loadScope,proto3,enum=milvus.proto.query.LoadScope" json:"load_scope,omitempty"` + IndexInfoList []*indexpb.IndexInfo `protobuf:"bytes,13,rep,name=index_info_list,json=indexInfoList,proto3" json:"index_info_list,omitempty"` + LazyLoad bool `protobuf:"varint,14,opt,name=lazy_load,json=lazyLoad,proto3" json:"lazy_load,omitempty"` +} + +func (x *LoadSegmentsRequest) Reset() { + *x = LoadSegmentsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[23] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *LoadSegmentsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LoadSegmentsRequest) ProtoMessage() {} + +func (x *LoadSegmentsRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[23] + 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 LoadSegmentsRequest.ProtoReflect.Descriptor instead. +func (*LoadSegmentsRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{23} +} + +func (x *LoadSegmentsRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *LoadSegmentsRequest) GetDstNodeID() int64 { + if x != nil { + return x.DstNodeID + } + return 0 +} + +func (x *LoadSegmentsRequest) GetInfos() []*SegmentLoadInfo { + if x != nil { + return x.Infos + } + return nil +} + +func (x *LoadSegmentsRequest) GetSchema() *schemapb.CollectionSchema { + if x != nil { + return x.Schema + } + return nil +} + +func (x *LoadSegmentsRequest) GetSourceNodeID() int64 { + if x != nil { + return x.SourceNodeID + } + return 0 +} + +func (x *LoadSegmentsRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *LoadSegmentsRequest) GetLoadMeta() *LoadMetaInfo { + if x != nil { + return x.LoadMeta + } + return nil +} + +func (x *LoadSegmentsRequest) GetReplicaID() int64 { + if x != nil { + return x.ReplicaID + } + return 0 +} + +func (x *LoadSegmentsRequest) GetDeltaPositions() []*msgpb.MsgPosition { + if x != nil { + return x.DeltaPositions + } + return nil +} + +func (x *LoadSegmentsRequest) GetVersion() int64 { + if x != nil { + return x.Version + } + return 0 +} + +func (x *LoadSegmentsRequest) GetNeedTransfer() bool { + if x != nil { + return x.NeedTransfer + } + return false +} + +func (x *LoadSegmentsRequest) GetLoadScope() LoadScope { + if x != nil { + return x.LoadScope + } + return LoadScope_Full +} + +func (x *LoadSegmentsRequest) GetIndexInfoList() []*indexpb.IndexInfo { + if x != nil { + return x.IndexInfoList + } + return nil +} + +func (x *LoadSegmentsRequest) GetLazyLoad() bool { + if x != nil { + return x.LazyLoad + } + return false +} + +type ReleaseSegmentsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + NodeID int64 `protobuf:"varint,2,opt,name=nodeID,proto3" json:"nodeID,omitempty"` + // Not useful for now + DbID int64 `protobuf:"varint,3,opt,name=dbID,proto3" json:"dbID,omitempty"` + CollectionID int64 `protobuf:"varint,4,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionIDs []int64 `protobuf:"varint,5,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` + SegmentIDs []int64 `protobuf:"varint,6,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,omitempty"` + Scope DataScope `protobuf:"varint,7,opt,name=scope,proto3,enum=milvus.proto.query.DataScope" json:"scope,omitempty"` // All, Streaming, Historical + Shard string `protobuf:"bytes,8,opt,name=shard,proto3" json:"shard,omitempty"` + NeedTransfer bool `protobuf:"varint,11,opt,name=need_transfer,json=needTransfer,proto3" json:"need_transfer,omitempty"` + Checkpoint *msgpb.MsgPosition `protobuf:"bytes,12,opt,name=checkpoint,proto3" json:"checkpoint,omitempty"` // channel's check point +} + +func (x *ReleaseSegmentsRequest) Reset() { + *x = ReleaseSegmentsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[24] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ReleaseSegmentsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ReleaseSegmentsRequest) ProtoMessage() {} + +func (x *ReleaseSegmentsRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[24] + 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 ReleaseSegmentsRequest.ProtoReflect.Descriptor instead. +func (*ReleaseSegmentsRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{24} +} + +func (x *ReleaseSegmentsRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *ReleaseSegmentsRequest) GetNodeID() int64 { + if x != nil { + return x.NodeID + } + return 0 +} + +func (x *ReleaseSegmentsRequest) GetDbID() int64 { + if x != nil { + return x.DbID + } + return 0 +} + +func (x *ReleaseSegmentsRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *ReleaseSegmentsRequest) GetPartitionIDs() []int64 { + if x != nil { + return x.PartitionIDs + } + return nil +} + +func (x *ReleaseSegmentsRequest) GetSegmentIDs() []int64 { + if x != nil { + return x.SegmentIDs + } + return nil +} + +func (x *ReleaseSegmentsRequest) GetScope() DataScope { + if x != nil { + return x.Scope + } + return DataScope_UnKnown +} + +func (x *ReleaseSegmentsRequest) GetShard() string { + if x != nil { + return x.Shard + } + return "" +} + +func (x *ReleaseSegmentsRequest) GetNeedTransfer() bool { + if x != nil { + return x.NeedTransfer + } + return false +} + +func (x *ReleaseSegmentsRequest) GetCheckpoint() *msgpb.MsgPosition { + if x != nil { + return x.Checkpoint + } + return nil +} + +type SearchRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Req *internalpb.SearchRequest `protobuf:"bytes,1,opt,name=req,proto3" json:"req,omitempty"` + DmlChannels []string `protobuf:"bytes,2,rep,name=dml_channels,json=dmlChannels,proto3" json:"dml_channels,omitempty"` + SegmentIDs []int64 `protobuf:"varint,3,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,omitempty"` + FromShardLeader bool `protobuf:"varint,4,opt,name=from_shard_leader,json=fromShardLeader,proto3" json:"from_shard_leader,omitempty"` + Scope DataScope `protobuf:"varint,5,opt,name=scope,proto3,enum=milvus.proto.query.DataScope" json:"scope,omitempty"` // All, Streaming, Historical + TotalChannelNum int32 `protobuf:"varint,6,opt,name=total_channel_num,json=totalChannelNum,proto3" json:"total_channel_num,omitempty"` +} + +func (x *SearchRequest) Reset() { + *x = SearchRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[25] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SearchRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SearchRequest) ProtoMessage() {} + +func (x *SearchRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[25] + 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 SearchRequest.ProtoReflect.Descriptor instead. +func (*SearchRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{25} +} + +func (x *SearchRequest) GetReq() *internalpb.SearchRequest { + if x != nil { + return x.Req + } + return nil +} + +func (x *SearchRequest) GetDmlChannels() []string { + if x != nil { + return x.DmlChannels + } + return nil +} + +func (x *SearchRequest) GetSegmentIDs() []int64 { + if x != nil { + return x.SegmentIDs + } + return nil +} + +func (x *SearchRequest) GetFromShardLeader() bool { + if x != nil { + return x.FromShardLeader + } + return false +} + +func (x *SearchRequest) GetScope() DataScope { + if x != nil { + return x.Scope + } + return DataScope_UnKnown +} + +func (x *SearchRequest) GetTotalChannelNum() int32 { + if x != nil { + return x.TotalChannelNum + } + return 0 +} + +type QueryRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Req *internalpb.RetrieveRequest `protobuf:"bytes,1,opt,name=req,proto3" json:"req,omitempty"` + DmlChannels []string `protobuf:"bytes,2,rep,name=dml_channels,json=dmlChannels,proto3" json:"dml_channels,omitempty"` + SegmentIDs []int64 `protobuf:"varint,3,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,omitempty"` + FromShardLeader bool `protobuf:"varint,4,opt,name=from_shard_leader,json=fromShardLeader,proto3" json:"from_shard_leader,omitempty"` + Scope DataScope `protobuf:"varint,5,opt,name=scope,proto3,enum=milvus.proto.query.DataScope" json:"scope,omitempty"` // All, Streaming, Historical +} + +func (x *QueryRequest) Reset() { + *x = QueryRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[26] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *QueryRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryRequest) ProtoMessage() {} + +func (x *QueryRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[26] + 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 QueryRequest.ProtoReflect.Descriptor instead. +func (*QueryRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{26} +} + +func (x *QueryRequest) GetReq() *internalpb.RetrieveRequest { + if x != nil { + return x.Req + } + return nil +} + +func (x *QueryRequest) GetDmlChannels() []string { + if x != nil { + return x.DmlChannels + } + return nil +} + +func (x *QueryRequest) GetSegmentIDs() []int64 { + if x != nil { + return x.SegmentIDs + } + return nil +} + +func (x *QueryRequest) GetFromShardLeader() bool { + if x != nil { + return x.FromShardLeader + } + return false +} + +func (x *QueryRequest) GetScope() DataScope { + if x != nil { + return x.Scope + } + return DataScope_UnKnown +} + +type SyncReplicaSegmentsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + VchannelName string `protobuf:"bytes,2,opt,name=vchannel_name,json=vchannelName,proto3" json:"vchannel_name,omitempty"` + ReplicaSegments []*ReplicaSegmentsInfo `protobuf:"bytes,3,rep,name=replica_segments,json=replicaSegments,proto3" json:"replica_segments,omitempty"` +} + +func (x *SyncReplicaSegmentsRequest) Reset() { + *x = SyncReplicaSegmentsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[27] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SyncReplicaSegmentsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SyncReplicaSegmentsRequest) ProtoMessage() {} + +func (x *SyncReplicaSegmentsRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[27] + 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 SyncReplicaSegmentsRequest.ProtoReflect.Descriptor instead. +func (*SyncReplicaSegmentsRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{27} +} + +func (x *SyncReplicaSegmentsRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *SyncReplicaSegmentsRequest) GetVchannelName() string { + if x != nil { + return x.VchannelName + } + return "" +} + +func (x *SyncReplicaSegmentsRequest) GetReplicaSegments() []*ReplicaSegmentsInfo { + if x != nil { + return x.ReplicaSegments + } + return nil +} + +type ReplicaSegmentsInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + NodeId int64 `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + PartitionId int64 `protobuf:"varint,2,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"` + SegmentIds []int64 `protobuf:"varint,3,rep,packed,name=segment_ids,json=segmentIds,proto3" json:"segment_ids,omitempty"` + Versions []int64 `protobuf:"varint,4,rep,packed,name=versions,proto3" json:"versions,omitempty"` +} + +func (x *ReplicaSegmentsInfo) Reset() { + *x = ReplicaSegmentsInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[28] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ReplicaSegmentsInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ReplicaSegmentsInfo) ProtoMessage() {} + +func (x *ReplicaSegmentsInfo) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_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 ReplicaSegmentsInfo.ProtoReflect.Descriptor instead. +func (*ReplicaSegmentsInfo) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{28} +} + +func (x *ReplicaSegmentsInfo) GetNodeId() int64 { + if x != nil { + return x.NodeId + } + return 0 +} + +func (x *ReplicaSegmentsInfo) GetPartitionId() int64 { + if x != nil { + return x.PartitionId + } + return 0 +} + +func (x *ReplicaSegmentsInfo) GetSegmentIds() []int64 { + if x != nil { + return x.SegmentIds + } + return nil +} + +func (x *ReplicaSegmentsInfo) GetVersions() []int64 { + if x != nil { + return x.Versions + } + return nil +} + +type GetLoadInfoRequest 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=collection_id,json=collectionId,proto3" json:"collection_id,omitempty"` +} + +func (x *GetLoadInfoRequest) Reset() { + *x = GetLoadInfoRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[29] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetLoadInfoRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetLoadInfoRequest) ProtoMessage() {} + +func (x *GetLoadInfoRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_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 GetLoadInfoRequest.ProtoReflect.Descriptor instead. +func (*GetLoadInfoRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{29} +} + +func (x *GetLoadInfoRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *GetLoadInfoRequest) GetCollectionId() int64 { + if x != nil { + return x.CollectionId + } + return 0 +} + +type GetLoadInfoResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + Schema *schemapb.CollectionSchema `protobuf:"bytes,2,opt,name=schema,proto3" json:"schema,omitempty"` + LoadType LoadType `protobuf:"varint,3,opt,name=load_type,json=loadType,proto3,enum=milvus.proto.query.LoadType" json:"load_type,omitempty"` + Partitions []int64 `protobuf:"varint,4,rep,packed,name=partitions,proto3" json:"partitions,omitempty"` +} + +func (x *GetLoadInfoResponse) Reset() { + *x = GetLoadInfoResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[30] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetLoadInfoResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetLoadInfoResponse) ProtoMessage() {} + +func (x *GetLoadInfoResponse) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[30] + 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 GetLoadInfoResponse.ProtoReflect.Descriptor instead. +func (*GetLoadInfoResponse) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{30} +} + +func (x *GetLoadInfoResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *GetLoadInfoResponse) GetSchema() *schemapb.CollectionSchema { + if x != nil { + return x.Schema + } + return nil +} + +func (x *GetLoadInfoResponse) GetLoadType() LoadType { + if x != nil { + return x.LoadType + } + return LoadType_UnKnownType +} + +func (x *GetLoadInfoResponse) GetPartitions() []int64 { + if x != nil { + return x.Partitions + } + return nil +} + +type HandoffSegmentsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + SegmentInfos []*SegmentInfo `protobuf:"bytes,2,rep,name=segmentInfos,proto3" json:"segmentInfos,omitempty"` + ReleasedSegments []int64 `protobuf:"varint,3,rep,packed,name=released_segments,json=releasedSegments,proto3" json:"released_segments,omitempty"` +} + +func (x *HandoffSegmentsRequest) Reset() { + *x = HandoffSegmentsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[31] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *HandoffSegmentsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*HandoffSegmentsRequest) ProtoMessage() {} + +func (x *HandoffSegmentsRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[31] + 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 HandoffSegmentsRequest.ProtoReflect.Descriptor instead. +func (*HandoffSegmentsRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{31} +} + +func (x *HandoffSegmentsRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *HandoffSegmentsRequest) GetSegmentInfos() []*SegmentInfo { + if x != nil { + return x.SegmentInfos + } + return nil +} + +func (x *HandoffSegmentsRequest) GetReleasedSegments() []int64 { + if x != nil { + return x.ReleasedSegments + } + return nil +} + +type LoadBalanceRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + SourceNodeIDs []int64 `protobuf:"varint,2,rep,packed,name=source_nodeIDs,json=sourceNodeIDs,proto3" json:"source_nodeIDs,omitempty"` + BalanceReason TriggerCondition `protobuf:"varint,3,opt,name=balance_reason,json=balanceReason,proto3,enum=milvus.proto.query.TriggerCondition" json:"balance_reason,omitempty"` + DstNodeIDs []int64 `protobuf:"varint,4,rep,packed,name=dst_nodeIDs,json=dstNodeIDs,proto3" json:"dst_nodeIDs,omitempty"` + SealedSegmentIDs []int64 `protobuf:"varint,5,rep,packed,name=sealed_segmentIDs,json=sealedSegmentIDs,proto3" json:"sealed_segmentIDs,omitempty"` + CollectionID int64 `protobuf:"varint,6,opt,name=collectionID,proto3" json:"collectionID,omitempty"` +} + +func (x *LoadBalanceRequest) Reset() { + *x = LoadBalanceRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[32] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *LoadBalanceRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LoadBalanceRequest) ProtoMessage() {} + +func (x *LoadBalanceRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[32] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use LoadBalanceRequest.ProtoReflect.Descriptor instead. +func (*LoadBalanceRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{32} +} + +func (x *LoadBalanceRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *LoadBalanceRequest) GetSourceNodeIDs() []int64 { + if x != nil { + return x.SourceNodeIDs + } + return nil +} + +func (x *LoadBalanceRequest) GetBalanceReason() TriggerCondition { + if x != nil { + return x.BalanceReason + } + return TriggerCondition_UnKnowCondition +} + +func (x *LoadBalanceRequest) GetDstNodeIDs() []int64 { + if x != nil { + return x.DstNodeIDs + } + return nil +} + +func (x *LoadBalanceRequest) GetSealedSegmentIDs() []int64 { + if x != nil { + return x.SealedSegmentIDs + } + return nil +} + +func (x *LoadBalanceRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +type DmChannelWatchInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + DmChannel string `protobuf:"bytes,2,opt,name=dmChannel,proto3" json:"dmChannel,omitempty"` + NodeIDLoaded int64 `protobuf:"varint,3,opt,name=nodeID_loaded,json=nodeIDLoaded,proto3" json:"nodeID_loaded,omitempty"` + ReplicaID int64 `protobuf:"varint,4,opt,name=replicaID,proto3" json:"replicaID,omitempty"` + NodeIds []int64 `protobuf:"varint,5,rep,packed,name=node_ids,json=nodeIds,proto3" json:"node_ids,omitempty"` +} + +func (x *DmChannelWatchInfo) Reset() { + *x = DmChannelWatchInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[33] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DmChannelWatchInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DmChannelWatchInfo) ProtoMessage() {} + +func (x *DmChannelWatchInfo) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[33] + 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 DmChannelWatchInfo.ProtoReflect.Descriptor instead. +func (*DmChannelWatchInfo) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{33} +} + +func (x *DmChannelWatchInfo) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *DmChannelWatchInfo) GetDmChannel() string { + if x != nil { + return x.DmChannel + } + return "" +} + +func (x *DmChannelWatchInfo) GetNodeIDLoaded() int64 { + if x != nil { + return x.NodeIDLoaded + } + return 0 +} + +func (x *DmChannelWatchInfo) GetReplicaID() int64 { + if x != nil { + return x.ReplicaID + } + return 0 +} + +func (x *DmChannelWatchInfo) GetNodeIds() []int64 { + if x != nil { + return x.NodeIds + } + return nil +} + +type QueryChannelInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + QueryChannel string `protobuf:"bytes,2,opt,name=query_channel,json=queryChannel,proto3" json:"query_channel,omitempty"` + QueryResultChannel string `protobuf:"bytes,3,opt,name=query_result_channel,json=queryResultChannel,proto3" json:"query_result_channel,omitempty"` + GlobalSealedSegments []*SegmentInfo `protobuf:"bytes,4,rep,name=global_sealed_segments,json=globalSealedSegments,proto3" json:"global_sealed_segments,omitempty"` + SeekPosition *msgpb.MsgPosition `protobuf:"bytes,5,opt,name=seek_position,json=seekPosition,proto3" json:"seek_position,omitempty"` +} + +func (x *QueryChannelInfo) Reset() { + *x = QueryChannelInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[34] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *QueryChannelInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryChannelInfo) ProtoMessage() {} + +func (x *QueryChannelInfo) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[34] + 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 QueryChannelInfo.ProtoReflect.Descriptor instead. +func (*QueryChannelInfo) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{34} +} + +func (x *QueryChannelInfo) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *QueryChannelInfo) GetQueryChannel() string { + if x != nil { + return x.QueryChannel + } + return "" +} + +func (x *QueryChannelInfo) GetQueryResultChannel() string { + if x != nil { + return x.QueryResultChannel + } + return "" +} + +func (x *QueryChannelInfo) GetGlobalSealedSegments() []*SegmentInfo { + if x != nil { + return x.GlobalSealedSegments + } + return nil +} + +func (x *QueryChannelInfo) GetSeekPosition() *msgpb.MsgPosition { + if x != nil { + return x.SeekPosition + } + return nil +} + +type PartitionStates struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + PartitionID int64 `protobuf:"varint,1,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + State PartitionState `protobuf:"varint,2,opt,name=state,proto3,enum=milvus.proto.query.PartitionState" json:"state,omitempty"` + InMemoryPercentage int64 `protobuf:"varint,3,opt,name=inMemory_percentage,json=inMemoryPercentage,proto3" json:"inMemory_percentage,omitempty"` +} + +func (x *PartitionStates) Reset() { + *x = PartitionStates{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[35] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PartitionStates) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PartitionStates) ProtoMessage() {} + +func (x *PartitionStates) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[35] + 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 PartitionStates.ProtoReflect.Descriptor instead. +func (*PartitionStates) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{35} +} + +func (x *PartitionStates) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *PartitionStates) GetState() PartitionState { + if x != nil { + return x.State + } + return PartitionState_NotExist +} + +func (x *PartitionStates) GetInMemoryPercentage() int64 { + if x != nil { + return x.InMemoryPercentage + } + return 0 +} + +type SegmentInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SegmentID int64 `protobuf:"varint,1,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + CollectionID int64 `protobuf:"varint,2,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionID int64 `protobuf:"varint,3,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + // deprecated, check node_ids(NodeIds) field + NodeID int64 `protobuf:"varint,4,opt,name=nodeID,proto3" json:"nodeID,omitempty"` + MemSize int64 `protobuf:"varint,5,opt,name=mem_size,json=memSize,proto3" json:"mem_size,omitempty"` + NumRows int64 `protobuf:"varint,6,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"` + IndexName string `protobuf:"bytes,7,opt,name=index_name,json=indexName,proto3" json:"index_name,omitempty"` + IndexID int64 `protobuf:"varint,8,opt,name=indexID,proto3" json:"indexID,omitempty"` + DmChannel string `protobuf:"bytes,9,opt,name=dmChannel,proto3" json:"dmChannel,omitempty"` + CompactionFrom []int64 `protobuf:"varint,10,rep,packed,name=compactionFrom,proto3" json:"compactionFrom,omitempty"` + CreatedByCompaction bool `protobuf:"varint,11,opt,name=createdByCompaction,proto3" json:"createdByCompaction,omitempty"` + SegmentState commonpb.SegmentState `protobuf:"varint,12,opt,name=segment_state,json=segmentState,proto3,enum=milvus.proto.common.SegmentState" json:"segment_state,omitempty"` + IndexInfos []*FieldIndexInfo `protobuf:"bytes,13,rep,name=index_infos,json=indexInfos,proto3" json:"index_infos,omitempty"` + ReplicaIds []int64 `protobuf:"varint,14,rep,packed,name=replica_ids,json=replicaIds,proto3" json:"replica_ids,omitempty"` + NodeIds []int64 `protobuf:"varint,15,rep,packed,name=node_ids,json=nodeIds,proto3" json:"node_ids,omitempty"` + EnableIndex bool `protobuf:"varint,16,opt,name=enable_index,json=enableIndex,proto3" json:"enable_index,omitempty"` + IsFake bool `protobuf:"varint,17,opt,name=is_fake,json=isFake,proto3" json:"is_fake,omitempty"` + Level datapb.SegmentLevel `protobuf:"varint,18,opt,name=level,proto3,enum=milvus.proto.data.SegmentLevel" json:"level,omitempty"` + IsSorted bool `protobuf:"varint,19,opt,name=is_sorted,json=isSorted,proto3" json:"is_sorted,omitempty"` +} + +func (x *SegmentInfo) Reset() { + *x = SegmentInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[36] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentInfo) ProtoMessage() {} + +func (x *SegmentInfo) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[36] + 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 SegmentInfo.ProtoReflect.Descriptor instead. +func (*SegmentInfo) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{36} +} + +func (x *SegmentInfo) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *SegmentInfo) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *SegmentInfo) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *SegmentInfo) GetNodeID() int64 { + if x != nil { + return x.NodeID + } + return 0 +} + +func (x *SegmentInfo) GetMemSize() int64 { + if x != nil { + return x.MemSize + } + return 0 +} + +func (x *SegmentInfo) GetNumRows() int64 { + if x != nil { + return x.NumRows + } + return 0 +} + +func (x *SegmentInfo) GetIndexName() string { + if x != nil { + return x.IndexName + } + return "" +} + +func (x *SegmentInfo) GetIndexID() int64 { + if x != nil { + return x.IndexID + } + return 0 +} + +func (x *SegmentInfo) GetDmChannel() string { + if x != nil { + return x.DmChannel + } + return "" +} + +func (x *SegmentInfo) GetCompactionFrom() []int64 { + if x != nil { + return x.CompactionFrom + } + return nil +} + +func (x *SegmentInfo) GetCreatedByCompaction() bool { + if x != nil { + return x.CreatedByCompaction + } + return false +} + +func (x *SegmentInfo) GetSegmentState() commonpb.SegmentState { + if x != nil { + return x.SegmentState + } + return commonpb.SegmentState(0) +} + +func (x *SegmentInfo) GetIndexInfos() []*FieldIndexInfo { + if x != nil { + return x.IndexInfos + } + return nil +} + +func (x *SegmentInfo) GetReplicaIds() []int64 { + if x != nil { + return x.ReplicaIds + } + return nil +} + +func (x *SegmentInfo) GetNodeIds() []int64 { + if x != nil { + return x.NodeIds + } + return nil +} + +func (x *SegmentInfo) GetEnableIndex() bool { + if x != nil { + return x.EnableIndex + } + return false +} + +func (x *SegmentInfo) GetIsFake() bool { + if x != nil { + return x.IsFake + } + return false +} + +func (x *SegmentInfo) GetLevel() datapb.SegmentLevel { + if x != nil { + return x.Level + } + return datapb.SegmentLevel(0) +} + +func (x *SegmentInfo) GetIsSorted() bool { + if x != nil { + return x.IsSorted + } + return false +} + +type CollectionInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionIDs []int64 `protobuf:"varint,2,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` + PartitionStates []*PartitionStates `protobuf:"bytes,3,rep,name=partition_states,json=partitionStates,proto3" json:"partition_states,omitempty"` + LoadType LoadType `protobuf:"varint,4,opt,name=load_type,json=loadType,proto3,enum=milvus.proto.query.LoadType" json:"load_type,omitempty"` + Schema *schemapb.CollectionSchema `protobuf:"bytes,5,opt,name=schema,proto3" json:"schema,omitempty"` + ReleasedPartitionIDs []int64 `protobuf:"varint,6,rep,packed,name=released_partitionIDs,json=releasedPartitionIDs,proto3" json:"released_partitionIDs,omitempty"` + InMemoryPercentage int64 `protobuf:"varint,7,opt,name=inMemory_percentage,json=inMemoryPercentage,proto3" json:"inMemory_percentage,omitempty"` + ReplicaIds []int64 `protobuf:"varint,8,rep,packed,name=replica_ids,json=replicaIds,proto3" json:"replica_ids,omitempty"` + ReplicaNumber int32 `protobuf:"varint,9,opt,name=replica_number,json=replicaNumber,proto3" json:"replica_number,omitempty"` +} + +func (x *CollectionInfo) Reset() { + *x = CollectionInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[37] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CollectionInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CollectionInfo) ProtoMessage() {} + +func (x *CollectionInfo) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[37] + 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 CollectionInfo.ProtoReflect.Descriptor instead. +func (*CollectionInfo) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{37} +} + +func (x *CollectionInfo) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *CollectionInfo) GetPartitionIDs() []int64 { + if x != nil { + return x.PartitionIDs + } + return nil +} + +func (x *CollectionInfo) GetPartitionStates() []*PartitionStates { + if x != nil { + return x.PartitionStates + } + return nil +} + +func (x *CollectionInfo) GetLoadType() LoadType { + if x != nil { + return x.LoadType + } + return LoadType_UnKnownType +} + +func (x *CollectionInfo) GetSchema() *schemapb.CollectionSchema { + if x != nil { + return x.Schema + } + return nil +} + +func (x *CollectionInfo) GetReleasedPartitionIDs() []int64 { + if x != nil { + return x.ReleasedPartitionIDs + } + return nil +} + +func (x *CollectionInfo) GetInMemoryPercentage() int64 { + if x != nil { + return x.InMemoryPercentage + } + return 0 +} + +func (x *CollectionInfo) GetReplicaIds() []int64 { + if x != nil { + return x.ReplicaIds + } + return nil +} + +func (x *CollectionInfo) GetReplicaNumber() int32 { + if x != nil { + return x.ReplicaNumber + } + return 0 +} + +type UnsubscribeChannels struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + Channels []string `protobuf:"bytes,2,rep,name=channels,proto3" json:"channels,omitempty"` +} + +func (x *UnsubscribeChannels) Reset() { + *x = UnsubscribeChannels{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[38] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UnsubscribeChannels) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UnsubscribeChannels) ProtoMessage() {} + +func (x *UnsubscribeChannels) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[38] + 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 UnsubscribeChannels.ProtoReflect.Descriptor instead. +func (*UnsubscribeChannels) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{38} +} + +func (x *UnsubscribeChannels) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *UnsubscribeChannels) GetChannels() []string { + if x != nil { + return x.Channels + } + return nil +} + +type UnsubscribeChannelInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + NodeID int64 `protobuf:"varint,1,opt,name=nodeID,proto3" json:"nodeID,omitempty"` + CollectionChannels []*UnsubscribeChannels `protobuf:"bytes,2,rep,name=collection_channels,json=collectionChannels,proto3" json:"collection_channels,omitempty"` +} + +func (x *UnsubscribeChannelInfo) Reset() { + *x = UnsubscribeChannelInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[39] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UnsubscribeChannelInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UnsubscribeChannelInfo) ProtoMessage() {} + +func (x *UnsubscribeChannelInfo) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[39] + 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 UnsubscribeChannelInfo.ProtoReflect.Descriptor instead. +func (*UnsubscribeChannelInfo) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{39} +} + +func (x *UnsubscribeChannelInfo) GetNodeID() int64 { + if x != nil { + return x.NodeID + } + return 0 +} + +func (x *UnsubscribeChannelInfo) GetCollectionChannels() []*UnsubscribeChannels { + if x != nil { + return x.CollectionChannels + } + return nil +} + +type SegmentChangeInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + OnlineNodeID int64 `protobuf:"varint,1,opt,name=online_nodeID,json=onlineNodeID,proto3" json:"online_nodeID,omitempty"` + OnlineSegments []*SegmentInfo `protobuf:"bytes,2,rep,name=online_segments,json=onlineSegments,proto3" json:"online_segments,omitempty"` + OfflineNodeID int64 `protobuf:"varint,3,opt,name=offline_nodeID,json=offlineNodeID,proto3" json:"offline_nodeID,omitempty"` + OfflineSegments []*SegmentInfo `protobuf:"bytes,4,rep,name=offline_segments,json=offlineSegments,proto3" json:"offline_segments,omitempty"` +} + +func (x *SegmentChangeInfo) Reset() { + *x = SegmentChangeInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[40] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentChangeInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentChangeInfo) ProtoMessage() {} + +func (x *SegmentChangeInfo) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[40] + 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 SegmentChangeInfo.ProtoReflect.Descriptor instead. +func (*SegmentChangeInfo) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{40} +} + +func (x *SegmentChangeInfo) GetOnlineNodeID() int64 { + if x != nil { + return x.OnlineNodeID + } + return 0 +} + +func (x *SegmentChangeInfo) GetOnlineSegments() []*SegmentInfo { + if x != nil { + return x.OnlineSegments + } + return nil +} + +func (x *SegmentChangeInfo) GetOfflineNodeID() int64 { + if x != nil { + return x.OfflineNodeID + } + return 0 +} + +func (x *SegmentChangeInfo) GetOfflineSegments() []*SegmentInfo { + if x != nil { + return x.OfflineSegments + } + return nil +} + +type SealedSegmentsChangeInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + Infos []*SegmentChangeInfo `protobuf:"bytes,2,rep,name=infos,proto3" json:"infos,omitempty"` +} + +func (x *SealedSegmentsChangeInfo) Reset() { + *x = SealedSegmentsChangeInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[41] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SealedSegmentsChangeInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SealedSegmentsChangeInfo) ProtoMessage() {} + +func (x *SealedSegmentsChangeInfo) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[41] + 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 SealedSegmentsChangeInfo.ProtoReflect.Descriptor instead. +func (*SealedSegmentsChangeInfo) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{41} +} + +func (x *SealedSegmentsChangeInfo) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *SealedSegmentsChangeInfo) GetInfos() []*SegmentChangeInfo { + if x != nil { + return x.Infos + } + return nil +} + +type GetDataDistributionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + Checkpoints map[string]*msgpb.MsgPosition `protobuf:"bytes,2,rep,name=checkpoints,proto3" json:"checkpoints,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + LastUpdateTs int64 `protobuf:"varint,3,opt,name=lastUpdateTs,proto3" json:"lastUpdateTs,omitempty"` +} + +func (x *GetDataDistributionRequest) Reset() { + *x = GetDataDistributionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[42] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetDataDistributionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetDataDistributionRequest) ProtoMessage() {} + +func (x *GetDataDistributionRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[42] + 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 GetDataDistributionRequest.ProtoReflect.Descriptor instead. +func (*GetDataDistributionRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{42} +} + +func (x *GetDataDistributionRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *GetDataDistributionRequest) GetCheckpoints() map[string]*msgpb.MsgPosition { + if x != nil { + return x.Checkpoints + } + return nil +} + +func (x *GetDataDistributionRequest) GetLastUpdateTs() int64 { + if x != nil { + return x.LastUpdateTs + } + return 0 +} + +type GetDataDistributionResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + NodeID int64 `protobuf:"varint,2,opt,name=nodeID,proto3" json:"nodeID,omitempty"` + Segments []*SegmentVersionInfo `protobuf:"bytes,3,rep,name=segments,proto3" json:"segments,omitempty"` + Channels []*ChannelVersionInfo `protobuf:"bytes,4,rep,name=channels,proto3" json:"channels,omitempty"` + LeaderViews []*LeaderView `protobuf:"bytes,5,rep,name=leader_views,json=leaderViews,proto3" json:"leader_views,omitempty"` + LastModifyTs int64 `protobuf:"varint,6,opt,name=lastModifyTs,proto3" json:"lastModifyTs,omitempty"` + MemCapacityInMB float64 `protobuf:"fixed64,7,opt,name=memCapacityInMB,proto3" json:"memCapacityInMB,omitempty"` +} + +func (x *GetDataDistributionResponse) Reset() { + *x = GetDataDistributionResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[43] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetDataDistributionResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetDataDistributionResponse) ProtoMessage() {} + +func (x *GetDataDistributionResponse) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[43] + 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 GetDataDistributionResponse.ProtoReflect.Descriptor instead. +func (*GetDataDistributionResponse) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{43} +} + +func (x *GetDataDistributionResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *GetDataDistributionResponse) GetNodeID() int64 { + if x != nil { + return x.NodeID + } + return 0 +} + +func (x *GetDataDistributionResponse) GetSegments() []*SegmentVersionInfo { + if x != nil { + return x.Segments + } + return nil +} + +func (x *GetDataDistributionResponse) GetChannels() []*ChannelVersionInfo { + if x != nil { + return x.Channels + } + return nil +} + +func (x *GetDataDistributionResponse) GetLeaderViews() []*LeaderView { + if x != nil { + return x.LeaderViews + } + return nil +} + +func (x *GetDataDistributionResponse) GetLastModifyTs() int64 { + if x != nil { + return x.LastModifyTs + } + return 0 +} + +func (x *GetDataDistributionResponse) GetMemCapacityInMB() float64 { + if x != nil { + return x.MemCapacityInMB + } + return 0 +} + +type LeaderView struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Collection int64 `protobuf:"varint,1,opt,name=collection,proto3" json:"collection,omitempty"` + Channel string `protobuf:"bytes,2,opt,name=channel,proto3" json:"channel,omitempty"` + SegmentDist map[int64]*SegmentDist `protobuf:"bytes,3,rep,name=segment_dist,json=segmentDist,proto3" json:"segment_dist,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + GrowingSegmentIDs []int64 `protobuf:"varint,4,rep,packed,name=growing_segmentIDs,json=growingSegmentIDs,proto3" json:"growing_segmentIDs,omitempty"` + GrowingSegments map[int64]*msgpb.MsgPosition `protobuf:"bytes,5,rep,name=growing_segments,json=growingSegments,proto3" json:"growing_segments,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + TargetVersion int64 `protobuf:"varint,6,opt,name=TargetVersion,proto3" json:"TargetVersion,omitempty"` + NumOfGrowingRows int64 `protobuf:"varint,7,opt,name=num_of_growing_rows,json=numOfGrowingRows,proto3" json:"num_of_growing_rows,omitempty"` + PartitionStatsVersions map[int64]int64 `protobuf:"bytes,8,rep,name=partition_stats_versions,json=partitionStatsVersions,proto3" json:"partition_stats_versions,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` +} + +func (x *LeaderView) Reset() { + *x = LeaderView{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[44] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *LeaderView) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LeaderView) ProtoMessage() {} + +func (x *LeaderView) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[44] + 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 LeaderView.ProtoReflect.Descriptor instead. +func (*LeaderView) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{44} +} + +func (x *LeaderView) GetCollection() int64 { + if x != nil { + return x.Collection + } + return 0 +} + +func (x *LeaderView) GetChannel() string { + if x != nil { + return x.Channel + } + return "" +} + +func (x *LeaderView) GetSegmentDist() map[int64]*SegmentDist { + if x != nil { + return x.SegmentDist + } + return nil +} + +func (x *LeaderView) GetGrowingSegmentIDs() []int64 { + if x != nil { + return x.GrowingSegmentIDs + } + return nil +} + +func (x *LeaderView) GetGrowingSegments() map[int64]*msgpb.MsgPosition { + if x != nil { + return x.GrowingSegments + } + return nil +} + +func (x *LeaderView) GetTargetVersion() int64 { + if x != nil { + return x.TargetVersion + } + return 0 +} + +func (x *LeaderView) GetNumOfGrowingRows() int64 { + if x != nil { + return x.NumOfGrowingRows + } + return 0 +} + +func (x *LeaderView) GetPartitionStatsVersions() map[int64]int64 { + if x != nil { + return x.PartitionStatsVersions + } + return nil +} + +type SegmentDist struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + NodeID int64 `protobuf:"varint,1,opt,name=nodeID,proto3" json:"nodeID,omitempty"` + Version int64 `protobuf:"varint,2,opt,name=version,proto3" json:"version,omitempty"` +} + +func (x *SegmentDist) Reset() { + *x = SegmentDist{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[45] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentDist) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentDist) ProtoMessage() {} + +func (x *SegmentDist) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[45] + 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 SegmentDist.ProtoReflect.Descriptor instead. +func (*SegmentDist) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{45} +} + +func (x *SegmentDist) GetNodeID() int64 { + if x != nil { + return x.NodeID + } + return 0 +} + +func (x *SegmentDist) GetVersion() int64 { + if x != nil { + return x.Version + } + return 0 +} + +type SegmentVersionInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ID int64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"` + Collection int64 `protobuf:"varint,2,opt,name=collection,proto3" json:"collection,omitempty"` + Partition int64 `protobuf:"varint,3,opt,name=partition,proto3" json:"partition,omitempty"` + Channel string `protobuf:"bytes,4,opt,name=channel,proto3" json:"channel,omitempty"` + Version int64 `protobuf:"varint,5,opt,name=version,proto3" json:"version,omitempty"` + LastDeltaTimestamp uint64 `protobuf:"varint,6,opt,name=last_delta_timestamp,json=lastDeltaTimestamp,proto3" json:"last_delta_timestamp,omitempty"` + IndexInfo map[int64]*FieldIndexInfo `protobuf:"bytes,7,rep,name=index_info,json=indexInfo,proto3" json:"index_info,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Level datapb.SegmentLevel `protobuf:"varint,8,opt,name=level,proto3,enum=milvus.proto.data.SegmentLevel" json:"level,omitempty"` + IsSorted bool `protobuf:"varint,9,opt,name=is_sorted,json=isSorted,proto3" json:"is_sorted,omitempty"` +} + +func (x *SegmentVersionInfo) Reset() { + *x = SegmentVersionInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[46] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentVersionInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentVersionInfo) ProtoMessage() {} + +func (x *SegmentVersionInfo) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[46] + 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 SegmentVersionInfo.ProtoReflect.Descriptor instead. +func (*SegmentVersionInfo) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{46} +} + +func (x *SegmentVersionInfo) GetID() int64 { + if x != nil { + return x.ID + } + return 0 +} + +func (x *SegmentVersionInfo) GetCollection() int64 { + if x != nil { + return x.Collection + } + return 0 +} + +func (x *SegmentVersionInfo) GetPartition() int64 { + if x != nil { + return x.Partition + } + return 0 +} + +func (x *SegmentVersionInfo) GetChannel() string { + if x != nil { + return x.Channel + } + return "" +} + +func (x *SegmentVersionInfo) GetVersion() int64 { + if x != nil { + return x.Version + } + return 0 +} + +func (x *SegmentVersionInfo) GetLastDeltaTimestamp() uint64 { + if x != nil { + return x.LastDeltaTimestamp + } + return 0 +} + +func (x *SegmentVersionInfo) GetIndexInfo() map[int64]*FieldIndexInfo { + if x != nil { + return x.IndexInfo + } + return nil +} + +func (x *SegmentVersionInfo) GetLevel() datapb.SegmentLevel { + if x != nil { + return x.Level + } + return datapb.SegmentLevel(0) +} + +func (x *SegmentVersionInfo) GetIsSorted() bool { + if x != nil { + return x.IsSorted + } + return false +} + +type ChannelVersionInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Channel string `protobuf:"bytes,1,opt,name=channel,proto3" json:"channel,omitempty"` + Collection int64 `protobuf:"varint,2,opt,name=collection,proto3" json:"collection,omitempty"` + Version int64 `protobuf:"varint,3,opt,name=version,proto3" json:"version,omitempty"` +} + +func (x *ChannelVersionInfo) Reset() { + *x = ChannelVersionInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[47] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ChannelVersionInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ChannelVersionInfo) ProtoMessage() {} + +func (x *ChannelVersionInfo) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[47] + 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 ChannelVersionInfo.ProtoReflect.Descriptor instead. +func (*ChannelVersionInfo) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{47} +} + +func (x *ChannelVersionInfo) GetChannel() string { + if x != nil { + return x.Channel + } + return "" +} + +func (x *ChannelVersionInfo) GetCollection() int64 { + if x != nil { + return x.Collection + } + return 0 +} + +func (x *ChannelVersionInfo) GetVersion() int64 { + if x != nil { + return x.Version + } + return 0 +} + +type CollectionLoadInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + ReleasedPartitions []int64 `protobuf:"varint,2,rep,packed,name=released_partitions,json=releasedPartitions,proto3" json:"released_partitions,omitempty"` // Deprecated: No longer used; kept for compatibility. + ReplicaNumber int32 `protobuf:"varint,3,opt,name=replica_number,json=replicaNumber,proto3" json:"replica_number,omitempty"` + Status LoadStatus `protobuf:"varint,4,opt,name=status,proto3,enum=milvus.proto.query.LoadStatus" json:"status,omitempty"` + FieldIndexID map[int64]int64 `protobuf:"bytes,5,rep,name=field_indexID,json=fieldIndexID,proto3" json:"field_indexID,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + LoadType LoadType `protobuf:"varint,6,opt,name=load_type,json=loadType,proto3,enum=milvus.proto.query.LoadType" json:"load_type,omitempty"` + RecoverTimes int32 `protobuf:"varint,7,opt,name=recover_times,json=recoverTimes,proto3" json:"recover_times,omitempty"` + LoadFields []int64 `protobuf:"varint,8,rep,packed,name=load_fields,json=loadFields,proto3" json:"load_fields,omitempty"` + DbID int64 `protobuf:"varint,9,opt,name=dbID,proto3" json:"dbID,omitempty"` +} + +func (x *CollectionLoadInfo) Reset() { + *x = CollectionLoadInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[48] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CollectionLoadInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CollectionLoadInfo) ProtoMessage() {} + +func (x *CollectionLoadInfo) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[48] + 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 CollectionLoadInfo.ProtoReflect.Descriptor instead. +func (*CollectionLoadInfo) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{48} +} + +func (x *CollectionLoadInfo) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *CollectionLoadInfo) GetReleasedPartitions() []int64 { + if x != nil { + return x.ReleasedPartitions + } + return nil +} + +func (x *CollectionLoadInfo) GetReplicaNumber() int32 { + if x != nil { + return x.ReplicaNumber + } + return 0 +} + +func (x *CollectionLoadInfo) GetStatus() LoadStatus { + if x != nil { + return x.Status + } + return LoadStatus_Invalid +} + +func (x *CollectionLoadInfo) GetFieldIndexID() map[int64]int64 { + if x != nil { + return x.FieldIndexID + } + return nil +} + +func (x *CollectionLoadInfo) GetLoadType() LoadType { + if x != nil { + return x.LoadType + } + return LoadType_UnKnownType +} + +func (x *CollectionLoadInfo) GetRecoverTimes() int32 { + if x != nil { + return x.RecoverTimes + } + return 0 +} + +func (x *CollectionLoadInfo) GetLoadFields() []int64 { + if x != nil { + return x.LoadFields + } + return nil +} + +func (x *CollectionLoadInfo) GetDbID() int64 { + if x != nil { + return x.DbID + } + return 0 +} + +type PartitionLoadInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionID int64 `protobuf:"varint,2,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + ReplicaNumber int32 `protobuf:"varint,3,opt,name=replica_number,json=replicaNumber,proto3" json:"replica_number,omitempty"` // Deprecated: No longer used; kept for compatibility. + Status LoadStatus `protobuf:"varint,4,opt,name=status,proto3,enum=milvus.proto.query.LoadStatus" json:"status,omitempty"` + FieldIndexID map[int64]int64 `protobuf:"bytes,5,rep,name=field_indexID,json=fieldIndexID,proto3" json:"field_indexID,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` // Deprecated: No longer used; kept for compatibility. + RecoverTimes int32 `protobuf:"varint,7,opt,name=recover_times,json=recoverTimes,proto3" json:"recover_times,omitempty"` +} + +func (x *PartitionLoadInfo) Reset() { + *x = PartitionLoadInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[49] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PartitionLoadInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PartitionLoadInfo) ProtoMessage() {} + +func (x *PartitionLoadInfo) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[49] + 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 PartitionLoadInfo.ProtoReflect.Descriptor instead. +func (*PartitionLoadInfo) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{49} +} + +func (x *PartitionLoadInfo) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *PartitionLoadInfo) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *PartitionLoadInfo) GetReplicaNumber() int32 { + if x != nil { + return x.ReplicaNumber + } + return 0 +} + +func (x *PartitionLoadInfo) GetStatus() LoadStatus { + if x != nil { + return x.Status + } + return LoadStatus_Invalid +} + +func (x *PartitionLoadInfo) GetFieldIndexID() map[int64]int64 { + if x != nil { + return x.FieldIndexID + } + return nil +} + +func (x *PartitionLoadInfo) GetRecoverTimes() int32 { + if x != nil { + return x.RecoverTimes + } + return 0 +} + +type ChannelNodeInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + RwNodes []int64 `protobuf:"varint,6,rep,packed,name=rw_nodes,json=rwNodes,proto3" json:"rw_nodes,omitempty"` +} + +func (x *ChannelNodeInfo) Reset() { + *x = ChannelNodeInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[50] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ChannelNodeInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ChannelNodeInfo) ProtoMessage() {} + +func (x *ChannelNodeInfo) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[50] + 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 ChannelNodeInfo.ProtoReflect.Descriptor instead. +func (*ChannelNodeInfo) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{50} +} + +func (x *ChannelNodeInfo) GetRwNodes() []int64 { + if x != nil { + return x.RwNodes + } + return nil +} + +type Replica struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ID int64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"` + CollectionID int64 `protobuf:"varint,2,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + Nodes []int64 `protobuf:"varint,3,rep,packed,name=nodes,proto3" json:"nodes,omitempty"` // all (read and write) nodes. mutual exclusive with ro_nodes. + ResourceGroup string `protobuf:"bytes,4,opt,name=resource_group,json=resourceGroup,proto3" json:"resource_group,omitempty"` + RoNodes []int64 `protobuf:"varint,5,rep,packed,name=ro_nodes,json=roNodes,proto3" json:"ro_nodes,omitempty"` // the in-using node but should not be assigned to these replica. + // can not load new channel or segment on it anymore. + ChannelNodeInfos map[string]*ChannelNodeInfo `protobuf:"bytes,6,rep,name=channel_node_infos,json=channelNodeInfos,proto3" json:"channel_node_infos,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *Replica) Reset() { + *x = Replica{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[51] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Replica) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Replica) ProtoMessage() {} + +func (x *Replica) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[51] + 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 Replica.ProtoReflect.Descriptor instead. +func (*Replica) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{51} +} + +func (x *Replica) GetID() int64 { + if x != nil { + return x.ID + } + return 0 +} + +func (x *Replica) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *Replica) GetNodes() []int64 { + if x != nil { + return x.Nodes + } + return nil +} + +func (x *Replica) GetResourceGroup() string { + if x != nil { + return x.ResourceGroup + } + return "" +} + +func (x *Replica) GetRoNodes() []int64 { + if x != nil { + return x.RoNodes + } + return nil +} + +func (x *Replica) GetChannelNodeInfos() map[string]*ChannelNodeInfo { + if x != nil { + return x.ChannelNodeInfos + } + return nil +} + +type SyncAction struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Type SyncType `protobuf:"varint,1,opt,name=type,proto3,enum=milvus.proto.query.SyncType" json:"type,omitempty"` + PartitionID int64 `protobuf:"varint,2,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + SegmentID int64 `protobuf:"varint,3,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + NodeID int64 `protobuf:"varint,4,opt,name=nodeID,proto3" json:"nodeID,omitempty"` + Version int64 `protobuf:"varint,5,opt,name=version,proto3" json:"version,omitempty"` + Info *SegmentLoadInfo `protobuf:"bytes,6,opt,name=info,proto3" json:"info,omitempty"` + GrowingInTarget []int64 `protobuf:"varint,7,rep,packed,name=growingInTarget,proto3" json:"growingInTarget,omitempty"` + SealedInTarget []int64 `protobuf:"varint,8,rep,packed,name=sealedInTarget,proto3" json:"sealedInTarget,omitempty"` + TargetVersion int64 `protobuf:"varint,9,opt,name=TargetVersion,proto3" json:"TargetVersion,omitempty"` + DroppedInTarget []int64 `protobuf:"varint,10,rep,packed,name=droppedInTarget,proto3" json:"droppedInTarget,omitempty"` + Checkpoint *msgpb.MsgPosition `protobuf:"bytes,11,opt,name=checkpoint,proto3" json:"checkpoint,omitempty"` + PartitionStatsVersions map[int64]int64 `protobuf:"bytes,12,rep,name=partition_stats_versions,json=partitionStatsVersions,proto3" json:"partition_stats_versions,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` +} + +func (x *SyncAction) Reset() { + *x = SyncAction{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[52] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SyncAction) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SyncAction) ProtoMessage() {} + +func (x *SyncAction) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[52] + 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 SyncAction.ProtoReflect.Descriptor instead. +func (*SyncAction) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{52} +} + +func (x *SyncAction) GetType() SyncType { + if x != nil { + return x.Type + } + return SyncType_Remove +} + +func (x *SyncAction) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *SyncAction) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *SyncAction) GetNodeID() int64 { + if x != nil { + return x.NodeID + } + return 0 +} + +func (x *SyncAction) GetVersion() int64 { + if x != nil { + return x.Version + } + return 0 +} + +func (x *SyncAction) GetInfo() *SegmentLoadInfo { + if x != nil { + return x.Info + } + return nil +} + +func (x *SyncAction) GetGrowingInTarget() []int64 { + if x != nil { + return x.GrowingInTarget + } + return nil +} + +func (x *SyncAction) GetSealedInTarget() []int64 { + if x != nil { + return x.SealedInTarget + } + return nil +} + +func (x *SyncAction) GetTargetVersion() int64 { + if x != nil { + return x.TargetVersion + } + return 0 +} + +func (x *SyncAction) GetDroppedInTarget() []int64 { + if x != nil { + return x.DroppedInTarget + } + return nil +} + +func (x *SyncAction) GetCheckpoint() *msgpb.MsgPosition { + if x != nil { + return x.Checkpoint + } + return nil +} + +func (x *SyncAction) GetPartitionStatsVersions() map[int64]int64 { + if x != nil { + return x.PartitionStatsVersions + } + return nil +} + +type SyncDistributionRequest 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"` + Channel string `protobuf:"bytes,3,opt,name=channel,proto3" json:"channel,omitempty"` + Actions []*SyncAction `protobuf:"bytes,4,rep,name=actions,proto3" json:"actions,omitempty"` + Schema *schemapb.CollectionSchema `protobuf:"bytes,5,opt,name=schema,proto3" json:"schema,omitempty"` + LoadMeta *LoadMetaInfo `protobuf:"bytes,6,opt,name=load_meta,json=loadMeta,proto3" json:"load_meta,omitempty"` + ReplicaID int64 `protobuf:"varint,7,opt,name=replicaID,proto3" json:"replicaID,omitempty"` + Version int64 `protobuf:"varint,8,opt,name=version,proto3" json:"version,omitempty"` + IndexInfoList []*indexpb.IndexInfo `protobuf:"bytes,9,rep,name=index_info_list,json=indexInfoList,proto3" json:"index_info_list,omitempty"` +} + +func (x *SyncDistributionRequest) Reset() { + *x = SyncDistributionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[53] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SyncDistributionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SyncDistributionRequest) ProtoMessage() {} + +func (x *SyncDistributionRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[53] + 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 SyncDistributionRequest.ProtoReflect.Descriptor instead. +func (*SyncDistributionRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{53} +} + +func (x *SyncDistributionRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *SyncDistributionRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *SyncDistributionRequest) GetChannel() string { + if x != nil { + return x.Channel + } + return "" +} + +func (x *SyncDistributionRequest) GetActions() []*SyncAction { + if x != nil { + return x.Actions + } + return nil +} + +func (x *SyncDistributionRequest) GetSchema() *schemapb.CollectionSchema { + if x != nil { + return x.Schema + } + return nil +} + +func (x *SyncDistributionRequest) GetLoadMeta() *LoadMetaInfo { + if x != nil { + return x.LoadMeta + } + return nil +} + +func (x *SyncDistributionRequest) GetReplicaID() int64 { + if x != nil { + return x.ReplicaID + } + return 0 +} + +func (x *SyncDistributionRequest) GetVersion() int64 { + if x != nil { + return x.Version + } + return 0 +} + +func (x *SyncDistributionRequest) GetIndexInfoList() []*indexpb.IndexInfo { + if x != nil { + return x.IndexInfoList + } + return nil +} + +type ResourceGroup struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + // Deprecated: Marked as deprecated in query_coord.proto. + Capacity int32 `protobuf:"varint,2,opt,name=capacity,proto3" json:"capacity,omitempty"` // capacity can be found in config.requests.nodeNum and config.limits.nodeNum. + Nodes []int64 `protobuf:"varint,3,rep,packed,name=nodes,proto3" json:"nodes,omitempty"` + Config *rgpb.ResourceGroupConfig `protobuf:"bytes,4,opt,name=config,proto3" json:"config,omitempty"` +} + +func (x *ResourceGroup) Reset() { + *x = ResourceGroup{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[54] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ResourceGroup) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResourceGroup) ProtoMessage() {} + +func (x *ResourceGroup) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[54] + 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 ResourceGroup.ProtoReflect.Descriptor instead. +func (*ResourceGroup) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{54} +} + +func (x *ResourceGroup) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +// Deprecated: Marked as deprecated in query_coord.proto. +func (x *ResourceGroup) GetCapacity() int32 { + if x != nil { + return x.Capacity + } + return 0 +} + +func (x *ResourceGroup) GetNodes() []int64 { + if x != nil { + return x.Nodes + } + return nil +} + +func (x *ResourceGroup) GetConfig() *rgpb.ResourceGroupConfig { + if x != nil { + return x.Config + } + return nil +} + +// transfer `replicaNum` replicas in `collectionID` from `source_resource_group` to `target_resource_groups` +type TransferReplicaRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + SourceResourceGroup string `protobuf:"bytes,2,opt,name=source_resource_group,json=sourceResourceGroup,proto3" json:"source_resource_group,omitempty"` + TargetResourceGroup string `protobuf:"bytes,3,opt,name=target_resource_group,json=targetResourceGroup,proto3" json:"target_resource_group,omitempty"` + CollectionID int64 `protobuf:"varint,4,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + NumReplica int64 `protobuf:"varint,5,opt,name=num_replica,json=numReplica,proto3" json:"num_replica,omitempty"` +} + +func (x *TransferReplicaRequest) Reset() { + *x = TransferReplicaRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[55] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TransferReplicaRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TransferReplicaRequest) ProtoMessage() {} + +func (x *TransferReplicaRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[55] + 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 TransferReplicaRequest.ProtoReflect.Descriptor instead. +func (*TransferReplicaRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{55} +} + +func (x *TransferReplicaRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *TransferReplicaRequest) GetSourceResourceGroup() string { + if x != nil { + return x.SourceResourceGroup + } + return "" +} + +func (x *TransferReplicaRequest) GetTargetResourceGroup() string { + if x != nil { + return x.TargetResourceGroup + } + return "" +} + +func (x *TransferReplicaRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *TransferReplicaRequest) GetNumReplica() int64 { + if x != nil { + return x.NumReplica + } + return 0 +} + +type DescribeResourceGroupRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + ResourceGroup string `protobuf:"bytes,2,opt,name=resource_group,json=resourceGroup,proto3" json:"resource_group,omitempty"` +} + +func (x *DescribeResourceGroupRequest) Reset() { + *x = DescribeResourceGroupRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[56] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DescribeResourceGroupRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DescribeResourceGroupRequest) ProtoMessage() {} + +func (x *DescribeResourceGroupRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[56] + 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 DescribeResourceGroupRequest.ProtoReflect.Descriptor instead. +func (*DescribeResourceGroupRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{56} +} + +func (x *DescribeResourceGroupRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *DescribeResourceGroupRequest) GetResourceGroup() string { + if x != nil { + return x.ResourceGroup + } + return "" +} + +type DescribeResourceGroupResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + ResourceGroup *ResourceGroupInfo `protobuf:"bytes,2,opt,name=resource_group,json=resourceGroup,proto3" json:"resource_group,omitempty"` +} + +func (x *DescribeResourceGroupResponse) Reset() { + *x = DescribeResourceGroupResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[57] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DescribeResourceGroupResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DescribeResourceGroupResponse) ProtoMessage() {} + +func (x *DescribeResourceGroupResponse) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[57] + 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 DescribeResourceGroupResponse.ProtoReflect.Descriptor instead. +func (*DescribeResourceGroupResponse) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{57} +} + +func (x *DescribeResourceGroupResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *DescribeResourceGroupResponse) GetResourceGroup() *ResourceGroupInfo { + if x != nil { + return x.ResourceGroup + } + return nil +} + +type ResourceGroupInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + // Deprecated: Marked as deprecated in query_coord.proto. + Capacity int32 `protobuf:"varint,2,opt,name=capacity,proto3" json:"capacity,omitempty"` // capacity can be found in config.requests.nodeNum and config.limits.nodeNum. + NumAvailableNode int32 `protobuf:"varint,3,opt,name=num_available_node,json=numAvailableNode,proto3" json:"num_available_node,omitempty"` + // collection id -> loaded replica num + NumLoadedReplica map[int64]int32 `protobuf:"bytes,4,rep,name=num_loaded_replica,json=numLoadedReplica,proto3" json:"num_loaded_replica,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + // collection id -> accessed other rg's node num + NumOutgoingNode map[int64]int32 `protobuf:"bytes,5,rep,name=num_outgoing_node,json=numOutgoingNode,proto3" json:"num_outgoing_node,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + // collection id -> be accessed node num by other rg + NumIncomingNode map[int64]int32 `protobuf:"bytes,6,rep,name=num_incoming_node,json=numIncomingNode,proto3" json:"num_incoming_node,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + // resource group configuration. + Config *rgpb.ResourceGroupConfig `protobuf:"bytes,7,opt,name=config,proto3" json:"config,omitempty"` + Nodes []*commonpb.NodeInfo `protobuf:"bytes,8,rep,name=nodes,proto3" json:"nodes,omitempty"` +} + +func (x *ResourceGroupInfo) Reset() { + *x = ResourceGroupInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[58] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ResourceGroupInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResourceGroupInfo) ProtoMessage() {} + +func (x *ResourceGroupInfo) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[58] + 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 ResourceGroupInfo.ProtoReflect.Descriptor instead. +func (*ResourceGroupInfo) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{58} +} + +func (x *ResourceGroupInfo) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +// Deprecated: Marked as deprecated in query_coord.proto. +func (x *ResourceGroupInfo) GetCapacity() int32 { + if x != nil { + return x.Capacity + } + return 0 +} + +func (x *ResourceGroupInfo) GetNumAvailableNode() int32 { + if x != nil { + return x.NumAvailableNode + } + return 0 +} + +func (x *ResourceGroupInfo) GetNumLoadedReplica() map[int64]int32 { + if x != nil { + return x.NumLoadedReplica + } + return nil +} + +func (x *ResourceGroupInfo) GetNumOutgoingNode() map[int64]int32 { + if x != nil { + return x.NumOutgoingNode + } + return nil +} + +func (x *ResourceGroupInfo) GetNumIncomingNode() map[int64]int32 { + if x != nil { + return x.NumIncomingNode + } + return nil +} + +func (x *ResourceGroupInfo) GetConfig() *rgpb.ResourceGroupConfig { + if x != nil { + return x.Config + } + return nil +} + +func (x *ResourceGroupInfo) GetNodes() []*commonpb.NodeInfo { + if x != nil { + return x.Nodes + } + return nil +} + +type DeleteRequest 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=collection_id,json=collectionId,proto3" json:"collection_id,omitempty"` + PartitionId int64 `protobuf:"varint,3,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"` + VchannelName string `protobuf:"bytes,4,opt,name=vchannel_name,json=vchannelName,proto3" json:"vchannel_name,omitempty"` + SegmentId int64 `protobuf:"varint,5,opt,name=segment_id,json=segmentId,proto3" json:"segment_id,omitempty"` + PrimaryKeys *schemapb.IDs `protobuf:"bytes,6,opt,name=primary_keys,json=primaryKeys,proto3" json:"primary_keys,omitempty"` + Timestamps []uint64 `protobuf:"varint,7,rep,packed,name=timestamps,proto3" json:"timestamps,omitempty"` + Scope DataScope `protobuf:"varint,8,opt,name=scope,proto3,enum=milvus.proto.query.DataScope" json:"scope,omitempty"` +} + +func (x *DeleteRequest) Reset() { + *x = DeleteRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[59] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeleteRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeleteRequest) ProtoMessage() {} + +func (x *DeleteRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[59] + 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 DeleteRequest.ProtoReflect.Descriptor instead. +func (*DeleteRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{59} +} + +func (x *DeleteRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *DeleteRequest) GetCollectionId() int64 { + if x != nil { + return x.CollectionId + } + return 0 +} + +func (x *DeleteRequest) GetPartitionId() int64 { + if x != nil { + return x.PartitionId + } + return 0 +} + +func (x *DeleteRequest) GetVchannelName() string { + if x != nil { + return x.VchannelName + } + return "" +} + +func (x *DeleteRequest) GetSegmentId() int64 { + if x != nil { + return x.SegmentId + } + return 0 +} + +func (x *DeleteRequest) GetPrimaryKeys() *schemapb.IDs { + if x != nil { + return x.PrimaryKeys + } + return nil +} + +func (x *DeleteRequest) GetTimestamps() []uint64 { + if x != nil { + return x.Timestamps + } + return nil +} + +func (x *DeleteRequest) GetScope() DataScope { + if x != nil { + return x.Scope + } + return DataScope_UnKnown +} + +type DeleteBatchRequest 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=collection_id,json=collectionId,proto3" json:"collection_id,omitempty"` + PartitionId int64 `protobuf:"varint,3,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"` + VchannelName string `protobuf:"bytes,4,opt,name=vchannel_name,json=vchannelName,proto3" json:"vchannel_name,omitempty"` + SegmentIds []int64 `protobuf:"varint,5,rep,packed,name=segment_ids,json=segmentIds,proto3" json:"segment_ids,omitempty"` + PrimaryKeys *schemapb.IDs `protobuf:"bytes,6,opt,name=primary_keys,json=primaryKeys,proto3" json:"primary_keys,omitempty"` + Timestamps []uint64 `protobuf:"varint,7,rep,packed,name=timestamps,proto3" json:"timestamps,omitempty"` + Scope DataScope `protobuf:"varint,8,opt,name=scope,proto3,enum=milvus.proto.query.DataScope" json:"scope,omitempty"` +} + +func (x *DeleteBatchRequest) Reset() { + *x = DeleteBatchRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[60] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeleteBatchRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeleteBatchRequest) ProtoMessage() {} + +func (x *DeleteBatchRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[60] + 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 DeleteBatchRequest.ProtoReflect.Descriptor instead. +func (*DeleteBatchRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{60} +} + +func (x *DeleteBatchRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *DeleteBatchRequest) GetCollectionId() int64 { + if x != nil { + return x.CollectionId + } + return 0 +} + +func (x *DeleteBatchRequest) GetPartitionId() int64 { + if x != nil { + return x.PartitionId + } + return 0 +} + +func (x *DeleteBatchRequest) GetVchannelName() string { + if x != nil { + return x.VchannelName + } + return "" +} + +func (x *DeleteBatchRequest) GetSegmentIds() []int64 { + if x != nil { + return x.SegmentIds + } + return nil +} + +func (x *DeleteBatchRequest) GetPrimaryKeys() *schemapb.IDs { + if x != nil { + return x.PrimaryKeys + } + return nil +} + +func (x *DeleteBatchRequest) GetTimestamps() []uint64 { + if x != nil { + return x.Timestamps + } + return nil +} + +func (x *DeleteBatchRequest) GetScope() DataScope { + if x != nil { + return x.Scope + } + return DataScope_UnKnown +} + +// DeleteBatchResponse returns failed/missing segment ids +// cannot just using common.Status to handle partial failure logic +type DeleteBatchResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + FailedIds []int64 `protobuf:"varint,2,rep,packed,name=failed_ids,json=failedIds,proto3" json:"failed_ids,omitempty"` + MissingIds []int64 `protobuf:"varint,3,rep,packed,name=missing_ids,json=missingIds,proto3" json:"missing_ids,omitempty"` +} + +func (x *DeleteBatchResponse) Reset() { + *x = DeleteBatchResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[61] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeleteBatchResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeleteBatchResponse) ProtoMessage() {} + +func (x *DeleteBatchResponse) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[61] + 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 DeleteBatchResponse.ProtoReflect.Descriptor instead. +func (*DeleteBatchResponse) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{61} +} + +func (x *DeleteBatchResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *DeleteBatchResponse) GetFailedIds() []int64 { + if x != nil { + return x.FailedIds + } + return nil +} + +func (x *DeleteBatchResponse) GetMissingIds() []int64 { + if x != nil { + return x.MissingIds + } + return nil +} + +type ActivateCheckerRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + CheckerID int32 `protobuf:"varint,2,opt,name=checkerID,proto3" json:"checkerID,omitempty"` +} + +func (x *ActivateCheckerRequest) Reset() { + *x = ActivateCheckerRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[62] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ActivateCheckerRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ActivateCheckerRequest) ProtoMessage() {} + +func (x *ActivateCheckerRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[62] + 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 ActivateCheckerRequest.ProtoReflect.Descriptor instead. +func (*ActivateCheckerRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{62} +} + +func (x *ActivateCheckerRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *ActivateCheckerRequest) GetCheckerID() int32 { + if x != nil { + return x.CheckerID + } + return 0 +} + +type DeactivateCheckerRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + CheckerID int32 `protobuf:"varint,2,opt,name=checkerID,proto3" json:"checkerID,omitempty"` +} + +func (x *DeactivateCheckerRequest) Reset() { + *x = DeactivateCheckerRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[63] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeactivateCheckerRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeactivateCheckerRequest) ProtoMessage() {} + +func (x *DeactivateCheckerRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[63] + 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 DeactivateCheckerRequest.ProtoReflect.Descriptor instead. +func (*DeactivateCheckerRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{63} +} + +func (x *DeactivateCheckerRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *DeactivateCheckerRequest) GetCheckerID() int32 { + if x != nil { + return x.CheckerID + } + return 0 +} + +type ListCheckersRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + CheckerIDs []int32 `protobuf:"varint,2,rep,packed,name=checkerIDs,proto3" json:"checkerIDs,omitempty"` +} + +func (x *ListCheckersRequest) Reset() { + *x = ListCheckersRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[64] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListCheckersRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListCheckersRequest) ProtoMessage() {} + +func (x *ListCheckersRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[64] + 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 ListCheckersRequest.ProtoReflect.Descriptor instead. +func (*ListCheckersRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{64} +} + +func (x *ListCheckersRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *ListCheckersRequest) GetCheckerIDs() []int32 { + if x != nil { + return x.CheckerIDs + } + return nil +} + +type ListCheckersResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + CheckerInfos []*CheckerInfo `protobuf:"bytes,2,rep,name=checkerInfos,proto3" json:"checkerInfos,omitempty"` +} + +func (x *ListCheckersResponse) Reset() { + *x = ListCheckersResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[65] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListCheckersResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListCheckersResponse) ProtoMessage() {} + +func (x *ListCheckersResponse) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[65] + 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 ListCheckersResponse.ProtoReflect.Descriptor instead. +func (*ListCheckersResponse) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{65} +} + +func (x *ListCheckersResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *ListCheckersResponse) GetCheckerInfos() []*CheckerInfo { + if x != nil { + return x.CheckerInfos + } + return nil +} + +type CheckerInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Id int32 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` + Desc string `protobuf:"bytes,2,opt,name=desc,proto3" json:"desc,omitempty"` + Activated bool `protobuf:"varint,3,opt,name=activated,proto3" json:"activated,omitempty"` + Found bool `protobuf:"varint,4,opt,name=found,proto3" json:"found,omitempty"` +} + +func (x *CheckerInfo) Reset() { + *x = CheckerInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[66] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CheckerInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CheckerInfo) ProtoMessage() {} + +func (x *CheckerInfo) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[66] + 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 CheckerInfo.ProtoReflect.Descriptor instead. +func (*CheckerInfo) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{66} +} + +func (x *CheckerInfo) GetId() int32 { + if x != nil { + return x.Id + } + return 0 +} + +func (x *CheckerInfo) GetDesc() string { + if x != nil { + return x.Desc + } + return "" +} + +func (x *CheckerInfo) GetActivated() bool { + if x != nil { + return x.Activated + } + return false +} + +func (x *CheckerInfo) GetFound() bool { + if x != nil { + return x.Found + } + return false +} + +type SegmentTarget struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ID int64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"` + Level datapb.SegmentLevel `protobuf:"varint,2,opt,name=level,proto3,enum=milvus.proto.data.SegmentLevel" json:"level,omitempty"` + NumOfRows int64 `protobuf:"varint,3,opt,name=num_of_rows,json=numOfRows,proto3" json:"num_of_rows,omitempty"` +} + +func (x *SegmentTarget) Reset() { + *x = SegmentTarget{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[67] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentTarget) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentTarget) ProtoMessage() {} + +func (x *SegmentTarget) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[67] + 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 SegmentTarget.ProtoReflect.Descriptor instead. +func (*SegmentTarget) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{67} +} + +func (x *SegmentTarget) GetID() int64 { + if x != nil { + return x.ID + } + return 0 +} + +func (x *SegmentTarget) GetLevel() datapb.SegmentLevel { + if x != nil { + return x.Level + } + return datapb.SegmentLevel(0) +} + +func (x *SegmentTarget) GetNumOfRows() int64 { + if x != nil { + return x.NumOfRows + } + return 0 +} + +type PartitionTarget struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + PartitionID int64 `protobuf:"varint,1,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + Segments []*SegmentTarget `protobuf:"bytes,2,rep,name=segments,proto3" json:"segments,omitempty"` +} + +func (x *PartitionTarget) Reset() { + *x = PartitionTarget{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[68] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PartitionTarget) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PartitionTarget) ProtoMessage() {} + +func (x *PartitionTarget) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[68] + 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 PartitionTarget.ProtoReflect.Descriptor instead. +func (*PartitionTarget) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{68} +} + +func (x *PartitionTarget) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *PartitionTarget) GetSegments() []*SegmentTarget { + if x != nil { + return x.Segments + } + return nil +} + +type ChannelTarget struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ChannelName string `protobuf:"bytes,1,opt,name=channelName,proto3" json:"channelName,omitempty"` + DroppedSegmentIDs []int64 `protobuf:"varint,2,rep,packed,name=dropped_segmentIDs,json=droppedSegmentIDs,proto3" json:"dropped_segmentIDs,omitempty"` + GrowingSegmentIDs []int64 `protobuf:"varint,3,rep,packed,name=growing_segmentIDs,json=growingSegmentIDs,proto3" json:"growing_segmentIDs,omitempty"` + PartitionTargets []*PartitionTarget `protobuf:"bytes,4,rep,name=partition_targets,json=partitionTargets,proto3" json:"partition_targets,omitempty"` + SeekPosition *msgpb.MsgPosition `protobuf:"bytes,5,opt,name=seek_position,json=seekPosition,proto3" json:"seek_position,omitempty"` +} + +func (x *ChannelTarget) Reset() { + *x = ChannelTarget{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[69] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ChannelTarget) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ChannelTarget) ProtoMessage() {} + +func (x *ChannelTarget) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[69] + 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 ChannelTarget.ProtoReflect.Descriptor instead. +func (*ChannelTarget) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{69} +} + +func (x *ChannelTarget) GetChannelName() string { + if x != nil { + return x.ChannelName + } + return "" +} + +func (x *ChannelTarget) GetDroppedSegmentIDs() []int64 { + if x != nil { + return x.DroppedSegmentIDs + } + return nil +} + +func (x *ChannelTarget) GetGrowingSegmentIDs() []int64 { + if x != nil { + return x.GrowingSegmentIDs + } + return nil +} + +func (x *ChannelTarget) GetPartitionTargets() []*PartitionTarget { + if x != nil { + return x.PartitionTargets + } + return nil +} + +func (x *ChannelTarget) GetSeekPosition() *msgpb.MsgPosition { + if x != nil { + return x.SeekPosition + } + return nil +} + +type CollectionTarget struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + ChannelTargets []*ChannelTarget `protobuf:"bytes,2,rep,name=Channel_targets,json=ChannelTargets,proto3" json:"Channel_targets,omitempty"` + Version int64 `protobuf:"varint,3,opt,name=version,proto3" json:"version,omitempty"` +} + +func (x *CollectionTarget) Reset() { + *x = CollectionTarget{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[70] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CollectionTarget) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CollectionTarget) ProtoMessage() {} + +func (x *CollectionTarget) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[70] + 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 CollectionTarget.ProtoReflect.Descriptor instead. +func (*CollectionTarget) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{70} +} + +func (x *CollectionTarget) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *CollectionTarget) GetChannelTargets() []*ChannelTarget { + if x != nil { + return x.ChannelTargets + } + return nil +} + +func (x *CollectionTarget) GetVersion() int64 { + if x != nil { + return x.Version + } + return 0 +} + +type NodeInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ID int64 `protobuf:"varint,2,opt,name=ID,proto3" json:"ID,omitempty"` + Address string `protobuf:"bytes,3,opt,name=address,proto3" json:"address,omitempty"` + State string `protobuf:"bytes,4,opt,name=state,proto3" json:"state,omitempty"` +} + +func (x *NodeInfo) Reset() { + *x = NodeInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[71] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NodeInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NodeInfo) ProtoMessage() {} + +func (x *NodeInfo) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[71] + 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 NodeInfo.ProtoReflect.Descriptor instead. +func (*NodeInfo) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{71} +} + +func (x *NodeInfo) GetID() int64 { + if x != nil { + return x.ID + } + return 0 +} + +func (x *NodeInfo) GetAddress() string { + if x != nil { + return x.Address + } + return "" +} + +func (x *NodeInfo) GetState() string { + if x != nil { + return x.State + } + return "" +} + +type ListQueryNodeRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` +} + +func (x *ListQueryNodeRequest) Reset() { + *x = ListQueryNodeRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[72] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListQueryNodeRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListQueryNodeRequest) ProtoMessage() {} + +func (x *ListQueryNodeRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[72] + 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 ListQueryNodeRequest.ProtoReflect.Descriptor instead. +func (*ListQueryNodeRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{72} +} + +func (x *ListQueryNodeRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +type ListQueryNodeResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + NodeInfos []*NodeInfo `protobuf:"bytes,2,rep,name=nodeInfos,proto3" json:"nodeInfos,omitempty"` +} + +func (x *ListQueryNodeResponse) Reset() { + *x = ListQueryNodeResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[73] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListQueryNodeResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListQueryNodeResponse) ProtoMessage() {} + +func (x *ListQueryNodeResponse) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[73] + 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 ListQueryNodeResponse.ProtoReflect.Descriptor instead. +func (*ListQueryNodeResponse) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{73} +} + +func (x *ListQueryNodeResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *ListQueryNodeResponse) GetNodeInfos() []*NodeInfo { + if x != nil { + return x.NodeInfos + } + return nil +} + +type GetQueryNodeDistributionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + NodeID int64 `protobuf:"varint,2,opt,name=nodeID,proto3" json:"nodeID,omitempty"` +} + +func (x *GetQueryNodeDistributionRequest) Reset() { + *x = GetQueryNodeDistributionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[74] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetQueryNodeDistributionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetQueryNodeDistributionRequest) ProtoMessage() {} + +func (x *GetQueryNodeDistributionRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[74] + 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 GetQueryNodeDistributionRequest.ProtoReflect.Descriptor instead. +func (*GetQueryNodeDistributionRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{74} +} + +func (x *GetQueryNodeDistributionRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *GetQueryNodeDistributionRequest) GetNodeID() int64 { + if x != nil { + return x.NodeID + } + return 0 +} + +type GetQueryNodeDistributionResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + ID int64 `protobuf:"varint,2,opt,name=ID,proto3" json:"ID,omitempty"` + ChannelNames []string `protobuf:"bytes,3,rep,name=channel_names,json=channelNames,proto3" json:"channel_names,omitempty"` + SealedSegmentIDs []int64 `protobuf:"varint,4,rep,packed,name=sealed_segmentIDs,json=sealedSegmentIDs,proto3" json:"sealed_segmentIDs,omitempty"` +} + +func (x *GetQueryNodeDistributionResponse) Reset() { + *x = GetQueryNodeDistributionResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[75] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetQueryNodeDistributionResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetQueryNodeDistributionResponse) ProtoMessage() {} + +func (x *GetQueryNodeDistributionResponse) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[75] + 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 GetQueryNodeDistributionResponse.ProtoReflect.Descriptor instead. +func (*GetQueryNodeDistributionResponse) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{75} +} + +func (x *GetQueryNodeDistributionResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *GetQueryNodeDistributionResponse) GetID() int64 { + if x != nil { + return x.ID + } + return 0 +} + +func (x *GetQueryNodeDistributionResponse) GetChannelNames() []string { + if x != nil { + return x.ChannelNames + } + return nil +} + +func (x *GetQueryNodeDistributionResponse) GetSealedSegmentIDs() []int64 { + if x != nil { + return x.SealedSegmentIDs + } + return nil +} + +type SuspendBalanceRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` +} + +func (x *SuspendBalanceRequest) Reset() { + *x = SuspendBalanceRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[76] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SuspendBalanceRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SuspendBalanceRequest) ProtoMessage() {} + +func (x *SuspendBalanceRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[76] + 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 SuspendBalanceRequest.ProtoReflect.Descriptor instead. +func (*SuspendBalanceRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{76} +} + +func (x *SuspendBalanceRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +type ResumeBalanceRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` +} + +func (x *ResumeBalanceRequest) Reset() { + *x = ResumeBalanceRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[77] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ResumeBalanceRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResumeBalanceRequest) ProtoMessage() {} + +func (x *ResumeBalanceRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[77] + 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 ResumeBalanceRequest.ProtoReflect.Descriptor instead. +func (*ResumeBalanceRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{77} +} + +func (x *ResumeBalanceRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +type SuspendNodeRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + NodeID int64 `protobuf:"varint,2,opt,name=nodeID,proto3" json:"nodeID,omitempty"` +} + +func (x *SuspendNodeRequest) Reset() { + *x = SuspendNodeRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[78] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SuspendNodeRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SuspendNodeRequest) ProtoMessage() {} + +func (x *SuspendNodeRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[78] + 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 SuspendNodeRequest.ProtoReflect.Descriptor instead. +func (*SuspendNodeRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{78} +} + +func (x *SuspendNodeRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *SuspendNodeRequest) GetNodeID() int64 { + if x != nil { + return x.NodeID + } + return 0 +} + +type ResumeNodeRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + NodeID int64 `protobuf:"varint,2,opt,name=nodeID,proto3" json:"nodeID,omitempty"` +} + +func (x *ResumeNodeRequest) Reset() { + *x = ResumeNodeRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[79] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ResumeNodeRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResumeNodeRequest) ProtoMessage() {} + +func (x *ResumeNodeRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[79] + 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 ResumeNodeRequest.ProtoReflect.Descriptor instead. +func (*ResumeNodeRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{79} +} + +func (x *ResumeNodeRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *ResumeNodeRequest) GetNodeID() int64 { + if x != nil { + return x.NodeID + } + return 0 +} + +type TransferSegmentRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + SegmentID int64 `protobuf:"varint,2,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + SourceNodeID int64 `protobuf:"varint,3,opt,name=source_nodeID,json=sourceNodeID,proto3" json:"source_nodeID,omitempty"` + TargetNodeID int64 `protobuf:"varint,4,opt,name=target_nodeID,json=targetNodeID,proto3" json:"target_nodeID,omitempty"` + TransferAll bool `protobuf:"varint,5,opt,name=transfer_all,json=transferAll,proto3" json:"transfer_all,omitempty"` + ToAllNodes bool `protobuf:"varint,6,opt,name=to_all_nodes,json=toAllNodes,proto3" json:"to_all_nodes,omitempty"` + CopyMode bool `protobuf:"varint,7,opt,name=copy_mode,json=copyMode,proto3" json:"copy_mode,omitempty"` +} + +func (x *TransferSegmentRequest) Reset() { + *x = TransferSegmentRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[80] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TransferSegmentRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TransferSegmentRequest) ProtoMessage() {} + +func (x *TransferSegmentRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[80] + 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 TransferSegmentRequest.ProtoReflect.Descriptor instead. +func (*TransferSegmentRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{80} +} + +func (x *TransferSegmentRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *TransferSegmentRequest) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *TransferSegmentRequest) GetSourceNodeID() int64 { + if x != nil { + return x.SourceNodeID + } + return 0 +} + +func (x *TransferSegmentRequest) GetTargetNodeID() int64 { + if x != nil { + return x.TargetNodeID + } + return 0 +} + +func (x *TransferSegmentRequest) GetTransferAll() bool { + if x != nil { + return x.TransferAll + } + return false +} + +func (x *TransferSegmentRequest) GetToAllNodes() bool { + if x != nil { + return x.ToAllNodes + } + return false +} + +func (x *TransferSegmentRequest) GetCopyMode() bool { + if x != nil { + return x.CopyMode + } + return false +} + +type TransferChannelRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + ChannelName string `protobuf:"bytes,2,opt,name=channel_name,json=channelName,proto3" json:"channel_name,omitempty"` + SourceNodeID int64 `protobuf:"varint,3,opt,name=source_nodeID,json=sourceNodeID,proto3" json:"source_nodeID,omitempty"` + TargetNodeID int64 `protobuf:"varint,4,opt,name=target_nodeID,json=targetNodeID,proto3" json:"target_nodeID,omitempty"` + TransferAll bool `protobuf:"varint,5,opt,name=transfer_all,json=transferAll,proto3" json:"transfer_all,omitempty"` + ToAllNodes bool `protobuf:"varint,6,opt,name=to_all_nodes,json=toAllNodes,proto3" json:"to_all_nodes,omitempty"` + CopyMode bool `protobuf:"varint,7,opt,name=copy_mode,json=copyMode,proto3" json:"copy_mode,omitempty"` +} + +func (x *TransferChannelRequest) Reset() { + *x = TransferChannelRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[81] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TransferChannelRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TransferChannelRequest) ProtoMessage() {} + +func (x *TransferChannelRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[81] + 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 TransferChannelRequest.ProtoReflect.Descriptor instead. +func (*TransferChannelRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{81} +} + +func (x *TransferChannelRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *TransferChannelRequest) GetChannelName() string { + if x != nil { + return x.ChannelName + } + return "" +} + +func (x *TransferChannelRequest) GetSourceNodeID() int64 { + if x != nil { + return x.SourceNodeID + } + return 0 +} + +func (x *TransferChannelRequest) GetTargetNodeID() int64 { + if x != nil { + return x.TargetNodeID + } + return 0 +} + +func (x *TransferChannelRequest) GetTransferAll() bool { + if x != nil { + return x.TransferAll + } + return false +} + +func (x *TransferChannelRequest) GetToAllNodes() bool { + if x != nil { + return x.ToAllNodes + } + return false +} + +func (x *TransferChannelRequest) GetCopyMode() bool { + if x != nil { + return x.CopyMode + } + return false +} + +type CheckQueryNodeDistributionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + SourceNodeID int64 `protobuf:"varint,3,opt,name=source_nodeID,json=sourceNodeID,proto3" json:"source_nodeID,omitempty"` + TargetNodeID int64 `protobuf:"varint,4,opt,name=target_nodeID,json=targetNodeID,proto3" json:"target_nodeID,omitempty"` +} + +func (x *CheckQueryNodeDistributionRequest) Reset() { + *x = CheckQueryNodeDistributionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[82] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CheckQueryNodeDistributionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CheckQueryNodeDistributionRequest) ProtoMessage() {} + +func (x *CheckQueryNodeDistributionRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[82] + 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 CheckQueryNodeDistributionRequest.ProtoReflect.Descriptor instead. +func (*CheckQueryNodeDistributionRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{82} +} + +func (x *CheckQueryNodeDistributionRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *CheckQueryNodeDistributionRequest) GetSourceNodeID() int64 { + if x != nil { + return x.SourceNodeID + } + return 0 +} + +func (x *CheckQueryNodeDistributionRequest) GetTargetNodeID() int64 { + if x != nil { + return x.TargetNodeID + } + return 0 +} + +type UpdateLoadConfigRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + DbID int64 `protobuf:"varint,2,opt,name=dbID,proto3" json:"dbID,omitempty"` + CollectionIDs []int64 `protobuf:"varint,3,rep,packed,name=collectionIDs,proto3" json:"collectionIDs,omitempty"` + ReplicaNumber int32 `protobuf:"varint,4,opt,name=replica_number,json=replicaNumber,proto3" json:"replica_number,omitempty"` + ResourceGroups []string `protobuf:"bytes,5,rep,name=resource_groups,json=resourceGroups,proto3" json:"resource_groups,omitempty"` +} + +func (x *UpdateLoadConfigRequest) Reset() { + *x = UpdateLoadConfigRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[83] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateLoadConfigRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateLoadConfigRequest) ProtoMessage() {} + +func (x *UpdateLoadConfigRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[83] + 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 UpdateLoadConfigRequest.ProtoReflect.Descriptor instead. +func (*UpdateLoadConfigRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{83} +} + +func (x *UpdateLoadConfigRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *UpdateLoadConfigRequest) GetDbID() int64 { + if x != nil { + return x.DbID + } + return 0 +} + +func (x *UpdateLoadConfigRequest) GetCollectionIDs() []int64 { + if x != nil { + return x.CollectionIDs + } + return nil +} + +func (x *UpdateLoadConfigRequest) GetReplicaNumber() int32 { + if x != nil { + return x.ReplicaNumber + } + return 0 +} + +func (x *UpdateLoadConfigRequest) GetResourceGroups() []string { + if x != nil { + return x.ResourceGroups + } + return nil +} + +var File_query_coord_proto protoreflect.FileDescriptor + +var file_query_coord_proto_rawDesc = []byte{ + 0x0a, 0x11, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x12, 0x12, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x1a, 0x0c, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x0c, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x1a, 0x08, 0x72, 0x67, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x0e, 0x69, + 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x0c, 0x73, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x09, 0x6d, 0x73, 0x67, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x10, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x63, 0x6f, 0x6f, + 0x72, 0x64, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x11, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, + 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x84, 0x01, 0x0a, 0x16, + 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, + 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x62, 0x49, 0x44, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x44, 0x12, 0x24, 0x0a, 0x0d, + 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x03, 0x20, + 0x03, 0x28, 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x44, 0x73, 0x22, 0xcb, 0x02, 0x0a, 0x17, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, + 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x12, 0x24, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x12, 0x31, 0x0a, 0x14, 0x69, 0x6e, 0x4d, + 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x5f, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, + 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x13, 0x69, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, + 0x79, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x73, 0x12, 0x36, 0x0a, 0x17, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x5f, 0x61, 0x76, + 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x04, 0x20, 0x03, 0x28, 0x08, 0x52, 0x15, 0x71, + 0x75, 0x65, 0x72, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x76, 0x61, 0x69, 0x6c, + 0x61, 0x62, 0x6c, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x72, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x5f, + 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0f, + 0x72, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, + 0x3f, 0x0a, 0x0b, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x06, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x4c, 0x6f, 0x6e, 0x67, 0x41, + 0x72, 0x72, 0x61, 0x79, 0x52, 0x0a, 0x6c, 0x6f, 0x61, 0x64, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, + 0x22, 0xa5, 0x01, 0x0a, 0x15, 0x53, 0x68, 0x6f, 0x77, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, + 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, + 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, + 0x64, 0x62, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x44, + 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x44, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x22, 0xcf, 0x01, 0x0a, 0x16, 0x53, 0x68, 0x6f, + 0x77, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x22, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0c, + 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x12, 0x31, 0x0a, 0x14, + 0x69, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x5f, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, + 0x61, 0x67, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x13, 0x69, 0x6e, 0x4d, 0x65, + 0x6d, 0x6f, 0x72, 0x79, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x73, 0x12, + 0x29, 0x0a, 0x10, 0x72, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x5f, 0x70, 0x72, 0x6f, 0x67, 0x72, + 0x65, 0x73, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0f, 0x72, 0x65, 0x66, 0x72, 0x65, + 0x73, 0x68, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x22, 0xee, 0x03, 0x0a, 0x15, 0x4c, + 0x6f, 0x61, 0x64, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, + 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x62, 0x49, 0x44, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x03, 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, 0x04, 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, 0x25, 0x0a, + 0x0e, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x4e, 0x75, + 0x6d, 0x62, 0x65, 0x72, 0x12, 0x60, 0x0a, 0x0d, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x49, 0x44, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3b, 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, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x64, 0x65, + 0x78, 0x49, 0x44, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, + 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x72, 0x65, 0x66, 0x72, 0x65, 0x73, + 0x68, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x72, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, + 0x12, 0x27, 0x0a, 0x0f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x67, 0x72, 0x6f, + 0x75, 0x70, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x72, 0x65, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x6c, 0x6f, 0x61, + 0x64, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, + 0x6c, 0x6f, 0x61, 0x64, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x1a, 0x3f, 0x0a, 0x11, 0x46, 0x69, + 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, + 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, + 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x9c, 0x01, 0x0a, 0x18, + 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, + 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x62, + 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x44, 0x12, 0x22, + 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x44, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x22, 0xf9, 0x01, 0x0a, 0x14, 0x47, + 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x3d, 0x0a, 0x03, 0x72, 0x65, 0x71, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x2b, 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x03, 0x72, + 0x65, 0x71, 0x12, 0x21, 0x0a, 0x0c, 0x64, 0x6d, 0x6c, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0b, 0x64, 0x6d, 0x6c, 0x43, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x49, 0x44, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, 0x2a, 0x0a, 0x11, 0x66, 0x72, 0x6f, 0x6d, 0x5f, 0x73, 0x68, + 0x61, 0x72, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x0f, 0x66, 0x72, 0x6f, 0x6d, 0x53, 0x68, 0x61, 0x72, 0x64, 0x4c, 0x65, 0x61, 0x64, 0x65, + 0x72, 0x12, 0x33, 0x0a, 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x52, + 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x22, 0xd9, 0x04, 0x0a, 0x15, 0x4c, 0x6f, 0x61, 0x64, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, + 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x62, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x04, 0x64, 0x62, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x70, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x03, + 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x12, 0x3d, + 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x05, 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, 0x25, 0x0a, + 0x0e, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, + 0x06, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x4e, 0x75, + 0x6d, 0x62, 0x65, 0x72, 0x12, 0x60, 0x0a, 0x0d, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x49, 0x44, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3b, 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, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x64, 0x65, + 0x78, 0x49, 0x44, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, + 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x72, 0x65, 0x66, 0x72, 0x65, 0x73, + 0x68, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x72, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, + 0x12, 0x27, 0x0a, 0x0f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x67, 0x72, 0x6f, + 0x75, 0x70, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x72, 0x65, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x12, 0x45, 0x0a, 0x0f, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x18, 0x0a, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, + 0x6f, 0x52, 0x0d, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x4c, 0x69, 0x73, 0x74, + 0x12, 0x1f, 0x0a, 0x0b, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, + 0x0b, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x6c, 0x6f, 0x61, 0x64, 0x46, 0x69, 0x65, 0x6c, 0x64, + 0x73, 0x1a, 0x3f, 0x0a, 0x11, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, + 0x44, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, + 0x38, 0x01, 0x22, 0xc0, 0x01, 0x0a, 0x18, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, + 0x65, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x62, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x04, 0x64, 0x62, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x70, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x03, 0x52, + 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x12, 0x16, 0x0a, + 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, + 0x6f, 0x64, 0x65, 0x49, 0x44, 0x22, 0xa9, 0x01, 0x0a, 0x19, 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, 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, 0x12, 0x0a, 0x04, 0x64, 0x62, 0x49, 0x44, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x22, 0x0a, + 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x04, 0x20, + 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, + 0x73, 0x22, 0xad, 0x01, 0x0a, 0x1a, 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, + 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x5a, 0x0a, 0x16, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, + 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x15, 0x70, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x22, 0x8d, 0x01, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x49, 0x6e, 0x66, 0x6f, 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, 0x1e, 0x0a, + 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, + 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, 0x22, 0x0a, + 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x44, 0x22, 0x84, 0x01, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, + 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x12, 0x35, 0x0a, 0x05, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, + 0x6f, 0x52, 0x05, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x22, 0x6e, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x53, + 0x68, 0x61, 0x72, 0x64, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, + 0x62, 0x61, 0x73, 0x65, 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, 0x22, 0x8c, 0x01, 0x0a, 0x17, 0x47, 0x65, 0x74, + 0x53, 0x68, 0x61, 0x72, 0x64, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x3c, 0x0a, 0x06, 0x73, 0x68, 0x61, + 0x72, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, + 0x68, 0x61, 0x72, 0x64, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x4c, 0x69, 0x73, 0x74, 0x52, + 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x22, 0xa6, 0x02, 0x0a, 0x1b, 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, 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, 0x6c, 0x0a, 0x0f, 0x72, 0x65, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x18, 0x02, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x43, 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, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, + 0x70, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x1a, 0x67, 0x0a, 0x13, 0x52, 0x65, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 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, 0x3a, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, + 0x67, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x43, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, + 0x22, 0x6f, 0x0a, 0x10, 0x53, 0x68, 0x61, 0x72, 0x64, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, + 0x4c, 0x69, 0x73, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x5f, + 0x69, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x07, 0x6e, 0x6f, 0x64, 0x65, 0x49, + 0x64, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x6e, 0x6f, 0x64, 0x65, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x73, + 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x6f, 0x64, 0x65, 0x41, 0x64, 0x64, 0x72, + 0x73, 0x22, 0x98, 0x01, 0x0a, 0x1e, 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, 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, 0x20, 0x0a, 0x0b, 0x70, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x22, 0xdf, 0x02, 0x0a, + 0x0c, 0x4c, 0x6f, 0x61, 0x64, 0x4d, 0x65, 0x74, 0x61, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x39, 0x0a, + 0x09, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x54, 0x79, 0x70, 0x65, 0x52, 0x08, + 0x6c, 0x6f, 0x61, 0x64, 0x54, 0x79, 0x70, 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, 0x22, 0x0a, 0x0c, + 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x03, 0x20, 0x03, + 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, + 0x12, 0x23, 0x0a, 0x0b, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x09, 0x42, 0x02, 0x18, 0x01, 0x52, 0x0a, 0x6d, 0x65, 0x74, 0x72, 0x69, + 0x63, 0x54, 0x79, 0x70, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x25, + 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, + 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x1f, 0x0a, 0x0b, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x66, 0x69, + 0x65, 0x6c, 0x64, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x6c, 0x6f, 0x61, 0x64, + 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x46, 0x0a, 0x0d, 0x64, 0x62, 0x5f, 0x70, 0x72, 0x6f, + 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, + 0x52, 0x0c, 0x64, 0x62, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x22, 0x8d, + 0x06, 0x0a, 0x16, 0x57, 0x61, 0x74, 0x63, 0x68, 0x44, 0x6d, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, + 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x6e, + 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, + 0x65, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x12, 0x35, 0x0a, 0x05, 0x69, + 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, + 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x05, 0x69, 0x6e, 0x66, + 0x6f, 0x73, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x06, 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, 0x43, 0x0a, 0x0d, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x66, + 0x6f, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0c, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, + 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x3d, 0x0a, 0x09, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x6d, + 0x65, 0x74, 0x61, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, + 0x6f, 0x61, 0x64, 0x4d, 0x65, 0x74, 0x61, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x6c, 0x6f, 0x61, + 0x64, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x1c, 0x0a, 0x09, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, + 0x49, 0x44, 0x18, 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, + 0x61, 0x49, 0x44, 0x12, 0x61, 0x0a, 0x0d, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, + 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3c, 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, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, + 0x66, 0x6f, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x24, 0x0a, 0x0d, 0x6f, 0x66, 0x66, 0x6c, 0x69, 0x6e, + 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x6f, + 0x66, 0x66, 0x6c, 0x69, 0x6e, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, + 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x45, 0x0a, 0x0f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, + 0x69, 0x6e, 0x66, 0x6f, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x18, 0x0d, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0d, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x4c, 0x69, 0x73, 0x74, 0x1a, 0x5f, 0x0a, + 0x11, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x03, 0x6b, 0x65, 0x79, 0x12, 0x34, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, + 0x6e, 0x66, 0x6f, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xa8, + 0x01, 0x0a, 0x15, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x44, 0x6d, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 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, 0x16, 0x0a, 0x06, 0x6e, 0x6f, + 0x64, 0x65, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, + 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xf3, 0x08, 0x0a, 0x0f, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1c, 0x0a, + 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x22, 0x0a, + 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x44, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x62, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x04, 0x64, 0x62, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x5f, 0x74, + 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x66, 0x6c, 0x75, 0x73, 0x68, + 0x54, 0x69, 0x6d, 0x65, 0x12, 0x41, 0x0a, 0x0c, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x5f, 0x70, + 0x61, 0x74, 0x68, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, + 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x0b, 0x62, 0x69, 0x6e, 0x6c, + 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, 0x12, 0x1e, 0x0a, 0x0b, 0x6e, 0x75, 0x6d, 0x5f, 0x6f, + 0x66, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x6e, 0x75, + 0x6d, 0x4f, 0x66, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x3c, 0x0a, 0x09, 0x73, 0x74, 0x61, 0x74, 0x73, + 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, + 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, + 0x73, 0x6c, 0x6f, 0x67, 0x73, 0x12, 0x3c, 0x0a, 0x09, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x6c, 0x6f, + 0x67, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, + 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x09, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x6c, + 0x6f, 0x67, 0x73, 0x12, 0x26, 0x0a, 0x0e, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x46, 0x72, 0x6f, 0x6d, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0e, 0x63, 0x6f, 0x6d, + 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x72, 0x6f, 0x6d, 0x12, 0x43, 0x0a, 0x0b, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x0b, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, + 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x73, + 0x12, 0x25, 0x0a, 0x0c, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x73, 0x69, 0x7a, 0x65, + 0x18, 0x0c, 0x20, 0x01, 0x28, 0x03, 0x42, 0x02, 0x18, 0x01, 0x52, 0x0b, 0x73, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x73, 0x65, 0x72, + 0x74, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0d, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x44, + 0x0a, 0x0e, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x4d, 0x73, 0x67, 0x50, 0x6f, 0x73, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0d, 0x73, 0x74, 0x61, 0x72, 0x74, 0x50, 0x6f, 0x73, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x44, 0x0a, 0x0e, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x5f, 0x70, 0x6f, + 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, + 0x4d, 0x73, 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0d, 0x64, 0x65, 0x6c, + 0x74, 0x61, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x28, 0x0a, 0x0f, 0x72, 0x65, + 0x61, 0x64, 0x61, 0x62, 0x6c, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x10, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0f, 0x72, 0x65, 0x61, 0x64, 0x61, 0x62, 0x6c, 0x65, 0x56, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x35, 0x0a, 0x05, 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x18, 0x11, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4c, + 0x65, 0x76, 0x65, 0x6c, 0x52, 0x05, 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x12, 0x26, 0x0a, 0x0e, 0x73, + 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x12, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0e, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x56, 0x65, 0x72, 0x73, + 0x69, 0x6f, 0x6e, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x73, 0x6f, 0x72, 0x74, 0x65, 0x64, + 0x18, 0x13, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x53, 0x6f, 0x72, 0x74, 0x65, 0x64, + 0x12, 0x5c, 0x0a, 0x0d, 0x74, 0x65, 0x78, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, + 0x73, 0x18, 0x14, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x54, 0x65, 0x78, + 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, + 0x0d, 0x74, 0x65, 0x78, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x3a, + 0x0a, 0x08, 0x62, 0x6d, 0x32, 0x35, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x15, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, + 0x52, 0x08, 0x62, 0x6d, 0x32, 0x35, 0x6c, 0x6f, 0x67, 0x73, 0x1a, 0x63, 0x0a, 0x12, 0x54, 0x65, + 0x78, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, + 0x65, 0x79, 0x12, 0x37, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x65, 0x78, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, + 0x74, 0x61, 0x74, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, + 0xd3, 0x03, 0x0a, 0x0e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, + 0x66, 0x6f, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x21, 0x0a, 0x0c, + 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0b, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, + 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x18, + 0x0a, 0x07, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x07, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x62, 0x75, 0x69, 0x6c, + 0x64, 0x49, 0x44, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, + 0x49, 0x44, 0x12, 0x44, 0x0a, 0x0c, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, + 0x6d, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, + 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0b, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x28, 0x0a, 0x10, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x73, 0x18, 0x07, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x0e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x46, 0x69, 0x6c, 0x65, 0x50, 0x61, 0x74, + 0x68, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x73, 0x69, 0x7a, 0x65, + 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x69, 0x7a, + 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x18, 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x56, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, + 0x77, 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, + 0x73, 0x12, 0x32, 0x0a, 0x15, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x05, + 0x52, 0x13, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x56, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x2e, 0x0a, 0x13, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x73, + 0x74, 0x6f, 0x72, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x0c, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x11, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x6f, 0x72, 0x65, 0x56, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0xaf, 0x05, 0x0a, 0x13, 0x4c, 0x6f, 0x61, 0x64, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, + 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, + 0x1d, 0x0a, 0x0a, 0x64, 0x73, 0x74, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x09, 0x64, 0x73, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x39, + 0x0a, 0x05, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, + 0x72, 0x79, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x49, 0x6e, + 0x66, 0x6f, 0x52, 0x05, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x18, 0x04, 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, 0x23, 0x0a, 0x0d, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0c, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x22, 0x0a, + 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x44, 0x12, 0x3d, 0x0a, 0x09, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x18, 0x07, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x4d, 0x65, + 0x74, 0x61, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x6c, 0x6f, 0x61, 0x64, 0x4d, 0x65, 0x74, 0x61, + 0x12, 0x1c, 0x0a, 0x09, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x49, 0x44, 0x18, 0x08, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x09, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x49, 0x44, 0x12, 0x46, + 0x0a, 0x0f, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x4d, 0x73, 0x67, 0x50, 0x6f, + 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x50, 0x6f, 0x73, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x12, 0x23, 0x0a, 0x0d, 0x6e, 0x65, 0x65, 0x64, 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, + 0x72, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x6e, 0x65, 0x65, 0x64, 0x54, 0x72, 0x61, + 0x6e, 0x73, 0x66, 0x65, 0x72, 0x12, 0x3c, 0x0a, 0x0a, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x73, 0x63, + 0x6f, 0x70, 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1d, 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, 0x63, 0x6f, 0x70, 0x65, 0x52, 0x09, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, + 0x6f, 0x70, 0x65, 0x12, 0x45, 0x0a, 0x0f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x69, 0x6e, 0x66, + 0x6f, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x18, 0x0d, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0d, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x6c, 0x61, + 0x7a, 0x79, 0x5f, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x6c, + 0x61, 0x7a, 0x79, 0x4c, 0x6f, 0x61, 0x64, 0x22, 0x8d, 0x03, 0x0a, 0x16, 0x52, 0x65, 0x6c, 0x65, + 0x61, 0x73, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, + 0x62, 0x61, 0x73, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x12, 0x0a, 0x04, + 0x64, 0x62, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x44, + 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x44, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, 0x33, 0x0a, 0x05, 0x73, 0x63, 0x6f, 0x70, + 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x44, 0x61, 0x74, + 0x61, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x52, 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x14, 0x0a, + 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, + 0x61, 0x72, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x6e, 0x65, 0x65, 0x64, 0x5f, 0x74, 0x72, 0x61, 0x6e, + 0x73, 0x66, 0x65, 0x72, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x6e, 0x65, 0x65, 0x64, + 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x12, 0x3d, 0x0a, 0x0a, 0x63, 0x68, 0x65, 0x63, + 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, + 0x4d, 0x73, 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x63, 0x68, 0x65, + 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x22, 0x97, 0x02, 0x0a, 0x0d, 0x53, 0x65, 0x61, 0x72, + 0x63, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x36, 0x0a, 0x03, 0x72, 0x65, 0x71, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x03, 0x72, 0x65, + 0x71, 0x12, 0x21, 0x0a, 0x0c, 0x64, 0x6d, 0x6c, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, + 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0b, 0x64, 0x6d, 0x6c, 0x43, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, + 0x44, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x44, 0x73, 0x12, 0x2a, 0x0a, 0x11, 0x66, 0x72, 0x6f, 0x6d, 0x5f, 0x73, 0x68, 0x61, + 0x72, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x0f, 0x66, 0x72, 0x6f, 0x6d, 0x53, 0x68, 0x61, 0x72, 0x64, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, + 0x12, 0x33, 0x0a, 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, + 0x75, 0x65, 0x72, 0x79, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x52, 0x05, + 0x73, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x2a, 0x0a, 0x11, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x63, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x6e, 0x75, 0x6d, 0x18, 0x06, 0x20, 0x01, 0x28, 0x05, + 0x52, 0x0f, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x75, + 0x6d, 0x22, 0xec, 0x01, 0x0a, 0x0c, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x38, 0x0a, 0x03, 0x72, 0x65, 0x71, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x03, 0x72, 0x65, 0x71, 0x12, 0x21, 0x0a, 0x0c, + 0x64, 0x6d, 0x6c, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x0b, 0x64, 0x6d, 0x6c, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, + 0x1e, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x03, 0x20, + 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, + 0x2a, 0x0a, 0x11, 0x66, 0x72, 0x6f, 0x6d, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x6c, 0x65, + 0x61, 0x64, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x66, 0x72, 0x6f, 0x6d, + 0x53, 0x68, 0x61, 0x72, 0x64, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x33, 0x0a, 0x05, 0x73, + 0x63, 0x6f, 0x70, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, + 0x44, 0x61, 0x74, 0x61, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x52, 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, + 0x22, 0xc7, 0x01, 0x0a, 0x1a, 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, 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, 0x23, 0x0a, 0x0d, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x52, 0x0a, 0x10, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, + 0x61, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x53, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0f, 0x72, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x8e, 0x01, 0x0a, 0x13, 0x52, + 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x49, 0x6e, + 0x66, 0x6f, 0x12, 0x17, 0x0a, 0x07, 0x6e, 0x6f, 0x64, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 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, 0x12, 0x1f, + 0x0a, 0x0b, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x03, 0x20, + 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x73, 0x12, + 0x1a, 0x0a, 0x08, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, + 0x03, 0x52, 0x08, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x6b, 0x0a, 0x12, 0x47, + 0x65, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x49, 0x6e, 0x66, 0x6f, 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, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0xe4, 0x01, 0x0a, 0x13, 0x47, 0x65, 0x74, + 0x4c, 0x6f, 0x61, 0x64, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, + 0x02, 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, 0x39, 0x0a, 0x09, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x74, 0x79, 0x70, + 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, + 0x64, 0x54, 0x79, 0x70, 0x65, 0x52, 0x08, 0x6c, 0x6f, 0x61, 0x64, 0x54, 0x79, 0x70, 0x65, 0x12, + 0x1e, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, + 0x03, 0x28, 0x03, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, + 0xbc, 0x01, 0x0a, 0x16, 0x48, 0x61, 0x6e, 0x64, 0x6f, 0x66, 0x66, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, + 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, + 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x43, 0x0a, 0x0c, + 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x02, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, + 0x6e, 0x66, 0x6f, 0x52, 0x0c, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, + 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x72, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x64, 0x5f, 0x73, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x10, 0x72, 0x65, + 0x6c, 0x65, 0x61, 0x73, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0xac, + 0x02, 0x0a, 0x12, 0x4c, 0x6f, 0x61, 0x64, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 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, 0x25, 0x0a, 0x0e, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, + 0x0d, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x73, 0x12, 0x4b, + 0x0a, 0x0e, 0x62, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54, 0x72, 0x69, 0x67, + 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0d, 0x62, 0x61, + 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x52, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x64, + 0x73, 0x74, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x03, + 0x52, 0x0a, 0x64, 0x73, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x73, 0x12, 0x2b, 0x0a, 0x11, + 0x73, 0x65, 0x61, 0x6c, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, + 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x03, 0x52, 0x10, 0x73, 0x65, 0x61, 0x6c, 0x65, 0x64, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x22, 0xb4, 0x01, + 0x0a, 0x12, 0x44, 0x6d, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x57, 0x61, 0x74, 0x63, 0x68, + 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x64, 0x6d, 0x43, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x64, 0x6d, 0x43, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x23, 0x0a, 0x0d, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, + 0x5f, 0x6c, 0x6f, 0x61, 0x64, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x6e, + 0x6f, 0x64, 0x65, 0x49, 0x44, 0x4c, 0x6f, 0x61, 0x64, 0x65, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x72, + 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, + 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x49, 0x44, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x6f, 0x64, + 0x65, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x03, 0x52, 0x07, 0x6e, 0x6f, 0x64, + 0x65, 0x49, 0x64, 0x73, 0x22, 0xa8, 0x02, 0x0a, 0x10, 0x51, 0x75, 0x65, 0x72, 0x79, 0x43, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x23, 0x0a, + 0x0d, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x71, 0x75, 0x65, 0x72, 0x79, 0x43, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x12, 0x30, 0x0a, 0x14, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x72, 0x65, 0x73, 0x75, + 0x6c, 0x74, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x12, 0x71, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x43, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x55, 0x0a, 0x16, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x5f, 0x73, + 0x65, 0x61, 0x6c, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x04, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x14, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x53, 0x65, 0x61, + 0x6c, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x42, 0x0a, 0x0d, 0x73, + 0x65, 0x65, 0x6b, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x4d, 0x73, 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x0c, 0x73, 0x65, 0x65, 0x6b, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, + 0x9e, 0x01, 0x0a, 0x0f, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x73, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x38, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0e, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, + 0x2f, 0x0a, 0x13, 0x69, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x5f, 0x70, 0x65, 0x72, 0x63, + 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x12, 0x69, 0x6e, + 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, + 0x22, 0xc9, 0x05, 0x0a, 0x0b, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, + 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 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, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x19, 0x0a, 0x08, + 0x6d, 0x65, 0x6d, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, + 0x6d, 0x65, 0x6d, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72, + 0x6f, 0x77, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, 0x6f, + 0x77, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, + 0x65, 0x12, 0x18, 0x0a, 0x07, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x18, 0x08, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x07, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x64, + 0x6d, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, + 0x64, 0x6d, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x26, 0x0a, 0x0e, 0x63, 0x6f, 0x6d, + 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x72, 0x6f, 0x6d, 0x18, 0x0a, 0x20, 0x03, 0x28, + 0x03, 0x52, 0x0e, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x72, 0x6f, + 0x6d, 0x12, 0x30, 0x0a, 0x13, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x42, 0x79, 0x43, 0x6f, + 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x08, 0x52, 0x13, + 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x42, 0x79, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x46, 0x0a, 0x0d, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x73, + 0x74, 0x61, 0x74, 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x0c, 0x73, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x43, 0x0a, 0x0b, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x0d, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, + 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x73, + 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x5f, 0x69, 0x64, 0x73, 0x18, + 0x0e, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x49, 0x64, + 0x73, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x0f, 0x20, + 0x03, 0x28, 0x03, 0x52, 0x07, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x64, 0x73, 0x12, 0x21, 0x0a, 0x0c, + 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x10, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0b, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, + 0x17, 0x0a, 0x07, 0x69, 0x73, 0x5f, 0x66, 0x61, 0x6b, 0x65, 0x18, 0x11, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x06, 0x69, 0x73, 0x46, 0x61, 0x6b, 0x65, 0x12, 0x35, 0x0a, 0x05, 0x6c, 0x65, 0x76, 0x65, + 0x6c, 0x18, 0x12, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x05, 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x12, + 0x1b, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x73, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x18, 0x13, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x53, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x22, 0xd0, 0x03, 0x0a, + 0x0e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, + 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x12, 0x4e, 0x0a, 0x10, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x0f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x39, 0x0a, 0x09, 0x6c, 0x6f, 0x61, 0x64, 0x5f, + 0x74, 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, + 0x4c, 0x6f, 0x61, 0x64, 0x54, 0x79, 0x70, 0x65, 0x52, 0x08, 0x6c, 0x6f, 0x61, 0x64, 0x54, 0x79, + 0x70, 0x65, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x05, 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, 0x33, 0x0a, 0x15, 0x72, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x64, 0x5f, 0x70, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x03, + 0x52, 0x14, 0x72, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x12, 0x2f, 0x0a, 0x13, 0x69, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, + 0x72, 0x79, 0x5f, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x18, 0x07, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x12, 0x69, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x50, 0x65, 0x72, + 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x72, 0x65, + 0x70, 0x6c, 0x69, 0x63, 0x61, 0x49, 0x64, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x72, 0x65, 0x70, 0x6c, + 0x69, 0x63, 0x61, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x09, 0x20, 0x01, 0x28, 0x05, + 0x52, 0x0d, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x22, + 0x55, 0x0a, 0x13, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x43, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1a, 0x0a, 0x08, 0x63, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, 0x63, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x22, 0x8a, 0x01, 0x0a, 0x16, 0x55, 0x6e, 0x73, 0x75, 0x62, + 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, + 0x6f, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x58, 0x0a, 0x13, 0x63, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 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, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x52, + 0x12, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x73, 0x22, 0xf5, 0x01, 0x0a, 0x11, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x43, + 0x68, 0x61, 0x6e, 0x67, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x23, 0x0a, 0x0d, 0x6f, 0x6e, 0x6c, + 0x69, 0x6e, 0x65, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x0c, 0x6f, 0x6e, 0x6c, 0x69, 0x6e, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x48, + 0x0a, 0x0f, 0x6f, 0x6e, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0e, 0x6f, 0x6e, 0x6c, 0x69, 0x6e, 0x65, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x6f, 0x66, 0x66, 0x6c, + 0x69, 0x6e, 0x65, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x0d, 0x6f, 0x66, 0x66, 0x6c, 0x69, 0x6e, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, + 0x4a, 0x0a, 0x10, 0x6f, 0x66, 0x66, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0f, 0x6f, 0x66, 0x66, 0x6c, + 0x69, 0x6e, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x89, 0x01, 0x0a, 0x18, + 0x53, 0x65, 0x61, 0x6c, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x43, 0x68, + 0x61, 0x6e, 0x67, 0x65, 0x49, 0x6e, 0x66, 0x6f, 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, 0x3b, 0x0a, 0x05, 0x69, 0x6e, + 0x66, 0x6f, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x49, 0x6e, 0x66, 0x6f, + 0x52, 0x05, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x22, 0xb4, 0x02, 0x0a, 0x1a, 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, 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, 0x61, 0x0a, 0x0b, 0x63, 0x68, 0x65, 0x63, + 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3f, 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, 0x2e, 0x43, 0x68, + 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0b, + 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x12, 0x22, 0x0a, 0x0c, 0x6c, + 0x61, 0x73, 0x74, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x54, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x0c, 0x6c, 0x61, 0x73, 0x74, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x54, 0x73, 0x1a, + 0x5d, 0x0a, 0x10, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 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, 0x33, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x4d, 0x73, 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x83, + 0x03, 0x0a, 0x1b, 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, 0x12, 0x33, + 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x42, 0x0a, 0x08, 0x73, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x26, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, + 0x72, 0x79, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, + 0x42, 0x0a, 0x08, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x56, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x63, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x73, 0x12, 0x41, 0x0a, 0x0c, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x5f, 0x76, 0x69, + 0x65, 0x77, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, + 0x65, 0x61, 0x64, 0x65, 0x72, 0x56, 0x69, 0x65, 0x77, 0x52, 0x0b, 0x6c, 0x65, 0x61, 0x64, 0x65, + 0x72, 0x56, 0x69, 0x65, 0x77, 0x73, 0x12, 0x22, 0x0a, 0x0c, 0x6c, 0x61, 0x73, 0x74, 0x4d, 0x6f, + 0x64, 0x69, 0x66, 0x79, 0x54, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x6c, 0x61, + 0x73, 0x74, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x54, 0x73, 0x12, 0x28, 0x0a, 0x0f, 0x6d, 0x65, + 0x6d, 0x43, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x49, 0x6e, 0x4d, 0x42, 0x18, 0x07, 0x20, + 0x01, 0x28, 0x01, 0x52, 0x0f, 0x6d, 0x65, 0x6d, 0x43, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, + 0x49, 0x6e, 0x4d, 0x42, 0x22, 0x83, 0x06, 0x0a, 0x0a, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x56, + 0x69, 0x65, 0x77, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x52, 0x0a, + 0x0c, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x64, 0x69, 0x73, 0x74, 0x18, 0x03, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x56, + 0x69, 0x65, 0x77, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x74, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0b, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, + 0x74, 0x12, 0x2d, 0x0a, 0x12, 0x67, 0x72, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x03, 0x52, 0x11, 0x67, + 0x72, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, + 0x12, 0x5e, 0x0a, 0x10, 0x67, 0x72, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, + 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x56, 0x69, 0x65, 0x77, 0x2e, 0x47, 0x72, 0x6f, 0x77, 0x69, + 0x6e, 0x67, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, + 0x0f, 0x67, 0x72, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, + 0x12, 0x24, 0x0a, 0x0d, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x56, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x2d, 0x0a, 0x13, 0x6e, 0x75, 0x6d, 0x5f, 0x6f, 0x66, + 0x5f, 0x67, 0x72, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x07, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x10, 0x6e, 0x75, 0x6d, 0x4f, 0x66, 0x47, 0x72, 0x6f, 0x77, 0x69, 0x6e, + 0x67, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x74, 0x0a, 0x18, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x65, 0x61, + 0x64, 0x65, 0x72, 0x56, 0x69, 0x65, 0x77, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x16, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, + 0x61, 0x74, 0x73, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x5f, 0x0a, 0x10, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x74, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, + 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, + 0x79, 0x12, 0x35, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, + 0x74, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x61, 0x0a, 0x14, + 0x47, 0x72, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x33, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x4d, 0x73, 0x67, 0x50, 0x6f, 0x73, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, + 0x49, 0x0a, 0x1b, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, + 0x73, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, + 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, + 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x3f, 0x0a, 0x0b, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, + 0x65, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, + 0x44, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0xd4, 0x03, 0x0a, 0x12, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, + 0x66, 0x6f, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, + 0x49, 0x44, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x1c, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x18, 0x0a, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x30, 0x0a, 0x14, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x64, 0x65, 0x6c, + 0x74, 0x61, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x04, 0x52, 0x12, 0x6c, 0x61, 0x73, 0x74, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x54, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x54, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, + 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, + 0x66, 0x6f, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x35, 0x0a, 0x05, + 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x05, 0x6c, 0x65, + 0x76, 0x65, 0x6c, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x73, 0x6f, 0x72, 0x74, 0x65, 0x64, + 0x18, 0x09, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x53, 0x6f, 0x72, 0x74, 0x65, 0x64, + 0x1a, 0x60, 0x0a, 0x0e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x03, 0x6b, 0x65, 0x79, 0x12, 0x38, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, + 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, + 0x38, 0x01, 0x22, 0x68, 0x0a, 0x12, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x56, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0xfd, 0x03, 0x0a, + 0x12, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x6f, 0x61, 0x64, 0x49, + 0x6e, 0x66, 0x6f, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x2f, 0x0a, 0x13, 0x72, 0x65, 0x6c, 0x65, 0x61, + 0x73, 0x65, 0x64, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, + 0x20, 0x03, 0x28, 0x03, 0x52, 0x12, 0x72, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x64, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x72, 0x65, 0x70, 0x6c, + 0x69, 0x63, 0x61, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, + 0x52, 0x0d, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, + 0x36, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x1e, 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, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, + 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x5d, 0x0a, 0x0d, 0x66, 0x69, 0x65, 0x6c, 0x64, + 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x38, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x6f, + 0x61, 0x64, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x64, 0x65, + 0x78, 0x49, 0x44, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, + 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x12, 0x39, 0x0a, 0x09, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x74, + 0x79, 0x70, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, + 0x6f, 0x61, 0x64, 0x54, 0x79, 0x70, 0x65, 0x52, 0x08, 0x6c, 0x6f, 0x61, 0x64, 0x54, 0x79, 0x70, + 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x5f, 0x74, 0x69, 0x6d, + 0x65, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0c, 0x72, 0x65, 0x63, 0x6f, 0x76, 0x65, + 0x72, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x66, + 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x6c, 0x6f, 0x61, + 0x64, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x62, 0x49, 0x44, 0x18, + 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x44, 0x1a, 0x3f, 0x0a, 0x11, 0x46, + 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, + 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xfc, 0x02, 0x0a, + 0x11, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x6f, 0x61, 0x64, 0x49, 0x6e, + 0x66, 0x6f, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x25, 0x0a, 0x0e, 0x72, 0x65, 0x70, 0x6c, + 0x69, 0x63, 0x61, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, + 0x52, 0x0d, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, + 0x36, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x1e, 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, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, + 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x5c, 0x0a, 0x0d, 0x66, 0x69, 0x65, 0x6c, 0x64, + 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x6f, 0x61, + 0x64, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, + 0x49, 0x44, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, + 0x64, 0x65, 0x78, 0x49, 0x44, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, + 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0c, 0x72, 0x65, + 0x63, 0x6f, 0x76, 0x65, 0x72, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x1a, 0x3f, 0x0a, 0x11, 0x46, 0x69, + 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, + 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, + 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x2c, 0x0a, 0x0f, 0x43, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x19, + 0x0a, 0x08, 0x72, 0x77, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x03, + 0x52, 0x07, 0x72, 0x77, 0x4e, 0x6f, 0x64, 0x65, 0x73, 0x22, 0xe0, 0x02, 0x0a, 0x07, 0x52, 0x65, + 0x70, 0x6c, 0x69, 0x63, 0x61, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x02, 0x49, 0x44, 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, 0x14, 0x0a, 0x05, 0x6e, 0x6f, 0x64, + 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x05, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x12, + 0x25, 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x75, + 0x70, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x19, 0x0a, 0x08, 0x72, 0x6f, 0x5f, 0x6e, 0x6f, 0x64, + 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x03, 0x52, 0x07, 0x72, 0x6f, 0x4e, 0x6f, 0x64, 0x65, + 0x73, 0x12, 0x5f, 0x0a, 0x12, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x6e, 0x6f, 0x64, + 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, + 0x72, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x52, 0x10, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, + 0x6f, 0x73, 0x1a, 0x68, 0x0a, 0x15, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x6f, 0x64, + 0x65, 0x49, 0x6e, 0x66, 0x6f, 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, 0x39, 0x0a, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, + 0x79, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, + 0x6f, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x8b, 0x05, 0x0a, + 0x0a, 0x53, 0x79, 0x6e, 0x63, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x30, 0x0a, 0x04, 0x74, + 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, + 0x79, 0x6e, 0x63, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x20, 0x0a, + 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, + 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x16, 0x0a, + 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, + 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, + 0x37, 0x0a, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, + 0x72, 0x79, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x49, 0x6e, + 0x66, 0x6f, 0x52, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x12, 0x28, 0x0a, 0x0f, 0x67, 0x72, 0x6f, 0x77, + 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x07, 0x20, 0x03, 0x28, + 0x03, 0x52, 0x0f, 0x67, 0x72, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x54, 0x61, 0x72, 0x67, + 0x65, 0x74, 0x12, 0x26, 0x0a, 0x0e, 0x73, 0x65, 0x61, 0x6c, 0x65, 0x64, 0x49, 0x6e, 0x54, 0x61, + 0x72, 0x67, 0x65, 0x74, 0x18, 0x08, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0e, 0x73, 0x65, 0x61, 0x6c, + 0x65, 0x64, 0x49, 0x6e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x24, 0x0a, 0x0d, 0x54, 0x61, + 0x72, 0x67, 0x65, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x09, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x0d, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x12, 0x28, 0x0a, 0x0f, 0x64, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x49, 0x6e, 0x54, 0x61, 0x72, + 0x67, 0x65, 0x74, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0f, 0x64, 0x72, 0x6f, 0x70, 0x70, + 0x65, 0x64, 0x49, 0x6e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x3d, 0x0a, 0x0a, 0x63, 0x68, + 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, + 0x67, 0x2e, 0x4d, 0x73, 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x63, + 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x74, 0x0a, 0x18, 0x70, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x76, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x0c, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x1a, + 0x49, 0x0a, 0x1b, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, + 0x73, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, + 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, + 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xc0, 0x03, 0x0a, 0x17, 0x53, + 0x79, 0x6e, 0x63, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, + 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, + 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, + 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x38, 0x0a, 0x07, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x79, 0x6e, + 0x63, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x05, 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, + 0x3d, 0x0a, 0x09, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x4d, 0x65, 0x74, 0x61, + 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x6c, 0x6f, 0x61, 0x64, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x1c, + 0x0a, 0x09, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x49, 0x44, 0x18, 0x07, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x09, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, + 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x45, 0x0a, 0x0f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, + 0x69, 0x6e, 0x66, 0x6f, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0d, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x4c, 0x69, 0x73, 0x74, 0x22, 0x97, 0x01, + 0x0a, 0x0d, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, + 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, + 0x61, 0x6d, 0x65, 0x12, 0x1e, 0x0a, 0x08, 0x63, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x05, 0x42, 0x02, 0x18, 0x01, 0x52, 0x08, 0x63, 0x61, 0x70, 0x61, 0x63, + 0x69, 0x74, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, + 0x28, 0x03, 0x52, 0x05, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x12, 0x3c, 0x0a, 0x06, 0x63, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x67, 0x2e, 0x52, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, + 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x22, 0xf7, 0x01, 0x0a, 0x16, 0x54, 0x72, 0x61, 0x6e, + 0x73, 0x66, 0x65, 0x72, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 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, 0x32, 0x0a, 0x15, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x72, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x13, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x32, 0x0a, 0x15, 0x74, 0x61, 0x72, 0x67, + 0x65, 0x74, 0x5f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x75, + 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x52, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x22, 0x0a, 0x0c, + 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, + 0x12, 0x1f, 0x0a, 0x0b, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x6e, 0x75, 0x6d, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, + 0x61, 0x22, 0x77, 0x0a, 0x1c, 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, 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, 0x25, 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, + 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x72, 0x65, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x22, 0xa2, 0x01, 0x0a, 0x1d, 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, 0x12, 0x33, 0x0a, 0x06, + 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x12, 0x4c, 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x67, 0x72, + 0x6f, 0x75, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x49, 0x6e, 0x66, 0x6f, + 0x52, 0x0d, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x22, + 0xf0, 0x05, 0x0a, 0x11, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, + 0x70, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1e, 0x0a, 0x08, 0x63, 0x61, 0x70, + 0x61, 0x63, 0x69, 0x74, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x42, 0x02, 0x18, 0x01, 0x52, + 0x08, 0x63, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x12, 0x2c, 0x0a, 0x12, 0x6e, 0x75, 0x6d, + 0x5f, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x10, 0x6e, 0x75, 0x6d, 0x41, 0x76, 0x61, 0x69, 0x6c, 0x61, + 0x62, 0x6c, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x69, 0x0a, 0x12, 0x6e, 0x75, 0x6d, 0x5f, 0x6c, + 0x6f, 0x61, 0x64, 0x65, 0x64, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x18, 0x04, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x4e, 0x75, 0x6d, 0x4c, 0x6f, + 0x61, 0x64, 0x65, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x52, 0x10, 0x6e, 0x75, 0x6d, 0x4c, 0x6f, 0x61, 0x64, 0x65, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x12, 0x66, 0x0a, 0x11, 0x6e, 0x75, 0x6d, 0x5f, 0x6f, 0x75, 0x74, 0x67, 0x6f, 0x69, + 0x6e, 0x67, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3a, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, + 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, + 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x4e, 0x75, 0x6d, 0x4f, 0x75, 0x74, 0x67, 0x6f, 0x69, 0x6e, 0x67, + 0x4e, 0x6f, 0x64, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0f, 0x6e, 0x75, 0x6d, 0x4f, 0x75, + 0x74, 0x67, 0x6f, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x66, 0x0a, 0x11, 0x6e, 0x75, + 0x6d, 0x5f, 0x69, 0x6e, 0x63, 0x6f, 0x6d, 0x69, 0x6e, 0x67, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x18, + 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x4e, 0x75, 0x6d, + 0x49, 0x6e, 0x63, 0x6f, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x0f, 0x6e, 0x75, 0x6d, 0x49, 0x6e, 0x63, 0x6f, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, + 0x64, 0x65, 0x12, 0x3c, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x07, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x72, 0x67, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, + 0x75, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x12, 0x33, 0x0a, 0x05, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x05, + 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x1a, 0x43, 0x0a, 0x15, 0x4e, 0x75, 0x6d, 0x4c, 0x6f, 0x61, 0x64, + 0x65, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, + 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, + 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x42, 0x0a, 0x14, 0x4e, 0x75, + 0x6d, 0x4f, 0x75, 0x74, 0x67, 0x6f, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x05, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x42, + 0x0a, 0x14, 0x4e, 0x75, 0x6d, 0x49, 0x6e, 0x63, 0x6f, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, + 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, + 0x38, 0x01, 0x22, 0xdf, 0x02, 0x0a, 0x0d, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 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, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 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, 0x03, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x23, + 0x0a, 0x0d, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, + 0x61, 0x6d, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, + 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x49, 0x64, 0x12, 0x3b, 0x0a, 0x0c, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x6b, 0x65, + 0x79, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x49, + 0x44, 0x73, 0x52, 0x0b, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x4b, 0x65, 0x79, 0x73, 0x12, + 0x1e, 0x0a, 0x0a, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x73, 0x18, 0x07, 0x20, + 0x03, 0x28, 0x04, 0x52, 0x0a, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x73, 0x12, + 0x33, 0x0a, 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1d, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x52, 0x05, 0x73, + 0x63, 0x6f, 0x70, 0x65, 0x22, 0xe6, 0x02, 0x0a, 0x12, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x42, + 0x61, 0x74, 0x63, 0x68, 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, 0x23, 0x0a, + 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, + 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, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x76, 0x63, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x03, 0x52, + 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x73, 0x12, 0x3b, 0x0a, 0x0c, 0x70, + 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x18, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x49, 0x44, 0x73, 0x52, 0x0b, 0x70, 0x72, 0x69, + 0x6d, 0x61, 0x72, 0x79, 0x4b, 0x65, 0x79, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x74, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x04, 0x52, 0x0a, 0x74, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x73, 0x12, 0x33, 0x0a, 0x05, 0x73, 0x63, 0x6f, 0x70, + 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x44, 0x61, 0x74, + 0x61, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x52, 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x22, 0x8a, 0x01, + 0x0a, 0x13, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x66, 0x61, + 0x69, 0x6c, 0x65, 0x64, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x09, + 0x66, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x49, 0x64, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x6d, 0x69, 0x73, + 0x73, 0x69, 0x6e, 0x67, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, + 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x49, 0x64, 0x73, 0x22, 0x68, 0x0a, 0x16, 0x41, 0x63, + 0x74, 0x69, 0x76, 0x61, 0x74, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, + 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x65, + 0x72, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x63, 0x68, 0x65, 0x63, 0x6b, + 0x65, 0x72, 0x49, 0x44, 0x22, 0x6a, 0x0a, 0x18, 0x44, 0x65, 0x61, 0x63, 0x74, 0x69, 0x76, 0x61, + 0x74, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, + 0x73, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x49, 0x44, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x49, 0x44, + 0x22, 0x67, 0x0a, 0x13, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, + 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x68, 0x65, + 0x63, 0x6b, 0x65, 0x72, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x05, 0x52, 0x0a, 0x63, + 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x49, 0x44, 0x73, 0x22, 0x90, 0x01, 0x0a, 0x14, 0x4c, 0x69, + 0x73, 0x74, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, + 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x43, 0x0a, 0x0c, 0x63, 0x68, 0x65, 0x63, 0x6b, + 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 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, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0c, + 0x63, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x22, 0x65, 0x0a, 0x0b, + 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x0e, 0x0a, 0x02, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x02, 0x69, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x64, + 0x65, 0x73, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x64, 0x65, 0x73, 0x63, 0x12, + 0x1c, 0x0a, 0x09, 0x61, 0x63, 0x74, 0x69, 0x76, 0x61, 0x74, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x09, 0x61, 0x63, 0x74, 0x69, 0x76, 0x61, 0x74, 0x65, 0x64, 0x12, 0x14, 0x0a, + 0x05, 0x66, 0x6f, 0x75, 0x6e, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, + 0x75, 0x6e, 0x64, 0x22, 0x76, 0x0a, 0x0d, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x54, 0x61, + 0x72, 0x67, 0x65, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x02, 0x49, 0x44, 0x12, 0x35, 0x0a, 0x05, 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4c, + 0x65, 0x76, 0x65, 0x6c, 0x52, 0x05, 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x12, 0x1e, 0x0a, 0x0b, 0x6e, + 0x75, 0x6d, 0x5f, 0x6f, 0x66, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x09, 0x6e, 0x75, 0x6d, 0x4f, 0x66, 0x52, 0x6f, 0x77, 0x73, 0x22, 0x72, 0x0a, 0x0f, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x20, + 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, + 0x12, 0x3d, 0x0a, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x54, + 0x61, 0x72, 0x67, 0x65, 0x74, 0x52, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, + 0xa5, 0x02, 0x0a, 0x0d, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x54, 0x61, 0x72, 0x67, 0x65, + 0x74, 0x12, 0x20, 0x0a, 0x0b, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, + 0x61, 0x6d, 0x65, 0x12, 0x2d, 0x0a, 0x12, 0x64, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x5f, 0x73, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, + 0x11, 0x64, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, + 0x44, 0x73, 0x12, 0x2d, 0x0a, 0x12, 0x67, 0x72, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x11, + 0x67, 0x72, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, + 0x73, 0x12, 0x50, 0x0a, 0x11, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, + 0x61, 0x72, 0x67, 0x65, 0x74, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, + 0x79, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x72, 0x67, 0x65, + 0x74, 0x52, 0x10, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x72, 0x67, + 0x65, 0x74, 0x73, 0x12, 0x42, 0x0a, 0x0d, 0x73, 0x65, 0x65, 0x6b, 0x5f, 0x70, 0x6f, 0x73, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x4d, 0x73, + 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0c, 0x73, 0x65, 0x65, 0x6b, 0x50, + 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x9c, 0x01, 0x0a, 0x10, 0x43, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x22, 0x0a, 0x0c, + 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, + 0x12, 0x4a, 0x0a, 0x0f, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x74, 0x61, 0x72, 0x67, + 0x65, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x43, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x52, 0x0e, 0x43, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x73, 0x12, 0x18, 0x0a, 0x07, + 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x4a, 0x0a, 0x08, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x6e, + 0x66, 0x6f, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, + 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x14, 0x0a, 0x05, + 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x74, 0x61, + 0x74, 0x65, 0x22, 0x48, 0x0a, 0x14, 0x4c, 0x69, 0x73, 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, + 0x6f, 0x64, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, + 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, + 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x22, 0x88, 0x01, 0x0a, + 0x15, 0x4c, 0x69, 0x73, 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x3a, 0x0a, 0x09, 0x6e, + 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x6e, 0x6f, + 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x22, 0x6b, 0x0a, 0x1f, 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, 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, 0x16, 0x0a, 0x06, + 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, + 0x64, 0x65, 0x49, 0x44, 0x22, 0xb9, 0x01, 0x0a, 0x20, 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, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x0e, + 0x0a, 0x02, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x49, 0x44, 0x12, 0x23, + 0x0a, 0x0d, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, + 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, + 0x6d, 0x65, 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x73, 0x65, 0x61, 0x6c, 0x65, 0x64, 0x5f, 0x73, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x03, 0x52, 0x10, + 0x73, 0x65, 0x61, 0x6c, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, + 0x22, 0x49, 0x0a, 0x15, 0x53, 0x75, 0x73, 0x70, 0x65, 0x6e, 0x64, 0x42, 0x61, 0x6c, 0x61, 0x6e, + 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, + 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x22, 0x48, 0x0a, 0x14, 0x52, + 0x65, 0x73, 0x75, 0x6d, 0x65, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, + 0x04, 0x62, 0x61, 0x73, 0x65, 0x22, 0x5e, 0x0a, 0x12, 0x53, 0x75, 0x73, 0x70, 0x65, 0x6e, 0x64, + 0x4e, 0x6f, 0x64, 0x65, 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, 0x16, 0x0a, + 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, + 0x6f, 0x64, 0x65, 0x49, 0x44, 0x22, 0x5d, 0x0a, 0x11, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x4e, + 0x6f, 0x64, 0x65, 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, 0x16, 0x0a, 0x06, + 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, + 0x64, 0x65, 0x49, 0x44, 0x22, 0x94, 0x02, 0x0a, 0x16, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, + 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, + 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, + 0x23, 0x0a, 0x0d, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4e, 0x6f, + 0x64, 0x65, 0x49, 0x44, 0x12, 0x23, 0x0a, 0x0d, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x6e, + 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x74, 0x61, 0x72, + 0x67, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, + 0x6e, 0x73, 0x66, 0x65, 0x72, 0x5f, 0x61, 0x6c, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x41, 0x6c, 0x6c, 0x12, 0x20, 0x0a, 0x0c, + 0x74, 0x6f, 0x5f, 0x61, 0x6c, 0x6c, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0a, 0x74, 0x6f, 0x41, 0x6c, 0x6c, 0x4e, 0x6f, 0x64, 0x65, 0x73, 0x12, 0x1b, + 0x0a, 0x09, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x08, 0x63, 0x6f, 0x70, 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x22, 0x99, 0x02, 0x0a, 0x16, + 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 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, 0x21, 0x0a, 0x0c, 0x63, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, + 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x0c, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x44, + 0x12, 0x23, 0x0a, 0x0d, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x49, + 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x4e, + 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, + 0x72, 0x5f, 0x61, 0x6c, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x74, 0x72, 0x61, + 0x6e, 0x73, 0x66, 0x65, 0x72, 0x41, 0x6c, 0x6c, 0x12, 0x20, 0x0a, 0x0c, 0x74, 0x6f, 0x5f, 0x61, + 0x6c, 0x6c, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, + 0x74, 0x6f, 0x41, 0x6c, 0x6c, 0x4e, 0x6f, 0x64, 0x65, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x63, 0x6f, + 0x70, 0x79, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x63, + 0x6f, 0x70, 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x22, 0x9f, 0x01, 0x0a, 0x21, 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, 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, + 0x23, 0x0a, 0x0d, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4e, 0x6f, + 0x64, 0x65, 0x49, 0x44, 0x12, 0x23, 0x0a, 0x0d, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x6e, + 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x74, 0x61, 0x72, + 0x67, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x22, 0xd5, 0x01, 0x0a, 0x17, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x4c, 0x6f, 0x61, 0x64, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 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, 0x12, 0x0a, 0x04, 0x64, 0x62, 0x49, 0x44, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x44, 0x12, 0x24, 0x0a, 0x0d, 0x63, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x03, 0x20, 0x03, + 0x28, 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, + 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x5f, 0x6e, 0x75, 0x6d, + 0x62, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x72, 0x65, 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, 0x2b, 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, 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, 0xd4, 0x1e, 0x0a, 0x0a, 0x51, 0x75, 0x65, 0x72, 0x79, 0x43, 0x6f, 0x6f, + 0x72, 0x64, 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, 0x6c, 0x0a, 0x0f, 0x53, 0x68, 0x6f, 0x77, 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, 0x69, 0x0a, 0x0e, 0x53, 0x68, 0x6f, 0x77, 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, 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, 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, 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, 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, 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, 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, + 0x42, 0x2f, 0x5a, 0x2d, 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, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x70, + 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_query_coord_proto_rawDescOnce sync.Once + file_query_coord_proto_rawDescData = file_query_coord_proto_rawDesc +) + +func file_query_coord_proto_rawDescGZIP() []byte { + file_query_coord_proto_rawDescOnce.Do(func() { + file_query_coord_proto_rawDescData = protoimpl.X.CompressGZIP(file_query_coord_proto_rawDescData) + }) + return file_query_coord_proto_rawDescData +} + +var file_query_coord_proto_enumTypes = make([]protoimpl.EnumInfo, 7) +var file_query_coord_proto_msgTypes = make([]protoimpl.MessageInfo, 101) +var file_query_coord_proto_goTypes = []interface{}{ + (LoadScope)(0), // 0: milvus.proto.query.LoadScope + (DataScope)(0), // 1: milvus.proto.query.DataScope + (PartitionState)(0), // 2: milvus.proto.query.PartitionState + (TriggerCondition)(0), // 3: milvus.proto.query.TriggerCondition + (LoadType)(0), // 4: milvus.proto.query.LoadType + (LoadStatus)(0), // 5: milvus.proto.query.LoadStatus + (SyncType)(0), // 6: milvus.proto.query.SyncType + (*ShowCollectionsRequest)(nil), // 7: milvus.proto.query.ShowCollectionsRequest + (*ShowCollectionsResponse)(nil), // 8: milvus.proto.query.ShowCollectionsResponse + (*ShowPartitionsRequest)(nil), // 9: milvus.proto.query.ShowPartitionsRequest + (*ShowPartitionsResponse)(nil), // 10: milvus.proto.query.ShowPartitionsResponse + (*LoadCollectionRequest)(nil), // 11: milvus.proto.query.LoadCollectionRequest + (*ReleaseCollectionRequest)(nil), // 12: milvus.proto.query.ReleaseCollectionRequest + (*GetStatisticsRequest)(nil), // 13: milvus.proto.query.GetStatisticsRequest + (*LoadPartitionsRequest)(nil), // 14: milvus.proto.query.LoadPartitionsRequest + (*ReleasePartitionsRequest)(nil), // 15: milvus.proto.query.ReleasePartitionsRequest + (*GetPartitionStatesRequest)(nil), // 16: milvus.proto.query.GetPartitionStatesRequest + (*GetPartitionStatesResponse)(nil), // 17: milvus.proto.query.GetPartitionStatesResponse + (*GetSegmentInfoRequest)(nil), // 18: milvus.proto.query.GetSegmentInfoRequest + (*GetSegmentInfoResponse)(nil), // 19: milvus.proto.query.GetSegmentInfoResponse + (*GetShardLeadersRequest)(nil), // 20: milvus.proto.query.GetShardLeadersRequest + (*GetShardLeadersResponse)(nil), // 21: milvus.proto.query.GetShardLeadersResponse + (*UpdateResourceGroupsRequest)(nil), // 22: milvus.proto.query.UpdateResourceGroupsRequest + (*ShardLeadersList)(nil), // 23: milvus.proto.query.ShardLeadersList + (*SyncNewCreatedPartitionRequest)(nil), // 24: milvus.proto.query.SyncNewCreatedPartitionRequest + (*LoadMetaInfo)(nil), // 25: milvus.proto.query.LoadMetaInfo + (*WatchDmChannelsRequest)(nil), // 26: milvus.proto.query.WatchDmChannelsRequest + (*UnsubDmChannelRequest)(nil), // 27: milvus.proto.query.UnsubDmChannelRequest + (*SegmentLoadInfo)(nil), // 28: milvus.proto.query.SegmentLoadInfo + (*FieldIndexInfo)(nil), // 29: milvus.proto.query.FieldIndexInfo + (*LoadSegmentsRequest)(nil), // 30: milvus.proto.query.LoadSegmentsRequest + (*ReleaseSegmentsRequest)(nil), // 31: milvus.proto.query.ReleaseSegmentsRequest + (*SearchRequest)(nil), // 32: milvus.proto.query.SearchRequest + (*QueryRequest)(nil), // 33: milvus.proto.query.QueryRequest + (*SyncReplicaSegmentsRequest)(nil), // 34: milvus.proto.query.SyncReplicaSegmentsRequest + (*ReplicaSegmentsInfo)(nil), // 35: milvus.proto.query.ReplicaSegmentsInfo + (*GetLoadInfoRequest)(nil), // 36: milvus.proto.query.GetLoadInfoRequest + (*GetLoadInfoResponse)(nil), // 37: milvus.proto.query.GetLoadInfoResponse + (*HandoffSegmentsRequest)(nil), // 38: milvus.proto.query.HandoffSegmentsRequest + (*LoadBalanceRequest)(nil), // 39: milvus.proto.query.LoadBalanceRequest + (*DmChannelWatchInfo)(nil), // 40: milvus.proto.query.DmChannelWatchInfo + (*QueryChannelInfo)(nil), // 41: milvus.proto.query.QueryChannelInfo + (*PartitionStates)(nil), // 42: milvus.proto.query.PartitionStates + (*SegmentInfo)(nil), // 43: milvus.proto.query.SegmentInfo + (*CollectionInfo)(nil), // 44: milvus.proto.query.CollectionInfo + (*UnsubscribeChannels)(nil), // 45: milvus.proto.query.UnsubscribeChannels + (*UnsubscribeChannelInfo)(nil), // 46: milvus.proto.query.UnsubscribeChannelInfo + (*SegmentChangeInfo)(nil), // 47: milvus.proto.query.SegmentChangeInfo + (*SealedSegmentsChangeInfo)(nil), // 48: milvus.proto.query.SealedSegmentsChangeInfo + (*GetDataDistributionRequest)(nil), // 49: milvus.proto.query.GetDataDistributionRequest + (*GetDataDistributionResponse)(nil), // 50: milvus.proto.query.GetDataDistributionResponse + (*LeaderView)(nil), // 51: milvus.proto.query.LeaderView + (*SegmentDist)(nil), // 52: milvus.proto.query.SegmentDist + (*SegmentVersionInfo)(nil), // 53: milvus.proto.query.SegmentVersionInfo + (*ChannelVersionInfo)(nil), // 54: milvus.proto.query.ChannelVersionInfo + (*CollectionLoadInfo)(nil), // 55: milvus.proto.query.CollectionLoadInfo + (*PartitionLoadInfo)(nil), // 56: milvus.proto.query.PartitionLoadInfo + (*ChannelNodeInfo)(nil), // 57: milvus.proto.query.ChannelNodeInfo + (*Replica)(nil), // 58: milvus.proto.query.Replica + (*SyncAction)(nil), // 59: milvus.proto.query.SyncAction + (*SyncDistributionRequest)(nil), // 60: milvus.proto.query.SyncDistributionRequest + (*ResourceGroup)(nil), // 61: milvus.proto.query.ResourceGroup + (*TransferReplicaRequest)(nil), // 62: milvus.proto.query.TransferReplicaRequest + (*DescribeResourceGroupRequest)(nil), // 63: milvus.proto.query.DescribeResourceGroupRequest + (*DescribeResourceGroupResponse)(nil), // 64: milvus.proto.query.DescribeResourceGroupResponse + (*ResourceGroupInfo)(nil), // 65: milvus.proto.query.ResourceGroupInfo + (*DeleteRequest)(nil), // 66: milvus.proto.query.DeleteRequest + (*DeleteBatchRequest)(nil), // 67: milvus.proto.query.DeleteBatchRequest + (*DeleteBatchResponse)(nil), // 68: milvus.proto.query.DeleteBatchResponse + (*ActivateCheckerRequest)(nil), // 69: milvus.proto.query.ActivateCheckerRequest + (*DeactivateCheckerRequest)(nil), // 70: milvus.proto.query.DeactivateCheckerRequest + (*ListCheckersRequest)(nil), // 71: milvus.proto.query.ListCheckersRequest + (*ListCheckersResponse)(nil), // 72: milvus.proto.query.ListCheckersResponse + (*CheckerInfo)(nil), // 73: milvus.proto.query.CheckerInfo + (*SegmentTarget)(nil), // 74: milvus.proto.query.SegmentTarget + (*PartitionTarget)(nil), // 75: milvus.proto.query.PartitionTarget + (*ChannelTarget)(nil), // 76: milvus.proto.query.ChannelTarget + (*CollectionTarget)(nil), // 77: milvus.proto.query.CollectionTarget + (*NodeInfo)(nil), // 78: milvus.proto.query.NodeInfo + (*ListQueryNodeRequest)(nil), // 79: milvus.proto.query.ListQueryNodeRequest + (*ListQueryNodeResponse)(nil), // 80: milvus.proto.query.ListQueryNodeResponse + (*GetQueryNodeDistributionRequest)(nil), // 81: milvus.proto.query.GetQueryNodeDistributionRequest + (*GetQueryNodeDistributionResponse)(nil), // 82: milvus.proto.query.GetQueryNodeDistributionResponse + (*SuspendBalanceRequest)(nil), // 83: milvus.proto.query.SuspendBalanceRequest + (*ResumeBalanceRequest)(nil), // 84: milvus.proto.query.ResumeBalanceRequest + (*SuspendNodeRequest)(nil), // 85: milvus.proto.query.SuspendNodeRequest + (*ResumeNodeRequest)(nil), // 86: milvus.proto.query.ResumeNodeRequest + (*TransferSegmentRequest)(nil), // 87: milvus.proto.query.TransferSegmentRequest + (*TransferChannelRequest)(nil), // 88: milvus.proto.query.TransferChannelRequest + (*CheckQueryNodeDistributionRequest)(nil), // 89: milvus.proto.query.CheckQueryNodeDistributionRequest + (*UpdateLoadConfigRequest)(nil), // 90: milvus.proto.query.UpdateLoadConfigRequest + nil, // 91: milvus.proto.query.LoadCollectionRequest.FieldIndexIDEntry + nil, // 92: milvus.proto.query.LoadPartitionsRequest.FieldIndexIDEntry + nil, // 93: milvus.proto.query.UpdateResourceGroupsRequest.ResourceGroupsEntry + nil, // 94: milvus.proto.query.WatchDmChannelsRequest.SegmentInfosEntry + nil, // 95: milvus.proto.query.SegmentLoadInfo.TextStatsLogsEntry + nil, // 96: milvus.proto.query.GetDataDistributionRequest.CheckpointsEntry + nil, // 97: milvus.proto.query.LeaderView.SegmentDistEntry + nil, // 98: milvus.proto.query.LeaderView.GrowingSegmentsEntry + nil, // 99: milvus.proto.query.LeaderView.PartitionStatsVersionsEntry + nil, // 100: milvus.proto.query.SegmentVersionInfo.IndexInfoEntry + nil, // 101: milvus.proto.query.CollectionLoadInfo.FieldIndexIDEntry + nil, // 102: milvus.proto.query.PartitionLoadInfo.FieldIndexIDEntry + nil, // 103: milvus.proto.query.Replica.ChannelNodeInfosEntry + nil, // 104: milvus.proto.query.SyncAction.PartitionStatsVersionsEntry + nil, // 105: milvus.proto.query.ResourceGroupInfo.NumLoadedReplicaEntry + nil, // 106: milvus.proto.query.ResourceGroupInfo.NumOutgoingNodeEntry + nil, // 107: milvus.proto.query.ResourceGroupInfo.NumIncomingNodeEntry + (*commonpb.MsgBase)(nil), // 108: milvus.proto.common.MsgBase + (*commonpb.Status)(nil), // 109: milvus.proto.common.Status + (*schemapb.LongArray)(nil), // 110: milvus.proto.schema.LongArray + (*schemapb.CollectionSchema)(nil), // 111: milvus.proto.schema.CollectionSchema + (*internalpb.GetStatisticsRequest)(nil), // 112: milvus.proto.internal.GetStatisticsRequest + (*indexpb.IndexInfo)(nil), // 113: milvus.proto.index.IndexInfo + (*commonpb.KeyValuePair)(nil), // 114: milvus.proto.common.KeyValuePair + (*datapb.VchannelInfo)(nil), // 115: milvus.proto.data.VchannelInfo + (*datapb.SegmentInfo)(nil), // 116: milvus.proto.data.SegmentInfo + (*datapb.FieldBinlog)(nil), // 117: milvus.proto.data.FieldBinlog + (*msgpb.MsgPosition)(nil), // 118: milvus.proto.msg.MsgPosition + (datapb.SegmentLevel)(0), // 119: milvus.proto.data.SegmentLevel + (*internalpb.SearchRequest)(nil), // 120: milvus.proto.internal.SearchRequest + (*internalpb.RetrieveRequest)(nil), // 121: milvus.proto.internal.RetrieveRequest + (commonpb.SegmentState)(0), // 122: milvus.proto.common.SegmentState + (*rgpb.ResourceGroupConfig)(nil), // 123: milvus.proto.rg.ResourceGroupConfig + (*commonpb.NodeInfo)(nil), // 124: milvus.proto.common.NodeInfo + (*schemapb.IDs)(nil), // 125: milvus.proto.schema.IDs + (*datapb.TextIndexStats)(nil), // 126: milvus.proto.data.TextIndexStats + (*milvuspb.GetComponentStatesRequest)(nil), // 127: milvus.proto.milvus.GetComponentStatesRequest + (*internalpb.GetTimeTickChannelRequest)(nil), // 128: milvus.proto.internal.GetTimeTickChannelRequest + (*internalpb.GetStatisticsChannelRequest)(nil), // 129: milvus.proto.internal.GetStatisticsChannelRequest + (*internalpb.ShowConfigurationsRequest)(nil), // 130: milvus.proto.internal.ShowConfigurationsRequest + (*milvuspb.GetMetricsRequest)(nil), // 131: milvus.proto.milvus.GetMetricsRequest + (*milvuspb.GetReplicasRequest)(nil), // 132: milvus.proto.milvus.GetReplicasRequest + (*milvuspb.CheckHealthRequest)(nil), // 133: milvus.proto.milvus.CheckHealthRequest + (*milvuspb.CreateResourceGroupRequest)(nil), // 134: milvus.proto.milvus.CreateResourceGroupRequest + (*milvuspb.DropResourceGroupRequest)(nil), // 135: milvus.proto.milvus.DropResourceGroupRequest + (*milvuspb.TransferNodeRequest)(nil), // 136: milvus.proto.milvus.TransferNodeRequest + (*milvuspb.ListResourceGroupsRequest)(nil), // 137: milvus.proto.milvus.ListResourceGroupsRequest + (*milvuspb.ComponentStates)(nil), // 138: milvus.proto.milvus.ComponentStates + (*milvuspb.StringResponse)(nil), // 139: milvus.proto.milvus.StringResponse + (*internalpb.ShowConfigurationsResponse)(nil), // 140: milvus.proto.internal.ShowConfigurationsResponse + (*milvuspb.GetMetricsResponse)(nil), // 141: milvus.proto.milvus.GetMetricsResponse + (*milvuspb.GetReplicasResponse)(nil), // 142: milvus.proto.milvus.GetReplicasResponse + (*milvuspb.CheckHealthResponse)(nil), // 143: milvus.proto.milvus.CheckHealthResponse + (*milvuspb.ListResourceGroupsResponse)(nil), // 144: milvus.proto.milvus.ListResourceGroupsResponse + (*internalpb.GetStatisticsResponse)(nil), // 145: milvus.proto.internal.GetStatisticsResponse + (*internalpb.SearchResults)(nil), // 146: milvus.proto.internal.SearchResults + (*internalpb.RetrieveResults)(nil), // 147: milvus.proto.internal.RetrieveResults +} +var file_query_coord_proto_depIdxs = []int32{ + 108, // 0: milvus.proto.query.ShowCollectionsRequest.base:type_name -> milvus.proto.common.MsgBase + 109, // 1: milvus.proto.query.ShowCollectionsResponse.status:type_name -> milvus.proto.common.Status + 110, // 2: milvus.proto.query.ShowCollectionsResponse.load_fields:type_name -> milvus.proto.schema.LongArray + 108, // 3: milvus.proto.query.ShowPartitionsRequest.base:type_name -> milvus.proto.common.MsgBase + 109, // 4: milvus.proto.query.ShowPartitionsResponse.status:type_name -> milvus.proto.common.Status + 108, // 5: milvus.proto.query.LoadCollectionRequest.base:type_name -> milvus.proto.common.MsgBase + 111, // 6: milvus.proto.query.LoadCollectionRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 91, // 7: milvus.proto.query.LoadCollectionRequest.field_indexID:type_name -> milvus.proto.query.LoadCollectionRequest.FieldIndexIDEntry + 108, // 8: milvus.proto.query.ReleaseCollectionRequest.base:type_name -> milvus.proto.common.MsgBase + 112, // 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 + 108, // 11: milvus.proto.query.LoadPartitionsRequest.base:type_name -> milvus.proto.common.MsgBase + 111, // 12: milvus.proto.query.LoadPartitionsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 92, // 13: milvus.proto.query.LoadPartitionsRequest.field_indexID:type_name -> milvus.proto.query.LoadPartitionsRequest.FieldIndexIDEntry + 113, // 14: milvus.proto.query.LoadPartitionsRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo + 108, // 15: milvus.proto.query.ReleasePartitionsRequest.base:type_name -> milvus.proto.common.MsgBase + 108, // 16: milvus.proto.query.GetPartitionStatesRequest.base:type_name -> milvus.proto.common.MsgBase + 109, // 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 + 108, // 19: milvus.proto.query.GetSegmentInfoRequest.base:type_name -> milvus.proto.common.MsgBase + 109, // 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 + 108, // 22: milvus.proto.query.GetShardLeadersRequest.base:type_name -> milvus.proto.common.MsgBase + 109, // 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 + 108, // 25: milvus.proto.query.UpdateResourceGroupsRequest.base:type_name -> milvus.proto.common.MsgBase + 93, // 26: milvus.proto.query.UpdateResourceGroupsRequest.resource_groups:type_name -> milvus.proto.query.UpdateResourceGroupsRequest.ResourceGroupsEntry + 108, // 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 + 114, // 29: milvus.proto.query.LoadMetaInfo.db_properties:type_name -> milvus.proto.common.KeyValuePair + 108, // 30: milvus.proto.query.WatchDmChannelsRequest.base:type_name -> milvus.proto.common.MsgBase + 115, // 31: milvus.proto.query.WatchDmChannelsRequest.infos:type_name -> milvus.proto.data.VchannelInfo + 111, // 32: milvus.proto.query.WatchDmChannelsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 116, // 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 + 94, // 35: milvus.proto.query.WatchDmChannelsRequest.segment_infos:type_name -> milvus.proto.query.WatchDmChannelsRequest.SegmentInfosEntry + 113, // 36: milvus.proto.query.WatchDmChannelsRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo + 108, // 37: milvus.proto.query.UnsubDmChannelRequest.base:type_name -> milvus.proto.common.MsgBase + 117, // 38: milvus.proto.query.SegmentLoadInfo.binlog_paths:type_name -> milvus.proto.data.FieldBinlog + 117, // 39: milvus.proto.query.SegmentLoadInfo.statslogs:type_name -> milvus.proto.data.FieldBinlog + 117, // 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 + 118, // 42: milvus.proto.query.SegmentLoadInfo.start_position:type_name -> milvus.proto.msg.MsgPosition + 118, // 43: milvus.proto.query.SegmentLoadInfo.delta_position:type_name -> milvus.proto.msg.MsgPosition + 119, // 44: milvus.proto.query.SegmentLoadInfo.level:type_name -> milvus.proto.data.SegmentLevel + 95, // 45: milvus.proto.query.SegmentLoadInfo.textStatsLogs:type_name -> milvus.proto.query.SegmentLoadInfo.TextStatsLogsEntry + 117, // 46: milvus.proto.query.SegmentLoadInfo.bm25logs:type_name -> milvus.proto.data.FieldBinlog + 114, // 47: milvus.proto.query.FieldIndexInfo.index_params:type_name -> milvus.proto.common.KeyValuePair + 108, // 48: milvus.proto.query.LoadSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase + 28, // 49: milvus.proto.query.LoadSegmentsRequest.infos:type_name -> milvus.proto.query.SegmentLoadInfo + 111, // 50: milvus.proto.query.LoadSegmentsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 25, // 51: milvus.proto.query.LoadSegmentsRequest.load_meta:type_name -> milvus.proto.query.LoadMetaInfo + 118, // 52: milvus.proto.query.LoadSegmentsRequest.delta_positions:type_name -> milvus.proto.msg.MsgPosition + 0, // 53: milvus.proto.query.LoadSegmentsRequest.load_scope:type_name -> milvus.proto.query.LoadScope + 113, // 54: milvus.proto.query.LoadSegmentsRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo + 108, // 55: milvus.proto.query.ReleaseSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase + 1, // 56: milvus.proto.query.ReleaseSegmentsRequest.scope:type_name -> milvus.proto.query.DataScope + 118, // 57: milvus.proto.query.ReleaseSegmentsRequest.checkpoint:type_name -> milvus.proto.msg.MsgPosition + 120, // 58: milvus.proto.query.SearchRequest.req:type_name -> milvus.proto.internal.SearchRequest + 1, // 59: milvus.proto.query.SearchRequest.scope:type_name -> milvus.proto.query.DataScope + 121, // 60: milvus.proto.query.QueryRequest.req:type_name -> milvus.proto.internal.RetrieveRequest + 1, // 61: milvus.proto.query.QueryRequest.scope:type_name -> milvus.proto.query.DataScope + 108, // 62: milvus.proto.query.SyncReplicaSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase + 35, // 63: milvus.proto.query.SyncReplicaSegmentsRequest.replica_segments:type_name -> milvus.proto.query.ReplicaSegmentsInfo + 108, // 64: milvus.proto.query.GetLoadInfoRequest.base:type_name -> milvus.proto.common.MsgBase + 109, // 65: milvus.proto.query.GetLoadInfoResponse.status:type_name -> milvus.proto.common.Status + 111, // 66: milvus.proto.query.GetLoadInfoResponse.schema:type_name -> milvus.proto.schema.CollectionSchema + 4, // 67: milvus.proto.query.GetLoadInfoResponse.load_type:type_name -> milvus.proto.query.LoadType + 108, // 68: milvus.proto.query.HandoffSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase + 43, // 69: milvus.proto.query.HandoffSegmentsRequest.segmentInfos:type_name -> milvus.proto.query.SegmentInfo + 108, // 70: milvus.proto.query.LoadBalanceRequest.base:type_name -> milvus.proto.common.MsgBase + 3, // 71: milvus.proto.query.LoadBalanceRequest.balance_reason:type_name -> milvus.proto.query.TriggerCondition + 43, // 72: milvus.proto.query.QueryChannelInfo.global_sealed_segments:type_name -> milvus.proto.query.SegmentInfo + 118, // 73: milvus.proto.query.QueryChannelInfo.seek_position:type_name -> milvus.proto.msg.MsgPosition + 2, // 74: milvus.proto.query.PartitionStates.state:type_name -> milvus.proto.query.PartitionState + 122, // 75: milvus.proto.query.SegmentInfo.segment_state:type_name -> milvus.proto.common.SegmentState + 29, // 76: milvus.proto.query.SegmentInfo.index_infos:type_name -> milvus.proto.query.FieldIndexInfo + 119, // 77: milvus.proto.query.SegmentInfo.level:type_name -> milvus.proto.data.SegmentLevel + 42, // 78: milvus.proto.query.CollectionInfo.partition_states:type_name -> milvus.proto.query.PartitionStates + 4, // 79: milvus.proto.query.CollectionInfo.load_type:type_name -> milvus.proto.query.LoadType + 111, // 80: milvus.proto.query.CollectionInfo.schema:type_name -> milvus.proto.schema.CollectionSchema + 45, // 81: milvus.proto.query.UnsubscribeChannelInfo.collection_channels:type_name -> milvus.proto.query.UnsubscribeChannels + 43, // 82: milvus.proto.query.SegmentChangeInfo.online_segments:type_name -> milvus.proto.query.SegmentInfo + 43, // 83: milvus.proto.query.SegmentChangeInfo.offline_segments:type_name -> milvus.proto.query.SegmentInfo + 108, // 84: milvus.proto.query.SealedSegmentsChangeInfo.base:type_name -> milvus.proto.common.MsgBase + 47, // 85: milvus.proto.query.SealedSegmentsChangeInfo.infos:type_name -> milvus.proto.query.SegmentChangeInfo + 108, // 86: milvus.proto.query.GetDataDistributionRequest.base:type_name -> milvus.proto.common.MsgBase + 96, // 87: milvus.proto.query.GetDataDistributionRequest.checkpoints:type_name -> milvus.proto.query.GetDataDistributionRequest.CheckpointsEntry + 109, // 88: milvus.proto.query.GetDataDistributionResponse.status:type_name -> milvus.proto.common.Status + 53, // 89: milvus.proto.query.GetDataDistributionResponse.segments:type_name -> milvus.proto.query.SegmentVersionInfo + 54, // 90: milvus.proto.query.GetDataDistributionResponse.channels:type_name -> milvus.proto.query.ChannelVersionInfo + 51, // 91: milvus.proto.query.GetDataDistributionResponse.leader_views:type_name -> milvus.proto.query.LeaderView + 97, // 92: milvus.proto.query.LeaderView.segment_dist:type_name -> milvus.proto.query.LeaderView.SegmentDistEntry + 98, // 93: milvus.proto.query.LeaderView.growing_segments:type_name -> milvus.proto.query.LeaderView.GrowingSegmentsEntry + 99, // 94: milvus.proto.query.LeaderView.partition_stats_versions:type_name -> milvus.proto.query.LeaderView.PartitionStatsVersionsEntry + 100, // 95: milvus.proto.query.SegmentVersionInfo.index_info:type_name -> milvus.proto.query.SegmentVersionInfo.IndexInfoEntry + 119, // 96: milvus.proto.query.SegmentVersionInfo.level:type_name -> milvus.proto.data.SegmentLevel + 5, // 97: milvus.proto.query.CollectionLoadInfo.status:type_name -> milvus.proto.query.LoadStatus + 101, // 98: milvus.proto.query.CollectionLoadInfo.field_indexID:type_name -> milvus.proto.query.CollectionLoadInfo.FieldIndexIDEntry + 4, // 99: milvus.proto.query.CollectionLoadInfo.load_type:type_name -> milvus.proto.query.LoadType + 5, // 100: milvus.proto.query.PartitionLoadInfo.status:type_name -> milvus.proto.query.LoadStatus + 102, // 101: milvus.proto.query.PartitionLoadInfo.field_indexID:type_name -> milvus.proto.query.PartitionLoadInfo.FieldIndexIDEntry + 103, // 102: milvus.proto.query.Replica.channel_node_infos:type_name -> milvus.proto.query.Replica.ChannelNodeInfosEntry + 6, // 103: milvus.proto.query.SyncAction.type:type_name -> milvus.proto.query.SyncType + 28, // 104: milvus.proto.query.SyncAction.info:type_name -> milvus.proto.query.SegmentLoadInfo + 118, // 105: milvus.proto.query.SyncAction.checkpoint:type_name -> milvus.proto.msg.MsgPosition + 104, // 106: milvus.proto.query.SyncAction.partition_stats_versions:type_name -> milvus.proto.query.SyncAction.PartitionStatsVersionsEntry + 108, // 107: milvus.proto.query.SyncDistributionRequest.base:type_name -> milvus.proto.common.MsgBase + 59, // 108: milvus.proto.query.SyncDistributionRequest.actions:type_name -> milvus.proto.query.SyncAction + 111, // 109: milvus.proto.query.SyncDistributionRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 25, // 110: milvus.proto.query.SyncDistributionRequest.load_meta:type_name -> milvus.proto.query.LoadMetaInfo + 113, // 111: milvus.proto.query.SyncDistributionRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo + 123, // 112: milvus.proto.query.ResourceGroup.config:type_name -> milvus.proto.rg.ResourceGroupConfig + 108, // 113: milvus.proto.query.TransferReplicaRequest.base:type_name -> milvus.proto.common.MsgBase + 108, // 114: milvus.proto.query.DescribeResourceGroupRequest.base:type_name -> milvus.proto.common.MsgBase + 109, // 115: milvus.proto.query.DescribeResourceGroupResponse.status:type_name -> milvus.proto.common.Status + 65, // 116: milvus.proto.query.DescribeResourceGroupResponse.resource_group:type_name -> milvus.proto.query.ResourceGroupInfo + 105, // 117: milvus.proto.query.ResourceGroupInfo.num_loaded_replica:type_name -> milvus.proto.query.ResourceGroupInfo.NumLoadedReplicaEntry + 106, // 118: milvus.proto.query.ResourceGroupInfo.num_outgoing_node:type_name -> milvus.proto.query.ResourceGroupInfo.NumOutgoingNodeEntry + 107, // 119: milvus.proto.query.ResourceGroupInfo.num_incoming_node:type_name -> milvus.proto.query.ResourceGroupInfo.NumIncomingNodeEntry + 123, // 120: milvus.proto.query.ResourceGroupInfo.config:type_name -> milvus.proto.rg.ResourceGroupConfig + 124, // 121: milvus.proto.query.ResourceGroupInfo.nodes:type_name -> milvus.proto.common.NodeInfo + 108, // 122: milvus.proto.query.DeleteRequest.base:type_name -> milvus.proto.common.MsgBase + 125, // 123: milvus.proto.query.DeleteRequest.primary_keys:type_name -> milvus.proto.schema.IDs + 1, // 124: milvus.proto.query.DeleteRequest.scope:type_name -> milvus.proto.query.DataScope + 108, // 125: milvus.proto.query.DeleteBatchRequest.base:type_name -> milvus.proto.common.MsgBase + 125, // 126: milvus.proto.query.DeleteBatchRequest.primary_keys:type_name -> milvus.proto.schema.IDs + 1, // 127: milvus.proto.query.DeleteBatchRequest.scope:type_name -> milvus.proto.query.DataScope + 109, // 128: milvus.proto.query.DeleteBatchResponse.status:type_name -> milvus.proto.common.Status + 108, // 129: milvus.proto.query.ActivateCheckerRequest.base:type_name -> milvus.proto.common.MsgBase + 108, // 130: milvus.proto.query.DeactivateCheckerRequest.base:type_name -> milvus.proto.common.MsgBase + 108, // 131: milvus.proto.query.ListCheckersRequest.base:type_name -> milvus.proto.common.MsgBase + 109, // 132: milvus.proto.query.ListCheckersResponse.status:type_name -> milvus.proto.common.Status + 73, // 133: milvus.proto.query.ListCheckersResponse.checkerInfos:type_name -> milvus.proto.query.CheckerInfo + 119, // 134: milvus.proto.query.SegmentTarget.level:type_name -> milvus.proto.data.SegmentLevel + 74, // 135: milvus.proto.query.PartitionTarget.segments:type_name -> milvus.proto.query.SegmentTarget + 75, // 136: milvus.proto.query.ChannelTarget.partition_targets:type_name -> milvus.proto.query.PartitionTarget + 118, // 137: milvus.proto.query.ChannelTarget.seek_position:type_name -> milvus.proto.msg.MsgPosition + 76, // 138: milvus.proto.query.CollectionTarget.Channel_targets:type_name -> milvus.proto.query.ChannelTarget + 108, // 139: milvus.proto.query.ListQueryNodeRequest.base:type_name -> milvus.proto.common.MsgBase + 109, // 140: milvus.proto.query.ListQueryNodeResponse.status:type_name -> milvus.proto.common.Status + 78, // 141: milvus.proto.query.ListQueryNodeResponse.nodeInfos:type_name -> milvus.proto.query.NodeInfo + 108, // 142: milvus.proto.query.GetQueryNodeDistributionRequest.base:type_name -> milvus.proto.common.MsgBase + 109, // 143: milvus.proto.query.GetQueryNodeDistributionResponse.status:type_name -> milvus.proto.common.Status + 108, // 144: milvus.proto.query.SuspendBalanceRequest.base:type_name -> milvus.proto.common.MsgBase + 108, // 145: milvus.proto.query.ResumeBalanceRequest.base:type_name -> milvus.proto.common.MsgBase + 108, // 146: milvus.proto.query.SuspendNodeRequest.base:type_name -> milvus.proto.common.MsgBase + 108, // 147: milvus.proto.query.ResumeNodeRequest.base:type_name -> milvus.proto.common.MsgBase + 108, // 148: milvus.proto.query.TransferSegmentRequest.base:type_name -> milvus.proto.common.MsgBase + 108, // 149: milvus.proto.query.TransferChannelRequest.base:type_name -> milvus.proto.common.MsgBase + 108, // 150: milvus.proto.query.CheckQueryNodeDistributionRequest.base:type_name -> milvus.proto.common.MsgBase + 108, // 151: milvus.proto.query.UpdateLoadConfigRequest.base:type_name -> milvus.proto.common.MsgBase + 123, // 152: milvus.proto.query.UpdateResourceGroupsRequest.ResourceGroupsEntry.value:type_name -> milvus.proto.rg.ResourceGroupConfig + 116, // 153: milvus.proto.query.WatchDmChannelsRequest.SegmentInfosEntry.value:type_name -> milvus.proto.data.SegmentInfo + 126, // 154: milvus.proto.query.SegmentLoadInfo.TextStatsLogsEntry.value:type_name -> milvus.proto.data.TextIndexStats + 118, // 155: milvus.proto.query.GetDataDistributionRequest.CheckpointsEntry.value:type_name -> milvus.proto.msg.MsgPosition + 52, // 156: milvus.proto.query.LeaderView.SegmentDistEntry.value:type_name -> milvus.proto.query.SegmentDist + 118, // 157: milvus.proto.query.LeaderView.GrowingSegmentsEntry.value:type_name -> milvus.proto.msg.MsgPosition + 29, // 158: milvus.proto.query.SegmentVersionInfo.IndexInfoEntry.value:type_name -> milvus.proto.query.FieldIndexInfo + 57, // 159: milvus.proto.query.Replica.ChannelNodeInfosEntry.value:type_name -> milvus.proto.query.ChannelNodeInfo + 127, // 160: milvus.proto.query.QueryCoord.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest + 128, // 161: milvus.proto.query.QueryCoord.GetTimeTickChannel:input_type -> milvus.proto.internal.GetTimeTickChannelRequest + 129, // 162: milvus.proto.query.QueryCoord.GetStatisticsChannel:input_type -> milvus.proto.internal.GetStatisticsChannelRequest + 7, // 163: milvus.proto.query.QueryCoord.ShowCollections:input_type -> milvus.proto.query.ShowCollectionsRequest + 9, // 164: milvus.proto.query.QueryCoord.ShowPartitions:input_type -> milvus.proto.query.ShowPartitionsRequest + 14, // 165: milvus.proto.query.QueryCoord.LoadPartitions:input_type -> milvus.proto.query.LoadPartitionsRequest + 15, // 166: milvus.proto.query.QueryCoord.ReleasePartitions:input_type -> milvus.proto.query.ReleasePartitionsRequest + 11, // 167: milvus.proto.query.QueryCoord.LoadCollection:input_type -> milvus.proto.query.LoadCollectionRequest + 12, // 168: milvus.proto.query.QueryCoord.ReleaseCollection:input_type -> milvus.proto.query.ReleaseCollectionRequest + 24, // 169: milvus.proto.query.QueryCoord.SyncNewCreatedPartition:input_type -> milvus.proto.query.SyncNewCreatedPartitionRequest + 16, // 170: milvus.proto.query.QueryCoord.GetPartitionStates:input_type -> milvus.proto.query.GetPartitionStatesRequest + 18, // 171: milvus.proto.query.QueryCoord.GetSegmentInfo:input_type -> milvus.proto.query.GetSegmentInfoRequest + 39, // 172: milvus.proto.query.QueryCoord.LoadBalance:input_type -> milvus.proto.query.LoadBalanceRequest + 130, // 173: milvus.proto.query.QueryCoord.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest + 131, // 174: milvus.proto.query.QueryCoord.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest + 132, // 175: milvus.proto.query.QueryCoord.GetReplicas:input_type -> milvus.proto.milvus.GetReplicasRequest + 20, // 176: milvus.proto.query.QueryCoord.GetShardLeaders:input_type -> milvus.proto.query.GetShardLeadersRequest + 133, // 177: milvus.proto.query.QueryCoord.CheckHealth:input_type -> milvus.proto.milvus.CheckHealthRequest + 134, // 178: milvus.proto.query.QueryCoord.CreateResourceGroup:input_type -> milvus.proto.milvus.CreateResourceGroupRequest + 22, // 179: milvus.proto.query.QueryCoord.UpdateResourceGroups:input_type -> milvus.proto.query.UpdateResourceGroupsRequest + 135, // 180: milvus.proto.query.QueryCoord.DropResourceGroup:input_type -> milvus.proto.milvus.DropResourceGroupRequest + 136, // 181: milvus.proto.query.QueryCoord.TransferNode:input_type -> milvus.proto.milvus.TransferNodeRequest + 62, // 182: milvus.proto.query.QueryCoord.TransferReplica:input_type -> milvus.proto.query.TransferReplicaRequest + 137, // 183: milvus.proto.query.QueryCoord.ListResourceGroups:input_type -> milvus.proto.milvus.ListResourceGroupsRequest + 63, // 184: milvus.proto.query.QueryCoord.DescribeResourceGroup:input_type -> milvus.proto.query.DescribeResourceGroupRequest + 71, // 185: milvus.proto.query.QueryCoord.ListCheckers:input_type -> milvus.proto.query.ListCheckersRequest + 69, // 186: milvus.proto.query.QueryCoord.ActivateChecker:input_type -> milvus.proto.query.ActivateCheckerRequest + 70, // 187: milvus.proto.query.QueryCoord.DeactivateChecker:input_type -> milvus.proto.query.DeactivateCheckerRequest + 79, // 188: milvus.proto.query.QueryCoord.ListQueryNode:input_type -> milvus.proto.query.ListQueryNodeRequest + 81, // 189: milvus.proto.query.QueryCoord.GetQueryNodeDistribution:input_type -> milvus.proto.query.GetQueryNodeDistributionRequest + 83, // 190: milvus.proto.query.QueryCoord.SuspendBalance:input_type -> milvus.proto.query.SuspendBalanceRequest + 84, // 191: milvus.proto.query.QueryCoord.ResumeBalance:input_type -> milvus.proto.query.ResumeBalanceRequest + 85, // 192: milvus.proto.query.QueryCoord.SuspendNode:input_type -> milvus.proto.query.SuspendNodeRequest + 86, // 193: milvus.proto.query.QueryCoord.ResumeNode:input_type -> milvus.proto.query.ResumeNodeRequest + 87, // 194: milvus.proto.query.QueryCoord.TransferSegment:input_type -> milvus.proto.query.TransferSegmentRequest + 88, // 195: milvus.proto.query.QueryCoord.TransferChannel:input_type -> milvus.proto.query.TransferChannelRequest + 89, // 196: milvus.proto.query.QueryCoord.CheckQueryNodeDistribution:input_type -> milvus.proto.query.CheckQueryNodeDistributionRequest + 90, // 197: milvus.proto.query.QueryCoord.UpdateLoadConfig:input_type -> milvus.proto.query.UpdateLoadConfigRequest + 127, // 198: milvus.proto.query.QueryNode.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest + 128, // 199: milvus.proto.query.QueryNode.GetTimeTickChannel:input_type -> milvus.proto.internal.GetTimeTickChannelRequest + 129, // 200: milvus.proto.query.QueryNode.GetStatisticsChannel:input_type -> milvus.proto.internal.GetStatisticsChannelRequest + 26, // 201: milvus.proto.query.QueryNode.WatchDmChannels:input_type -> milvus.proto.query.WatchDmChannelsRequest + 27, // 202: milvus.proto.query.QueryNode.UnsubDmChannel:input_type -> milvus.proto.query.UnsubDmChannelRequest + 30, // 203: milvus.proto.query.QueryNode.LoadSegments:input_type -> milvus.proto.query.LoadSegmentsRequest + 12, // 204: milvus.proto.query.QueryNode.ReleaseCollection:input_type -> milvus.proto.query.ReleaseCollectionRequest + 14, // 205: milvus.proto.query.QueryNode.LoadPartitions:input_type -> milvus.proto.query.LoadPartitionsRequest + 15, // 206: milvus.proto.query.QueryNode.ReleasePartitions:input_type -> milvus.proto.query.ReleasePartitionsRequest + 31, // 207: milvus.proto.query.QueryNode.ReleaseSegments:input_type -> milvus.proto.query.ReleaseSegmentsRequest + 18, // 208: milvus.proto.query.QueryNode.GetSegmentInfo:input_type -> milvus.proto.query.GetSegmentInfoRequest + 34, // 209: milvus.proto.query.QueryNode.SyncReplicaSegments:input_type -> milvus.proto.query.SyncReplicaSegmentsRequest + 13, // 210: milvus.proto.query.QueryNode.GetStatistics:input_type -> milvus.proto.query.GetStatisticsRequest + 32, // 211: milvus.proto.query.QueryNode.Search:input_type -> milvus.proto.query.SearchRequest + 32, // 212: milvus.proto.query.QueryNode.SearchSegments:input_type -> milvus.proto.query.SearchRequest + 33, // 213: milvus.proto.query.QueryNode.Query:input_type -> milvus.proto.query.QueryRequest + 33, // 214: milvus.proto.query.QueryNode.QueryStream:input_type -> milvus.proto.query.QueryRequest + 33, // 215: milvus.proto.query.QueryNode.QuerySegments:input_type -> milvus.proto.query.QueryRequest + 33, // 216: milvus.proto.query.QueryNode.QueryStreamSegments:input_type -> milvus.proto.query.QueryRequest + 130, // 217: milvus.proto.query.QueryNode.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest + 131, // 218: milvus.proto.query.QueryNode.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest + 49, // 219: milvus.proto.query.QueryNode.GetDataDistribution:input_type -> milvus.proto.query.GetDataDistributionRequest + 60, // 220: milvus.proto.query.QueryNode.SyncDistribution:input_type -> milvus.proto.query.SyncDistributionRequest + 66, // 221: milvus.proto.query.QueryNode.Delete:input_type -> milvus.proto.query.DeleteRequest + 67, // 222: milvus.proto.query.QueryNode.DeleteBatch:input_type -> milvus.proto.query.DeleteBatchRequest + 138, // 223: milvus.proto.query.QueryCoord.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates + 139, // 224: milvus.proto.query.QueryCoord.GetTimeTickChannel:output_type -> milvus.proto.milvus.StringResponse + 139, // 225: milvus.proto.query.QueryCoord.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse + 8, // 226: milvus.proto.query.QueryCoord.ShowCollections:output_type -> milvus.proto.query.ShowCollectionsResponse + 10, // 227: milvus.proto.query.QueryCoord.ShowPartitions:output_type -> milvus.proto.query.ShowPartitionsResponse + 109, // 228: milvus.proto.query.QueryCoord.LoadPartitions:output_type -> milvus.proto.common.Status + 109, // 229: milvus.proto.query.QueryCoord.ReleasePartitions:output_type -> milvus.proto.common.Status + 109, // 230: milvus.proto.query.QueryCoord.LoadCollection:output_type -> milvus.proto.common.Status + 109, // 231: milvus.proto.query.QueryCoord.ReleaseCollection:output_type -> milvus.proto.common.Status + 109, // 232: milvus.proto.query.QueryCoord.SyncNewCreatedPartition:output_type -> milvus.proto.common.Status + 17, // 233: milvus.proto.query.QueryCoord.GetPartitionStates:output_type -> milvus.proto.query.GetPartitionStatesResponse + 19, // 234: milvus.proto.query.QueryCoord.GetSegmentInfo:output_type -> milvus.proto.query.GetSegmentInfoResponse + 109, // 235: milvus.proto.query.QueryCoord.LoadBalance:output_type -> milvus.proto.common.Status + 140, // 236: milvus.proto.query.QueryCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse + 141, // 237: milvus.proto.query.QueryCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse + 142, // 238: milvus.proto.query.QueryCoord.GetReplicas:output_type -> milvus.proto.milvus.GetReplicasResponse + 21, // 239: milvus.proto.query.QueryCoord.GetShardLeaders:output_type -> milvus.proto.query.GetShardLeadersResponse + 143, // 240: milvus.proto.query.QueryCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse + 109, // 241: milvus.proto.query.QueryCoord.CreateResourceGroup:output_type -> milvus.proto.common.Status + 109, // 242: milvus.proto.query.QueryCoord.UpdateResourceGroups:output_type -> milvus.proto.common.Status + 109, // 243: milvus.proto.query.QueryCoord.DropResourceGroup:output_type -> milvus.proto.common.Status + 109, // 244: milvus.proto.query.QueryCoord.TransferNode:output_type -> milvus.proto.common.Status + 109, // 245: milvus.proto.query.QueryCoord.TransferReplica:output_type -> milvus.proto.common.Status + 144, // 246: milvus.proto.query.QueryCoord.ListResourceGroups:output_type -> milvus.proto.milvus.ListResourceGroupsResponse + 64, // 247: milvus.proto.query.QueryCoord.DescribeResourceGroup:output_type -> milvus.proto.query.DescribeResourceGroupResponse + 72, // 248: milvus.proto.query.QueryCoord.ListCheckers:output_type -> milvus.proto.query.ListCheckersResponse + 109, // 249: milvus.proto.query.QueryCoord.ActivateChecker:output_type -> milvus.proto.common.Status + 109, // 250: milvus.proto.query.QueryCoord.DeactivateChecker:output_type -> milvus.proto.common.Status + 80, // 251: milvus.proto.query.QueryCoord.ListQueryNode:output_type -> milvus.proto.query.ListQueryNodeResponse + 82, // 252: milvus.proto.query.QueryCoord.GetQueryNodeDistribution:output_type -> milvus.proto.query.GetQueryNodeDistributionResponse + 109, // 253: milvus.proto.query.QueryCoord.SuspendBalance:output_type -> milvus.proto.common.Status + 109, // 254: milvus.proto.query.QueryCoord.ResumeBalance:output_type -> milvus.proto.common.Status + 109, // 255: milvus.proto.query.QueryCoord.SuspendNode:output_type -> milvus.proto.common.Status + 109, // 256: milvus.proto.query.QueryCoord.ResumeNode:output_type -> milvus.proto.common.Status + 109, // 257: milvus.proto.query.QueryCoord.TransferSegment:output_type -> milvus.proto.common.Status + 109, // 258: milvus.proto.query.QueryCoord.TransferChannel:output_type -> milvus.proto.common.Status + 109, // 259: milvus.proto.query.QueryCoord.CheckQueryNodeDistribution:output_type -> milvus.proto.common.Status + 109, // 260: milvus.proto.query.QueryCoord.UpdateLoadConfig:output_type -> milvus.proto.common.Status + 138, // 261: milvus.proto.query.QueryNode.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates + 139, // 262: milvus.proto.query.QueryNode.GetTimeTickChannel:output_type -> milvus.proto.milvus.StringResponse + 139, // 263: milvus.proto.query.QueryNode.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse + 109, // 264: milvus.proto.query.QueryNode.WatchDmChannels:output_type -> milvus.proto.common.Status + 109, // 265: milvus.proto.query.QueryNode.UnsubDmChannel:output_type -> milvus.proto.common.Status + 109, // 266: milvus.proto.query.QueryNode.LoadSegments:output_type -> milvus.proto.common.Status + 109, // 267: milvus.proto.query.QueryNode.ReleaseCollection:output_type -> milvus.proto.common.Status + 109, // 268: milvus.proto.query.QueryNode.LoadPartitions:output_type -> milvus.proto.common.Status + 109, // 269: milvus.proto.query.QueryNode.ReleasePartitions:output_type -> milvus.proto.common.Status + 109, // 270: milvus.proto.query.QueryNode.ReleaseSegments:output_type -> milvus.proto.common.Status + 19, // 271: milvus.proto.query.QueryNode.GetSegmentInfo:output_type -> milvus.proto.query.GetSegmentInfoResponse + 109, // 272: milvus.proto.query.QueryNode.SyncReplicaSegments:output_type -> milvus.proto.common.Status + 145, // 273: milvus.proto.query.QueryNode.GetStatistics:output_type -> milvus.proto.internal.GetStatisticsResponse + 146, // 274: milvus.proto.query.QueryNode.Search:output_type -> milvus.proto.internal.SearchResults + 146, // 275: milvus.proto.query.QueryNode.SearchSegments:output_type -> milvus.proto.internal.SearchResults + 147, // 276: milvus.proto.query.QueryNode.Query:output_type -> milvus.proto.internal.RetrieveResults + 147, // 277: milvus.proto.query.QueryNode.QueryStream:output_type -> milvus.proto.internal.RetrieveResults + 147, // 278: milvus.proto.query.QueryNode.QuerySegments:output_type -> milvus.proto.internal.RetrieveResults + 147, // 279: milvus.proto.query.QueryNode.QueryStreamSegments:output_type -> milvus.proto.internal.RetrieveResults + 140, // 280: milvus.proto.query.QueryNode.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse + 141, // 281: milvus.proto.query.QueryNode.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse + 50, // 282: milvus.proto.query.QueryNode.GetDataDistribution:output_type -> milvus.proto.query.GetDataDistributionResponse + 109, // 283: milvus.proto.query.QueryNode.SyncDistribution:output_type -> milvus.proto.common.Status + 109, // 284: milvus.proto.query.QueryNode.Delete:output_type -> milvus.proto.common.Status + 68, // 285: milvus.proto.query.QueryNode.DeleteBatch:output_type -> milvus.proto.query.DeleteBatchResponse + 223, // [223:286] is the sub-list for method output_type + 160, // [160:223] is the sub-list for method input_type + 160, // [160:160] is the sub-list for extension type_name + 160, // [160:160] is the sub-list for extension extendee + 0, // [0:160] is the sub-list for field type_name +} + +func init() { file_query_coord_proto_init() } +func file_query_coord_proto_init() { + if File_query_coord_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_query_coord_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ShowCollectionsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ShowCollectionsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ShowPartitionsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ShowPartitionsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*LoadCollectionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ReleaseCollectionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetStatisticsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*LoadPartitionsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ReleasePartitionsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetPartitionStatesRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetPartitionStatesResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetSegmentInfoRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetSegmentInfoResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetShardLeadersRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetShardLeadersResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*UpdateResourceGroupsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ShardLeadersList); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SyncNewCreatedPartitionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*LoadMetaInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*WatchDmChannelsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*UnsubDmChannelRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentLoadInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*FieldIndexInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*LoadSegmentsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ReleaseSegmentsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SearchRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QueryRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SyncReplicaSegmentsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ReplicaSegmentsInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetLoadInfoRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetLoadInfoResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*HandoffSegmentsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*LoadBalanceRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DmChannelWatchInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QueryChannelInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PartitionStates); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CollectionInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*UnsubscribeChannels); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*UnsubscribeChannelInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentChangeInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SealedSegmentsChangeInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetDataDistributionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetDataDistributionResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*LeaderView); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentDist); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentVersionInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ChannelVersionInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CollectionLoadInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PartitionLoadInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[50].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ChannelNodeInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Replica); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SyncAction); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SyncDistributionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ResourceGroup); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[55].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*TransferReplicaRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[56].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DescribeResourceGroupRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[57].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DescribeResourceGroupResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[58].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ResourceGroupInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[59].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DeleteRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[60].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DeleteBatchRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[61].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DeleteBatchResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[62].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ActivateCheckerRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[63].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DeactivateCheckerRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[64].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListCheckersRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[65].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListCheckersResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[66].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CheckerInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[67].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentTarget); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[68].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PartitionTarget); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[69].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ChannelTarget); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[70].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CollectionTarget); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[71].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*NodeInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[72].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListQueryNodeRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[73].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListQueryNodeResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[74].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetQueryNodeDistributionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[75].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetQueryNodeDistributionResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[76].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SuspendBalanceRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[77].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ResumeBalanceRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[78].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SuspendNodeRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[79].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ResumeNodeRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[80].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*TransferSegmentRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[81].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*TransferChannelRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[82].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CheckQueryNodeDistributionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[83].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*UpdateLoadConfigRequest); 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{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_query_coord_proto_rawDesc, + NumEnums: 7, + NumMessages: 101, + NumExtensions: 0, + NumServices: 2, + }, + GoTypes: file_query_coord_proto_goTypes, + DependencyIndexes: file_query_coord_proto_depIdxs, + EnumInfos: file_query_coord_proto_enumTypes, + MessageInfos: file_query_coord_proto_msgTypes, + }.Build() + File_query_coord_proto = out.File + file_query_coord_proto_rawDesc = nil + file_query_coord_proto_goTypes = nil + file_query_coord_proto_depIdxs = nil +} diff --git a/pkg/proto/querypb/query_coord_grpc.pb.go b/pkg/proto/querypb/query_coord_grpc.pb.go new file mode 100644 index 0000000000000..6e3d9f8234ee9 --- /dev/null +++ b/pkg/proto/querypb/query_coord_grpc.pb.go @@ -0,0 +1,2522 @@ +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. +// versions: +// - protoc-gen-go-grpc v1.3.0 +// - protoc v3.21.4 +// source: query_coord.proto + +package querypb + +import ( + context "context" + commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" +) + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +// Requires gRPC-Go v1.32.0 or later. +const _ = grpc.SupportPackageIsVersion7 + +const ( + QueryCoord_GetComponentStates_FullMethodName = "/milvus.proto.query.QueryCoord/GetComponentStates" + QueryCoord_GetTimeTickChannel_FullMethodName = "/milvus.proto.query.QueryCoord/GetTimeTickChannel" + QueryCoord_GetStatisticsChannel_FullMethodName = "/milvus.proto.query.QueryCoord/GetStatisticsChannel" + QueryCoord_ShowCollections_FullMethodName = "/milvus.proto.query.QueryCoord/ShowCollections" + QueryCoord_ShowPartitions_FullMethodName = "/milvus.proto.query.QueryCoord/ShowPartitions" + QueryCoord_LoadPartitions_FullMethodName = "/milvus.proto.query.QueryCoord/LoadPartitions" + QueryCoord_ReleasePartitions_FullMethodName = "/milvus.proto.query.QueryCoord/ReleasePartitions" + QueryCoord_LoadCollection_FullMethodName = "/milvus.proto.query.QueryCoord/LoadCollection" + QueryCoord_ReleaseCollection_FullMethodName = "/milvus.proto.query.QueryCoord/ReleaseCollection" + QueryCoord_SyncNewCreatedPartition_FullMethodName = "/milvus.proto.query.QueryCoord/SyncNewCreatedPartition" + QueryCoord_GetPartitionStates_FullMethodName = "/milvus.proto.query.QueryCoord/GetPartitionStates" + QueryCoord_GetSegmentInfo_FullMethodName = "/milvus.proto.query.QueryCoord/GetSegmentInfo" + QueryCoord_LoadBalance_FullMethodName = "/milvus.proto.query.QueryCoord/LoadBalance" + QueryCoord_ShowConfigurations_FullMethodName = "/milvus.proto.query.QueryCoord/ShowConfigurations" + QueryCoord_GetMetrics_FullMethodName = "/milvus.proto.query.QueryCoord/GetMetrics" + QueryCoord_GetReplicas_FullMethodName = "/milvus.proto.query.QueryCoord/GetReplicas" + QueryCoord_GetShardLeaders_FullMethodName = "/milvus.proto.query.QueryCoord/GetShardLeaders" + QueryCoord_CheckHealth_FullMethodName = "/milvus.proto.query.QueryCoord/CheckHealth" + QueryCoord_CreateResourceGroup_FullMethodName = "/milvus.proto.query.QueryCoord/CreateResourceGroup" + QueryCoord_UpdateResourceGroups_FullMethodName = "/milvus.proto.query.QueryCoord/UpdateResourceGroups" + QueryCoord_DropResourceGroup_FullMethodName = "/milvus.proto.query.QueryCoord/DropResourceGroup" + QueryCoord_TransferNode_FullMethodName = "/milvus.proto.query.QueryCoord/TransferNode" + QueryCoord_TransferReplica_FullMethodName = "/milvus.proto.query.QueryCoord/TransferReplica" + QueryCoord_ListResourceGroups_FullMethodName = "/milvus.proto.query.QueryCoord/ListResourceGroups" + QueryCoord_DescribeResourceGroup_FullMethodName = "/milvus.proto.query.QueryCoord/DescribeResourceGroup" + QueryCoord_ListCheckers_FullMethodName = "/milvus.proto.query.QueryCoord/ListCheckers" + QueryCoord_ActivateChecker_FullMethodName = "/milvus.proto.query.QueryCoord/ActivateChecker" + QueryCoord_DeactivateChecker_FullMethodName = "/milvus.proto.query.QueryCoord/DeactivateChecker" + QueryCoord_ListQueryNode_FullMethodName = "/milvus.proto.query.QueryCoord/ListQueryNode" + QueryCoord_GetQueryNodeDistribution_FullMethodName = "/milvus.proto.query.QueryCoord/GetQueryNodeDistribution" + QueryCoord_SuspendBalance_FullMethodName = "/milvus.proto.query.QueryCoord/SuspendBalance" + QueryCoord_ResumeBalance_FullMethodName = "/milvus.proto.query.QueryCoord/ResumeBalance" + QueryCoord_SuspendNode_FullMethodName = "/milvus.proto.query.QueryCoord/SuspendNode" + QueryCoord_ResumeNode_FullMethodName = "/milvus.proto.query.QueryCoord/ResumeNode" + QueryCoord_TransferSegment_FullMethodName = "/milvus.proto.query.QueryCoord/TransferSegment" + QueryCoord_TransferChannel_FullMethodName = "/milvus.proto.query.QueryCoord/TransferChannel" + QueryCoord_CheckQueryNodeDistribution_FullMethodName = "/milvus.proto.query.QueryCoord/CheckQueryNodeDistribution" + QueryCoord_UpdateLoadConfig_FullMethodName = "/milvus.proto.query.QueryCoord/UpdateLoadConfig" +) + +// QueryCoordClient is the client API for QueryCoord service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +type QueryCoordClient interface { + GetComponentStates(ctx context.Context, in *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) + GetTimeTickChannel(ctx context.Context, in *internalpb.GetTimeTickChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) + GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) + ShowCollections(ctx context.Context, in *ShowCollectionsRequest, opts ...grpc.CallOption) (*ShowCollectionsResponse, error) + ShowPartitions(ctx context.Context, in *ShowPartitionsRequest, opts ...grpc.CallOption) (*ShowPartitionsResponse, error) + LoadPartitions(ctx context.Context, in *LoadPartitionsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + ReleasePartitions(ctx context.Context, in *ReleasePartitionsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + LoadCollection(ctx context.Context, in *LoadCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + ReleaseCollection(ctx context.Context, in *ReleaseCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + SyncNewCreatedPartition(ctx context.Context, in *SyncNewCreatedPartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + GetPartitionStates(ctx context.Context, in *GetPartitionStatesRequest, opts ...grpc.CallOption) (*GetPartitionStatesResponse, error) + GetSegmentInfo(ctx context.Context, in *GetSegmentInfoRequest, opts ...grpc.CallOption) (*GetSegmentInfoResponse, error) + LoadBalance(ctx context.Context, in *LoadBalanceRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + ShowConfigurations(ctx context.Context, in *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error) + // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy + GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) + // https://wiki.lfaidata.foundation/display/MIL/MEP+23+--+Multiple+memory+replication+design + GetReplicas(ctx context.Context, in *milvuspb.GetReplicasRequest, opts ...grpc.CallOption) (*milvuspb.GetReplicasResponse, error) + GetShardLeaders(ctx context.Context, in *GetShardLeadersRequest, opts ...grpc.CallOption) (*GetShardLeadersResponse, error) + CheckHealth(ctx context.Context, in *milvuspb.CheckHealthRequest, opts ...grpc.CallOption) (*milvuspb.CheckHealthResponse, error) + CreateResourceGroup(ctx context.Context, in *milvuspb.CreateResourceGroupRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + UpdateResourceGroups(ctx context.Context, in *UpdateResourceGroupsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + DropResourceGroup(ctx context.Context, in *milvuspb.DropResourceGroupRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + TransferNode(ctx context.Context, in *milvuspb.TransferNodeRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + TransferReplica(ctx context.Context, in *TransferReplicaRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + ListResourceGroups(ctx context.Context, in *milvuspb.ListResourceGroupsRequest, opts ...grpc.CallOption) (*milvuspb.ListResourceGroupsResponse, error) + DescribeResourceGroup(ctx context.Context, in *DescribeResourceGroupRequest, opts ...grpc.CallOption) (*DescribeResourceGroupResponse, error) + // ops interfaces + ListCheckers(ctx context.Context, in *ListCheckersRequest, opts ...grpc.CallOption) (*ListCheckersResponse, error) + ActivateChecker(ctx context.Context, in *ActivateCheckerRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + DeactivateChecker(ctx context.Context, in *DeactivateCheckerRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + ListQueryNode(ctx context.Context, in *ListQueryNodeRequest, opts ...grpc.CallOption) (*ListQueryNodeResponse, error) + GetQueryNodeDistribution(ctx context.Context, in *GetQueryNodeDistributionRequest, opts ...grpc.CallOption) (*GetQueryNodeDistributionResponse, error) + SuspendBalance(ctx context.Context, in *SuspendBalanceRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + ResumeBalance(ctx context.Context, in *ResumeBalanceRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + SuspendNode(ctx context.Context, in *SuspendNodeRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + ResumeNode(ctx context.Context, in *ResumeNodeRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + TransferSegment(ctx context.Context, in *TransferSegmentRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + TransferChannel(ctx context.Context, in *TransferChannelRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + CheckQueryNodeDistribution(ctx context.Context, in *CheckQueryNodeDistributionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + UpdateLoadConfig(ctx context.Context, in *UpdateLoadConfigRequest, opts ...grpc.CallOption) (*commonpb.Status, error) +} + +type queryCoordClient struct { + cc grpc.ClientConnInterface +} + +func NewQueryCoordClient(cc grpc.ClientConnInterface) QueryCoordClient { + return &queryCoordClient{cc} +} + +func (c *queryCoordClient) GetComponentStates(ctx context.Context, in *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) { + out := new(milvuspb.ComponentStates) + err := c.cc.Invoke(ctx, QueryCoord_GetComponentStates_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) GetTimeTickChannel(ctx context.Context, in *internalpb.GetTimeTickChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) { + out := new(milvuspb.StringResponse) + err := c.cc.Invoke(ctx, QueryCoord_GetTimeTickChannel_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) { + out := new(milvuspb.StringResponse) + err := c.cc.Invoke(ctx, QueryCoord_GetStatisticsChannel_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) ShowCollections(ctx context.Context, in *ShowCollectionsRequest, opts ...grpc.CallOption) (*ShowCollectionsResponse, error) { + out := new(ShowCollectionsResponse) + err := c.cc.Invoke(ctx, QueryCoord_ShowCollections_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) ShowPartitions(ctx context.Context, in *ShowPartitionsRequest, opts ...grpc.CallOption) (*ShowPartitionsResponse, error) { + out := new(ShowPartitionsResponse) + err := c.cc.Invoke(ctx, QueryCoord_ShowPartitions_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) LoadPartitions(ctx context.Context, in *LoadPartitionsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryCoord_LoadPartitions_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) ReleasePartitions(ctx context.Context, in *ReleasePartitionsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryCoord_ReleasePartitions_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) LoadCollection(ctx context.Context, in *LoadCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryCoord_LoadCollection_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) ReleaseCollection(ctx context.Context, in *ReleaseCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryCoord_ReleaseCollection_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) SyncNewCreatedPartition(ctx context.Context, in *SyncNewCreatedPartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryCoord_SyncNewCreatedPartition_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) GetPartitionStates(ctx context.Context, in *GetPartitionStatesRequest, opts ...grpc.CallOption) (*GetPartitionStatesResponse, error) { + out := new(GetPartitionStatesResponse) + err := c.cc.Invoke(ctx, QueryCoord_GetPartitionStates_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) GetSegmentInfo(ctx context.Context, in *GetSegmentInfoRequest, opts ...grpc.CallOption) (*GetSegmentInfoResponse, error) { + out := new(GetSegmentInfoResponse) + err := c.cc.Invoke(ctx, QueryCoord_GetSegmentInfo_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) LoadBalance(ctx context.Context, in *LoadBalanceRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryCoord_LoadBalance_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) ShowConfigurations(ctx context.Context, in *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error) { + out := new(internalpb.ShowConfigurationsResponse) + err := c.cc.Invoke(ctx, QueryCoord_ShowConfigurations_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) { + out := new(milvuspb.GetMetricsResponse) + err := c.cc.Invoke(ctx, QueryCoord_GetMetrics_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) GetReplicas(ctx context.Context, in *milvuspb.GetReplicasRequest, opts ...grpc.CallOption) (*milvuspb.GetReplicasResponse, error) { + out := new(milvuspb.GetReplicasResponse) + err := c.cc.Invoke(ctx, QueryCoord_GetReplicas_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) GetShardLeaders(ctx context.Context, in *GetShardLeadersRequest, opts ...grpc.CallOption) (*GetShardLeadersResponse, error) { + out := new(GetShardLeadersResponse) + err := c.cc.Invoke(ctx, QueryCoord_GetShardLeaders_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) CheckHealth(ctx context.Context, in *milvuspb.CheckHealthRequest, opts ...grpc.CallOption) (*milvuspb.CheckHealthResponse, error) { + out := new(milvuspb.CheckHealthResponse) + err := c.cc.Invoke(ctx, QueryCoord_CheckHealth_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) CreateResourceGroup(ctx context.Context, in *milvuspb.CreateResourceGroupRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryCoord_CreateResourceGroup_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) UpdateResourceGroups(ctx context.Context, in *UpdateResourceGroupsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryCoord_UpdateResourceGroups_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) DropResourceGroup(ctx context.Context, in *milvuspb.DropResourceGroupRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryCoord_DropResourceGroup_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) TransferNode(ctx context.Context, in *milvuspb.TransferNodeRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryCoord_TransferNode_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) TransferReplica(ctx context.Context, in *TransferReplicaRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryCoord_TransferReplica_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) ListResourceGroups(ctx context.Context, in *milvuspb.ListResourceGroupsRequest, opts ...grpc.CallOption) (*milvuspb.ListResourceGroupsResponse, error) { + out := new(milvuspb.ListResourceGroupsResponse) + err := c.cc.Invoke(ctx, QueryCoord_ListResourceGroups_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) DescribeResourceGroup(ctx context.Context, in *DescribeResourceGroupRequest, opts ...grpc.CallOption) (*DescribeResourceGroupResponse, error) { + out := new(DescribeResourceGroupResponse) + err := c.cc.Invoke(ctx, QueryCoord_DescribeResourceGroup_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) ListCheckers(ctx context.Context, in *ListCheckersRequest, opts ...grpc.CallOption) (*ListCheckersResponse, error) { + out := new(ListCheckersResponse) + err := c.cc.Invoke(ctx, QueryCoord_ListCheckers_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) ActivateChecker(ctx context.Context, in *ActivateCheckerRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryCoord_ActivateChecker_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) DeactivateChecker(ctx context.Context, in *DeactivateCheckerRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryCoord_DeactivateChecker_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) ListQueryNode(ctx context.Context, in *ListQueryNodeRequest, opts ...grpc.CallOption) (*ListQueryNodeResponse, error) { + out := new(ListQueryNodeResponse) + err := c.cc.Invoke(ctx, QueryCoord_ListQueryNode_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) GetQueryNodeDistribution(ctx context.Context, in *GetQueryNodeDistributionRequest, opts ...grpc.CallOption) (*GetQueryNodeDistributionResponse, error) { + out := new(GetQueryNodeDistributionResponse) + err := c.cc.Invoke(ctx, QueryCoord_GetQueryNodeDistribution_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) SuspendBalance(ctx context.Context, in *SuspendBalanceRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryCoord_SuspendBalance_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) ResumeBalance(ctx context.Context, in *ResumeBalanceRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryCoord_ResumeBalance_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) SuspendNode(ctx context.Context, in *SuspendNodeRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryCoord_SuspendNode_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) ResumeNode(ctx context.Context, in *ResumeNodeRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryCoord_ResumeNode_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) TransferSegment(ctx context.Context, in *TransferSegmentRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryCoord_TransferSegment_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) TransferChannel(ctx context.Context, in *TransferChannelRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryCoord_TransferChannel_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) CheckQueryNodeDistribution(ctx context.Context, in *CheckQueryNodeDistributionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryCoord_CheckQueryNodeDistribution_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryCoordClient) UpdateLoadConfig(ctx context.Context, in *UpdateLoadConfigRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryCoord_UpdateLoadConfig_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// QueryCoordServer is the server API for QueryCoord service. +// All implementations should embed UnimplementedQueryCoordServer +// for forward compatibility +type QueryCoordServer interface { + GetComponentStates(context.Context, *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) + GetTimeTickChannel(context.Context, *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error) + GetStatisticsChannel(context.Context, *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) + ShowCollections(context.Context, *ShowCollectionsRequest) (*ShowCollectionsResponse, error) + ShowPartitions(context.Context, *ShowPartitionsRequest) (*ShowPartitionsResponse, error) + LoadPartitions(context.Context, *LoadPartitionsRequest) (*commonpb.Status, error) + ReleasePartitions(context.Context, *ReleasePartitionsRequest) (*commonpb.Status, error) + LoadCollection(context.Context, *LoadCollectionRequest) (*commonpb.Status, error) + ReleaseCollection(context.Context, *ReleaseCollectionRequest) (*commonpb.Status, error) + SyncNewCreatedPartition(context.Context, *SyncNewCreatedPartitionRequest) (*commonpb.Status, error) + GetPartitionStates(context.Context, *GetPartitionStatesRequest) (*GetPartitionStatesResponse, error) + GetSegmentInfo(context.Context, *GetSegmentInfoRequest) (*GetSegmentInfoResponse, error) + LoadBalance(context.Context, *LoadBalanceRequest) (*commonpb.Status, error) + ShowConfigurations(context.Context, *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) + // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy + GetMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) + // https://wiki.lfaidata.foundation/display/MIL/MEP+23+--+Multiple+memory+replication+design + GetReplicas(context.Context, *milvuspb.GetReplicasRequest) (*milvuspb.GetReplicasResponse, error) + GetShardLeaders(context.Context, *GetShardLeadersRequest) (*GetShardLeadersResponse, error) + CheckHealth(context.Context, *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error) + CreateResourceGroup(context.Context, *milvuspb.CreateResourceGroupRequest) (*commonpb.Status, error) + UpdateResourceGroups(context.Context, *UpdateResourceGroupsRequest) (*commonpb.Status, error) + DropResourceGroup(context.Context, *milvuspb.DropResourceGroupRequest) (*commonpb.Status, error) + TransferNode(context.Context, *milvuspb.TransferNodeRequest) (*commonpb.Status, error) + TransferReplica(context.Context, *TransferReplicaRequest) (*commonpb.Status, error) + ListResourceGroups(context.Context, *milvuspb.ListResourceGroupsRequest) (*milvuspb.ListResourceGroupsResponse, error) + DescribeResourceGroup(context.Context, *DescribeResourceGroupRequest) (*DescribeResourceGroupResponse, error) + // ops interfaces + ListCheckers(context.Context, *ListCheckersRequest) (*ListCheckersResponse, error) + ActivateChecker(context.Context, *ActivateCheckerRequest) (*commonpb.Status, error) + DeactivateChecker(context.Context, *DeactivateCheckerRequest) (*commonpb.Status, error) + ListQueryNode(context.Context, *ListQueryNodeRequest) (*ListQueryNodeResponse, error) + GetQueryNodeDistribution(context.Context, *GetQueryNodeDistributionRequest) (*GetQueryNodeDistributionResponse, error) + SuspendBalance(context.Context, *SuspendBalanceRequest) (*commonpb.Status, error) + ResumeBalance(context.Context, *ResumeBalanceRequest) (*commonpb.Status, error) + SuspendNode(context.Context, *SuspendNodeRequest) (*commonpb.Status, error) + ResumeNode(context.Context, *ResumeNodeRequest) (*commonpb.Status, error) + TransferSegment(context.Context, *TransferSegmentRequest) (*commonpb.Status, error) + TransferChannel(context.Context, *TransferChannelRequest) (*commonpb.Status, error) + CheckQueryNodeDistribution(context.Context, *CheckQueryNodeDistributionRequest) (*commonpb.Status, error) + UpdateLoadConfig(context.Context, *UpdateLoadConfigRequest) (*commonpb.Status, error) +} + +// UnimplementedQueryCoordServer should be embedded to have forward compatible implementations. +type UnimplementedQueryCoordServer struct { +} + +func (UnimplementedQueryCoordServer) GetComponentStates(context.Context, *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetComponentStates not implemented") +} +func (UnimplementedQueryCoordServer) GetTimeTickChannel(context.Context, *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetTimeTickChannel not implemented") +} +func (UnimplementedQueryCoordServer) GetStatisticsChannel(context.Context, *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetStatisticsChannel not implemented") +} +func (UnimplementedQueryCoordServer) ShowCollections(context.Context, *ShowCollectionsRequest) (*ShowCollectionsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ShowCollections not implemented") +} +func (UnimplementedQueryCoordServer) ShowPartitions(context.Context, *ShowPartitionsRequest) (*ShowPartitionsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ShowPartitions not implemented") +} +func (UnimplementedQueryCoordServer) LoadPartitions(context.Context, *LoadPartitionsRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method LoadPartitions not implemented") +} +func (UnimplementedQueryCoordServer) ReleasePartitions(context.Context, *ReleasePartitionsRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method ReleasePartitions not implemented") +} +func (UnimplementedQueryCoordServer) LoadCollection(context.Context, *LoadCollectionRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method LoadCollection not implemented") +} +func (UnimplementedQueryCoordServer) ReleaseCollection(context.Context, *ReleaseCollectionRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method ReleaseCollection not implemented") +} +func (UnimplementedQueryCoordServer) SyncNewCreatedPartition(context.Context, *SyncNewCreatedPartitionRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method SyncNewCreatedPartition not implemented") +} +func (UnimplementedQueryCoordServer) GetPartitionStates(context.Context, *GetPartitionStatesRequest) (*GetPartitionStatesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetPartitionStates not implemented") +} +func (UnimplementedQueryCoordServer) GetSegmentInfo(context.Context, *GetSegmentInfoRequest) (*GetSegmentInfoResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetSegmentInfo not implemented") +} +func (UnimplementedQueryCoordServer) LoadBalance(context.Context, *LoadBalanceRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method LoadBalance not implemented") +} +func (UnimplementedQueryCoordServer) ShowConfigurations(context.Context, *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ShowConfigurations not implemented") +} +func (UnimplementedQueryCoordServer) GetMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetMetrics not implemented") +} +func (UnimplementedQueryCoordServer) GetReplicas(context.Context, *milvuspb.GetReplicasRequest) (*milvuspb.GetReplicasResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetReplicas not implemented") +} +func (UnimplementedQueryCoordServer) GetShardLeaders(context.Context, *GetShardLeadersRequest) (*GetShardLeadersResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetShardLeaders not implemented") +} +func (UnimplementedQueryCoordServer) CheckHealth(context.Context, *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method CheckHealth not implemented") +} +func (UnimplementedQueryCoordServer) CreateResourceGroup(context.Context, *milvuspb.CreateResourceGroupRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method CreateResourceGroup not implemented") +} +func (UnimplementedQueryCoordServer) UpdateResourceGroups(context.Context, *UpdateResourceGroupsRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateResourceGroups not implemented") +} +func (UnimplementedQueryCoordServer) DropResourceGroup(context.Context, *milvuspb.DropResourceGroupRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method DropResourceGroup not implemented") +} +func (UnimplementedQueryCoordServer) TransferNode(context.Context, *milvuspb.TransferNodeRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method TransferNode not implemented") +} +func (UnimplementedQueryCoordServer) TransferReplica(context.Context, *TransferReplicaRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method TransferReplica not implemented") +} +func (UnimplementedQueryCoordServer) ListResourceGroups(context.Context, *milvuspb.ListResourceGroupsRequest) (*milvuspb.ListResourceGroupsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListResourceGroups not implemented") +} +func (UnimplementedQueryCoordServer) DescribeResourceGroup(context.Context, *DescribeResourceGroupRequest) (*DescribeResourceGroupResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DescribeResourceGroup not implemented") +} +func (UnimplementedQueryCoordServer) ListCheckers(context.Context, *ListCheckersRequest) (*ListCheckersResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListCheckers not implemented") +} +func (UnimplementedQueryCoordServer) ActivateChecker(context.Context, *ActivateCheckerRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method ActivateChecker not implemented") +} +func (UnimplementedQueryCoordServer) DeactivateChecker(context.Context, *DeactivateCheckerRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method DeactivateChecker not implemented") +} +func (UnimplementedQueryCoordServer) ListQueryNode(context.Context, *ListQueryNodeRequest) (*ListQueryNodeResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListQueryNode not implemented") +} +func (UnimplementedQueryCoordServer) GetQueryNodeDistribution(context.Context, *GetQueryNodeDistributionRequest) (*GetQueryNodeDistributionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetQueryNodeDistribution not implemented") +} +func (UnimplementedQueryCoordServer) SuspendBalance(context.Context, *SuspendBalanceRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method SuspendBalance not implemented") +} +func (UnimplementedQueryCoordServer) ResumeBalance(context.Context, *ResumeBalanceRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResumeBalance not implemented") +} +func (UnimplementedQueryCoordServer) SuspendNode(context.Context, *SuspendNodeRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method SuspendNode not implemented") +} +func (UnimplementedQueryCoordServer) ResumeNode(context.Context, *ResumeNodeRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResumeNode not implemented") +} +func (UnimplementedQueryCoordServer) TransferSegment(context.Context, *TransferSegmentRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method TransferSegment not implemented") +} +func (UnimplementedQueryCoordServer) TransferChannel(context.Context, *TransferChannelRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method TransferChannel not implemented") +} +func (UnimplementedQueryCoordServer) CheckQueryNodeDistribution(context.Context, *CheckQueryNodeDistributionRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method CheckQueryNodeDistribution not implemented") +} +func (UnimplementedQueryCoordServer) UpdateLoadConfig(context.Context, *UpdateLoadConfigRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateLoadConfig not implemented") +} + +// UnsafeQueryCoordServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to QueryCoordServer will +// result in compilation errors. +type UnsafeQueryCoordServer interface { + mustEmbedUnimplementedQueryCoordServer() +} + +func RegisterQueryCoordServer(s grpc.ServiceRegistrar, srv QueryCoordServer) { + s.RegisterService(&QueryCoord_ServiceDesc, srv) +} + +func _QueryCoord_GetComponentStates_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.GetComponentStatesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).GetComponentStates(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_GetComponentStates_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).GetComponentStates(ctx, req.(*milvuspb.GetComponentStatesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_GetTimeTickChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.GetTimeTickChannelRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).GetTimeTickChannel(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_GetTimeTickChannel_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).GetTimeTickChannel(ctx, req.(*internalpb.GetTimeTickChannelRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_GetStatisticsChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.GetStatisticsChannelRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).GetStatisticsChannel(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_GetStatisticsChannel_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).GetStatisticsChannel(ctx, req.(*internalpb.GetStatisticsChannelRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_ShowCollections_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ShowCollectionsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).ShowCollections(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_ShowCollections_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).ShowCollections(ctx, req.(*ShowCollectionsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_ShowPartitions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ShowPartitionsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).ShowPartitions(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_ShowPartitions_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).ShowPartitions(ctx, req.(*ShowPartitionsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_LoadPartitions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(LoadPartitionsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).LoadPartitions(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_LoadPartitions_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).LoadPartitions(ctx, req.(*LoadPartitionsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_ReleasePartitions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ReleasePartitionsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).ReleasePartitions(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_ReleasePartitions_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).ReleasePartitions(ctx, req.(*ReleasePartitionsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_LoadCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(LoadCollectionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).LoadCollection(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_LoadCollection_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).LoadCollection(ctx, req.(*LoadCollectionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_ReleaseCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ReleaseCollectionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).ReleaseCollection(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_ReleaseCollection_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).ReleaseCollection(ctx, req.(*ReleaseCollectionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_SyncNewCreatedPartition_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(SyncNewCreatedPartitionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).SyncNewCreatedPartition(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_SyncNewCreatedPartition_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).SyncNewCreatedPartition(ctx, req.(*SyncNewCreatedPartitionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_GetPartitionStates_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetPartitionStatesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).GetPartitionStates(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_GetPartitionStates_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).GetPartitionStates(ctx, req.(*GetPartitionStatesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_GetSegmentInfo_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetSegmentInfoRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).GetSegmentInfo(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_GetSegmentInfo_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).GetSegmentInfo(ctx, req.(*GetSegmentInfoRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_LoadBalance_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(LoadBalanceRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).LoadBalance(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_LoadBalance_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).LoadBalance(ctx, req.(*LoadBalanceRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_ShowConfigurations_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.ShowConfigurationsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).ShowConfigurations(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_ShowConfigurations_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).ShowConfigurations(ctx, req.(*internalpb.ShowConfigurationsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_GetMetrics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.GetMetricsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).GetMetrics(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_GetMetrics_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).GetMetrics(ctx, req.(*milvuspb.GetMetricsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_GetReplicas_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.GetReplicasRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).GetReplicas(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_GetReplicas_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).GetReplicas(ctx, req.(*milvuspb.GetReplicasRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_GetShardLeaders_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetShardLeadersRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).GetShardLeaders(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_GetShardLeaders_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).GetShardLeaders(ctx, req.(*GetShardLeadersRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_CheckHealth_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.CheckHealthRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).CheckHealth(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_CheckHealth_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).CheckHealth(ctx, req.(*milvuspb.CheckHealthRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_CreateResourceGroup_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.CreateResourceGroupRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).CreateResourceGroup(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_CreateResourceGroup_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).CreateResourceGroup(ctx, req.(*milvuspb.CreateResourceGroupRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_UpdateResourceGroups_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(UpdateResourceGroupsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).UpdateResourceGroups(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_UpdateResourceGroups_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).UpdateResourceGroups(ctx, req.(*UpdateResourceGroupsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_DropResourceGroup_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.DropResourceGroupRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).DropResourceGroup(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_DropResourceGroup_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).DropResourceGroup(ctx, req.(*milvuspb.DropResourceGroupRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_TransferNode_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.TransferNodeRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).TransferNode(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_TransferNode_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).TransferNode(ctx, req.(*milvuspb.TransferNodeRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_TransferReplica_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(TransferReplicaRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).TransferReplica(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_TransferReplica_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).TransferReplica(ctx, req.(*TransferReplicaRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_ListResourceGroups_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.ListResourceGroupsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).ListResourceGroups(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_ListResourceGroups_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).ListResourceGroups(ctx, req.(*milvuspb.ListResourceGroupsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_DescribeResourceGroup_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DescribeResourceGroupRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).DescribeResourceGroup(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_DescribeResourceGroup_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).DescribeResourceGroup(ctx, req.(*DescribeResourceGroupRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_ListCheckers_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ListCheckersRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).ListCheckers(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_ListCheckers_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).ListCheckers(ctx, req.(*ListCheckersRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_ActivateChecker_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ActivateCheckerRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).ActivateChecker(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_ActivateChecker_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).ActivateChecker(ctx, req.(*ActivateCheckerRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_DeactivateChecker_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DeactivateCheckerRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).DeactivateChecker(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_DeactivateChecker_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).DeactivateChecker(ctx, req.(*DeactivateCheckerRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_ListQueryNode_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ListQueryNodeRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).ListQueryNode(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_ListQueryNode_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).ListQueryNode(ctx, req.(*ListQueryNodeRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_GetQueryNodeDistribution_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetQueryNodeDistributionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).GetQueryNodeDistribution(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_GetQueryNodeDistribution_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).GetQueryNodeDistribution(ctx, req.(*GetQueryNodeDistributionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_SuspendBalance_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(SuspendBalanceRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).SuspendBalance(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_SuspendBalance_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).SuspendBalance(ctx, req.(*SuspendBalanceRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_ResumeBalance_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResumeBalanceRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).ResumeBalance(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_ResumeBalance_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).ResumeBalance(ctx, req.(*ResumeBalanceRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_SuspendNode_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(SuspendNodeRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).SuspendNode(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_SuspendNode_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).SuspendNode(ctx, req.(*SuspendNodeRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_ResumeNode_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResumeNodeRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).ResumeNode(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_ResumeNode_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).ResumeNode(ctx, req.(*ResumeNodeRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_TransferSegment_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(TransferSegmentRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).TransferSegment(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_TransferSegment_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).TransferSegment(ctx, req.(*TransferSegmentRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_TransferChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(TransferChannelRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).TransferChannel(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_TransferChannel_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).TransferChannel(ctx, req.(*TransferChannelRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_CheckQueryNodeDistribution_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CheckQueryNodeDistributionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).CheckQueryNodeDistribution(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_CheckQueryNodeDistribution_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).CheckQueryNodeDistribution(ctx, req.(*CheckQueryNodeDistributionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryCoord_UpdateLoadConfig_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(UpdateLoadConfigRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryCoordServer).UpdateLoadConfig(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryCoord_UpdateLoadConfig_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryCoordServer).UpdateLoadConfig(ctx, req.(*UpdateLoadConfigRequest)) + } + return interceptor(ctx, in, info, handler) +} + +// QueryCoord_ServiceDesc is the grpc.ServiceDesc for QueryCoord service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var QueryCoord_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "milvus.proto.query.QueryCoord", + HandlerType: (*QueryCoordServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "GetComponentStates", + Handler: _QueryCoord_GetComponentStates_Handler, + }, + { + MethodName: "GetTimeTickChannel", + Handler: _QueryCoord_GetTimeTickChannel_Handler, + }, + { + MethodName: "GetStatisticsChannel", + Handler: _QueryCoord_GetStatisticsChannel_Handler, + }, + { + MethodName: "ShowCollections", + Handler: _QueryCoord_ShowCollections_Handler, + }, + { + MethodName: "ShowPartitions", + Handler: _QueryCoord_ShowPartitions_Handler, + }, + { + MethodName: "LoadPartitions", + Handler: _QueryCoord_LoadPartitions_Handler, + }, + { + MethodName: "ReleasePartitions", + Handler: _QueryCoord_ReleasePartitions_Handler, + }, + { + MethodName: "LoadCollection", + Handler: _QueryCoord_LoadCollection_Handler, + }, + { + MethodName: "ReleaseCollection", + Handler: _QueryCoord_ReleaseCollection_Handler, + }, + { + MethodName: "SyncNewCreatedPartition", + Handler: _QueryCoord_SyncNewCreatedPartition_Handler, + }, + { + MethodName: "GetPartitionStates", + Handler: _QueryCoord_GetPartitionStates_Handler, + }, + { + MethodName: "GetSegmentInfo", + Handler: _QueryCoord_GetSegmentInfo_Handler, + }, + { + MethodName: "LoadBalance", + Handler: _QueryCoord_LoadBalance_Handler, + }, + { + MethodName: "ShowConfigurations", + Handler: _QueryCoord_ShowConfigurations_Handler, + }, + { + MethodName: "GetMetrics", + Handler: _QueryCoord_GetMetrics_Handler, + }, + { + MethodName: "GetReplicas", + Handler: _QueryCoord_GetReplicas_Handler, + }, + { + MethodName: "GetShardLeaders", + Handler: _QueryCoord_GetShardLeaders_Handler, + }, + { + MethodName: "CheckHealth", + Handler: _QueryCoord_CheckHealth_Handler, + }, + { + MethodName: "CreateResourceGroup", + Handler: _QueryCoord_CreateResourceGroup_Handler, + }, + { + MethodName: "UpdateResourceGroups", + Handler: _QueryCoord_UpdateResourceGroups_Handler, + }, + { + MethodName: "DropResourceGroup", + Handler: _QueryCoord_DropResourceGroup_Handler, + }, + { + MethodName: "TransferNode", + Handler: _QueryCoord_TransferNode_Handler, + }, + { + MethodName: "TransferReplica", + Handler: _QueryCoord_TransferReplica_Handler, + }, + { + MethodName: "ListResourceGroups", + Handler: _QueryCoord_ListResourceGroups_Handler, + }, + { + MethodName: "DescribeResourceGroup", + Handler: _QueryCoord_DescribeResourceGroup_Handler, + }, + { + MethodName: "ListCheckers", + Handler: _QueryCoord_ListCheckers_Handler, + }, + { + MethodName: "ActivateChecker", + Handler: _QueryCoord_ActivateChecker_Handler, + }, + { + MethodName: "DeactivateChecker", + Handler: _QueryCoord_DeactivateChecker_Handler, + }, + { + MethodName: "ListQueryNode", + Handler: _QueryCoord_ListQueryNode_Handler, + }, + { + MethodName: "GetQueryNodeDistribution", + Handler: _QueryCoord_GetQueryNodeDistribution_Handler, + }, + { + MethodName: "SuspendBalance", + Handler: _QueryCoord_SuspendBalance_Handler, + }, + { + MethodName: "ResumeBalance", + Handler: _QueryCoord_ResumeBalance_Handler, + }, + { + MethodName: "SuspendNode", + Handler: _QueryCoord_SuspendNode_Handler, + }, + { + MethodName: "ResumeNode", + Handler: _QueryCoord_ResumeNode_Handler, + }, + { + MethodName: "TransferSegment", + Handler: _QueryCoord_TransferSegment_Handler, + }, + { + MethodName: "TransferChannel", + Handler: _QueryCoord_TransferChannel_Handler, + }, + { + MethodName: "CheckQueryNodeDistribution", + Handler: _QueryCoord_CheckQueryNodeDistribution_Handler, + }, + { + MethodName: "UpdateLoadConfig", + Handler: _QueryCoord_UpdateLoadConfig_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "query_coord.proto", +} + +const ( + QueryNode_GetComponentStates_FullMethodName = "/milvus.proto.query.QueryNode/GetComponentStates" + QueryNode_GetTimeTickChannel_FullMethodName = "/milvus.proto.query.QueryNode/GetTimeTickChannel" + QueryNode_GetStatisticsChannel_FullMethodName = "/milvus.proto.query.QueryNode/GetStatisticsChannel" + QueryNode_WatchDmChannels_FullMethodName = "/milvus.proto.query.QueryNode/WatchDmChannels" + QueryNode_UnsubDmChannel_FullMethodName = "/milvus.proto.query.QueryNode/UnsubDmChannel" + QueryNode_LoadSegments_FullMethodName = "/milvus.proto.query.QueryNode/LoadSegments" + QueryNode_ReleaseCollection_FullMethodName = "/milvus.proto.query.QueryNode/ReleaseCollection" + QueryNode_LoadPartitions_FullMethodName = "/milvus.proto.query.QueryNode/LoadPartitions" + QueryNode_ReleasePartitions_FullMethodName = "/milvus.proto.query.QueryNode/ReleasePartitions" + QueryNode_ReleaseSegments_FullMethodName = "/milvus.proto.query.QueryNode/ReleaseSegments" + QueryNode_GetSegmentInfo_FullMethodName = "/milvus.proto.query.QueryNode/GetSegmentInfo" + QueryNode_SyncReplicaSegments_FullMethodName = "/milvus.proto.query.QueryNode/SyncReplicaSegments" + QueryNode_GetStatistics_FullMethodName = "/milvus.proto.query.QueryNode/GetStatistics" + QueryNode_Search_FullMethodName = "/milvus.proto.query.QueryNode/Search" + QueryNode_SearchSegments_FullMethodName = "/milvus.proto.query.QueryNode/SearchSegments" + QueryNode_Query_FullMethodName = "/milvus.proto.query.QueryNode/Query" + QueryNode_QueryStream_FullMethodName = "/milvus.proto.query.QueryNode/QueryStream" + QueryNode_QuerySegments_FullMethodName = "/milvus.proto.query.QueryNode/QuerySegments" + QueryNode_QueryStreamSegments_FullMethodName = "/milvus.proto.query.QueryNode/QueryStreamSegments" + QueryNode_ShowConfigurations_FullMethodName = "/milvus.proto.query.QueryNode/ShowConfigurations" + QueryNode_GetMetrics_FullMethodName = "/milvus.proto.query.QueryNode/GetMetrics" + QueryNode_GetDataDistribution_FullMethodName = "/milvus.proto.query.QueryNode/GetDataDistribution" + QueryNode_SyncDistribution_FullMethodName = "/milvus.proto.query.QueryNode/SyncDistribution" + QueryNode_Delete_FullMethodName = "/milvus.proto.query.QueryNode/Delete" + QueryNode_DeleteBatch_FullMethodName = "/milvus.proto.query.QueryNode/DeleteBatch" +) + +// QueryNodeClient is the client API for QueryNode service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +type QueryNodeClient interface { + GetComponentStates(ctx context.Context, in *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) + GetTimeTickChannel(ctx context.Context, in *internalpb.GetTimeTickChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) + GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) + WatchDmChannels(ctx context.Context, in *WatchDmChannelsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + UnsubDmChannel(ctx context.Context, in *UnsubDmChannelRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + LoadSegments(ctx context.Context, in *LoadSegmentsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + ReleaseCollection(ctx context.Context, in *ReleaseCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + LoadPartitions(ctx context.Context, in *LoadPartitionsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + ReleasePartitions(ctx context.Context, in *ReleasePartitionsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + ReleaseSegments(ctx context.Context, in *ReleaseSegmentsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + GetSegmentInfo(ctx context.Context, in *GetSegmentInfoRequest, opts ...grpc.CallOption) (*GetSegmentInfoResponse, error) + SyncReplicaSegments(ctx context.Context, in *SyncReplicaSegmentsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + GetStatistics(ctx context.Context, in *GetStatisticsRequest, opts ...grpc.CallOption) (*internalpb.GetStatisticsResponse, error) + Search(ctx context.Context, in *SearchRequest, opts ...grpc.CallOption) (*internalpb.SearchResults, error) + SearchSegments(ctx context.Context, in *SearchRequest, opts ...grpc.CallOption) (*internalpb.SearchResults, error) + Query(ctx context.Context, in *QueryRequest, opts ...grpc.CallOption) (*internalpb.RetrieveResults, error) + QueryStream(ctx context.Context, in *QueryRequest, opts ...grpc.CallOption) (QueryNode_QueryStreamClient, error) + QuerySegments(ctx context.Context, in *QueryRequest, opts ...grpc.CallOption) (*internalpb.RetrieveResults, error) + QueryStreamSegments(ctx context.Context, in *QueryRequest, opts ...grpc.CallOption) (QueryNode_QueryStreamSegmentsClient, error) + ShowConfigurations(ctx context.Context, in *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error) + // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy + GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) + GetDataDistribution(ctx context.Context, in *GetDataDistributionRequest, opts ...grpc.CallOption) (*GetDataDistributionResponse, error) + SyncDistribution(ctx context.Context, in *SyncDistributionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + Delete(ctx context.Context, in *DeleteRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + // 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) +} + +type queryNodeClient struct { + cc grpc.ClientConnInterface +} + +func NewQueryNodeClient(cc grpc.ClientConnInterface) QueryNodeClient { + return &queryNodeClient{cc} +} + +func (c *queryNodeClient) GetComponentStates(ctx context.Context, in *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) { + out := new(milvuspb.ComponentStates) + err := c.cc.Invoke(ctx, QueryNode_GetComponentStates_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryNodeClient) GetTimeTickChannel(ctx context.Context, in *internalpb.GetTimeTickChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) { + out := new(milvuspb.StringResponse) + err := c.cc.Invoke(ctx, QueryNode_GetTimeTickChannel_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryNodeClient) GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) { + out := new(milvuspb.StringResponse) + err := c.cc.Invoke(ctx, QueryNode_GetStatisticsChannel_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryNodeClient) WatchDmChannels(ctx context.Context, in *WatchDmChannelsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryNode_WatchDmChannels_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryNodeClient) UnsubDmChannel(ctx context.Context, in *UnsubDmChannelRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryNode_UnsubDmChannel_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryNodeClient) LoadSegments(ctx context.Context, in *LoadSegmentsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryNode_LoadSegments_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryNodeClient) ReleaseCollection(ctx context.Context, in *ReleaseCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryNode_ReleaseCollection_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryNodeClient) LoadPartitions(ctx context.Context, in *LoadPartitionsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryNode_LoadPartitions_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryNodeClient) ReleasePartitions(ctx context.Context, in *ReleasePartitionsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryNode_ReleasePartitions_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryNodeClient) ReleaseSegments(ctx context.Context, in *ReleaseSegmentsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryNode_ReleaseSegments_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryNodeClient) GetSegmentInfo(ctx context.Context, in *GetSegmentInfoRequest, opts ...grpc.CallOption) (*GetSegmentInfoResponse, error) { + out := new(GetSegmentInfoResponse) + err := c.cc.Invoke(ctx, QueryNode_GetSegmentInfo_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryNodeClient) SyncReplicaSegments(ctx context.Context, in *SyncReplicaSegmentsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryNode_SyncReplicaSegments_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryNodeClient) GetStatistics(ctx context.Context, in *GetStatisticsRequest, opts ...grpc.CallOption) (*internalpb.GetStatisticsResponse, error) { + out := new(internalpb.GetStatisticsResponse) + err := c.cc.Invoke(ctx, QueryNode_GetStatistics_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryNodeClient) Search(ctx context.Context, in *SearchRequest, opts ...grpc.CallOption) (*internalpb.SearchResults, error) { + out := new(internalpb.SearchResults) + err := c.cc.Invoke(ctx, QueryNode_Search_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryNodeClient) SearchSegments(ctx context.Context, in *SearchRequest, opts ...grpc.CallOption) (*internalpb.SearchResults, error) { + out := new(internalpb.SearchResults) + err := c.cc.Invoke(ctx, QueryNode_SearchSegments_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryNodeClient) Query(ctx context.Context, in *QueryRequest, opts ...grpc.CallOption) (*internalpb.RetrieveResults, error) { + out := new(internalpb.RetrieveResults) + err := c.cc.Invoke(ctx, QueryNode_Query_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryNodeClient) QueryStream(ctx context.Context, in *QueryRequest, opts ...grpc.CallOption) (QueryNode_QueryStreamClient, error) { + stream, err := c.cc.NewStream(ctx, &QueryNode_ServiceDesc.Streams[0], QueryNode_QueryStream_FullMethodName, opts...) + if err != nil { + return nil, err + } + x := &queryNodeQueryStreamClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type QueryNode_QueryStreamClient interface { + Recv() (*internalpb.RetrieveResults, error) + grpc.ClientStream +} + +type queryNodeQueryStreamClient struct { + grpc.ClientStream +} + +func (x *queryNodeQueryStreamClient) Recv() (*internalpb.RetrieveResults, error) { + m := new(internalpb.RetrieveResults) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *queryNodeClient) QuerySegments(ctx context.Context, in *QueryRequest, opts ...grpc.CallOption) (*internalpb.RetrieveResults, error) { + out := new(internalpb.RetrieveResults) + err := c.cc.Invoke(ctx, QueryNode_QuerySegments_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryNodeClient) QueryStreamSegments(ctx context.Context, in *QueryRequest, opts ...grpc.CallOption) (QueryNode_QueryStreamSegmentsClient, error) { + stream, err := c.cc.NewStream(ctx, &QueryNode_ServiceDesc.Streams[1], QueryNode_QueryStreamSegments_FullMethodName, opts...) + if err != nil { + return nil, err + } + x := &queryNodeQueryStreamSegmentsClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type QueryNode_QueryStreamSegmentsClient interface { + Recv() (*internalpb.RetrieveResults, error) + grpc.ClientStream +} + +type queryNodeQueryStreamSegmentsClient struct { + grpc.ClientStream +} + +func (x *queryNodeQueryStreamSegmentsClient) Recv() (*internalpb.RetrieveResults, error) { + m := new(internalpb.RetrieveResults) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *queryNodeClient) ShowConfigurations(ctx context.Context, in *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error) { + out := new(internalpb.ShowConfigurationsResponse) + err := c.cc.Invoke(ctx, QueryNode_ShowConfigurations_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryNodeClient) GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) { + out := new(milvuspb.GetMetricsResponse) + err := c.cc.Invoke(ctx, QueryNode_GetMetrics_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryNodeClient) GetDataDistribution(ctx context.Context, in *GetDataDistributionRequest, opts ...grpc.CallOption) (*GetDataDistributionResponse, error) { + out := new(GetDataDistributionResponse) + err := c.cc.Invoke(ctx, QueryNode_GetDataDistribution_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryNodeClient) SyncDistribution(ctx context.Context, in *SyncDistributionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryNode_SyncDistribution_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryNodeClient) Delete(ctx context.Context, in *DeleteRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryNode_Delete_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *queryNodeClient) DeleteBatch(ctx context.Context, in *DeleteBatchRequest, opts ...grpc.CallOption) (*DeleteBatchResponse, error) { + out := new(DeleteBatchResponse) + err := c.cc.Invoke(ctx, QueryNode_DeleteBatch_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 +type QueryNodeServer interface { + GetComponentStates(context.Context, *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) + GetTimeTickChannel(context.Context, *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error) + GetStatisticsChannel(context.Context, *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) + WatchDmChannels(context.Context, *WatchDmChannelsRequest) (*commonpb.Status, error) + UnsubDmChannel(context.Context, *UnsubDmChannelRequest) (*commonpb.Status, error) + LoadSegments(context.Context, *LoadSegmentsRequest) (*commonpb.Status, error) + ReleaseCollection(context.Context, *ReleaseCollectionRequest) (*commonpb.Status, error) + LoadPartitions(context.Context, *LoadPartitionsRequest) (*commonpb.Status, error) + ReleasePartitions(context.Context, *ReleasePartitionsRequest) (*commonpb.Status, error) + ReleaseSegments(context.Context, *ReleaseSegmentsRequest) (*commonpb.Status, error) + GetSegmentInfo(context.Context, *GetSegmentInfoRequest) (*GetSegmentInfoResponse, error) + SyncReplicaSegments(context.Context, *SyncReplicaSegmentsRequest) (*commonpb.Status, error) + GetStatistics(context.Context, *GetStatisticsRequest) (*internalpb.GetStatisticsResponse, error) + Search(context.Context, *SearchRequest) (*internalpb.SearchResults, error) + SearchSegments(context.Context, *SearchRequest) (*internalpb.SearchResults, error) + Query(context.Context, *QueryRequest) (*internalpb.RetrieveResults, error) + QueryStream(*QueryRequest, QueryNode_QueryStreamServer) error + QuerySegments(context.Context, *QueryRequest) (*internalpb.RetrieveResults, error) + QueryStreamSegments(*QueryRequest, QueryNode_QueryStreamSegmentsServer) error + ShowConfigurations(context.Context, *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) + // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy + GetMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) + GetDataDistribution(context.Context, *GetDataDistributionRequest) (*GetDataDistributionResponse, error) + SyncDistribution(context.Context, *SyncDistributionRequest) (*commonpb.Status, error) + Delete(context.Context, *DeleteRequest) (*commonpb.Status, error) + // 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) +} + +// UnimplementedQueryNodeServer should be embedded to have forward compatible implementations. +type UnimplementedQueryNodeServer struct { +} + +func (UnimplementedQueryNodeServer) GetComponentStates(context.Context, *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetComponentStates not implemented") +} +func (UnimplementedQueryNodeServer) GetTimeTickChannel(context.Context, *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetTimeTickChannel not implemented") +} +func (UnimplementedQueryNodeServer) GetStatisticsChannel(context.Context, *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetStatisticsChannel not implemented") +} +func (UnimplementedQueryNodeServer) WatchDmChannels(context.Context, *WatchDmChannelsRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method WatchDmChannels not implemented") +} +func (UnimplementedQueryNodeServer) UnsubDmChannel(context.Context, *UnsubDmChannelRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method UnsubDmChannel not implemented") +} +func (UnimplementedQueryNodeServer) LoadSegments(context.Context, *LoadSegmentsRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method LoadSegments not implemented") +} +func (UnimplementedQueryNodeServer) ReleaseCollection(context.Context, *ReleaseCollectionRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method ReleaseCollection not implemented") +} +func (UnimplementedQueryNodeServer) LoadPartitions(context.Context, *LoadPartitionsRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method LoadPartitions not implemented") +} +func (UnimplementedQueryNodeServer) ReleasePartitions(context.Context, *ReleasePartitionsRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method ReleasePartitions not implemented") +} +func (UnimplementedQueryNodeServer) ReleaseSegments(context.Context, *ReleaseSegmentsRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method ReleaseSegments not implemented") +} +func (UnimplementedQueryNodeServer) GetSegmentInfo(context.Context, *GetSegmentInfoRequest) (*GetSegmentInfoResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetSegmentInfo not implemented") +} +func (UnimplementedQueryNodeServer) SyncReplicaSegments(context.Context, *SyncReplicaSegmentsRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method SyncReplicaSegments not implemented") +} +func (UnimplementedQueryNodeServer) GetStatistics(context.Context, *GetStatisticsRequest) (*internalpb.GetStatisticsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetStatistics not implemented") +} +func (UnimplementedQueryNodeServer) Search(context.Context, *SearchRequest) (*internalpb.SearchResults, error) { + return nil, status.Errorf(codes.Unimplemented, "method Search not implemented") +} +func (UnimplementedQueryNodeServer) SearchSegments(context.Context, *SearchRequest) (*internalpb.SearchResults, error) { + return nil, status.Errorf(codes.Unimplemented, "method SearchSegments not implemented") +} +func (UnimplementedQueryNodeServer) Query(context.Context, *QueryRequest) (*internalpb.RetrieveResults, error) { + return nil, status.Errorf(codes.Unimplemented, "method Query not implemented") +} +func (UnimplementedQueryNodeServer) QueryStream(*QueryRequest, QueryNode_QueryStreamServer) error { + return status.Errorf(codes.Unimplemented, "method QueryStream not implemented") +} +func (UnimplementedQueryNodeServer) QuerySegments(context.Context, *QueryRequest) (*internalpb.RetrieveResults, error) { + return nil, status.Errorf(codes.Unimplemented, "method QuerySegments not implemented") +} +func (UnimplementedQueryNodeServer) QueryStreamSegments(*QueryRequest, QueryNode_QueryStreamSegmentsServer) error { + return status.Errorf(codes.Unimplemented, "method QueryStreamSegments not implemented") +} +func (UnimplementedQueryNodeServer) ShowConfigurations(context.Context, *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ShowConfigurations not implemented") +} +func (UnimplementedQueryNodeServer) GetMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetMetrics not implemented") +} +func (UnimplementedQueryNodeServer) GetDataDistribution(context.Context, *GetDataDistributionRequest) (*GetDataDistributionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetDataDistribution not implemented") +} +func (UnimplementedQueryNodeServer) SyncDistribution(context.Context, *SyncDistributionRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method SyncDistribution not implemented") +} +func (UnimplementedQueryNodeServer) Delete(context.Context, *DeleteRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method Delete not implemented") +} +func (UnimplementedQueryNodeServer) DeleteBatch(context.Context, *DeleteBatchRequest) (*DeleteBatchResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DeleteBatch 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 +// result in compilation errors. +type UnsafeQueryNodeServer interface { + mustEmbedUnimplementedQueryNodeServer() +} + +func RegisterQueryNodeServer(s grpc.ServiceRegistrar, srv QueryNodeServer) { + s.RegisterService(&QueryNode_ServiceDesc, srv) +} + +func _QueryNode_GetComponentStates_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.GetComponentStatesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).GetComponentStates(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_GetComponentStates_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).GetComponentStates(ctx, req.(*milvuspb.GetComponentStatesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryNode_GetTimeTickChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.GetTimeTickChannelRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).GetTimeTickChannel(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_GetTimeTickChannel_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).GetTimeTickChannel(ctx, req.(*internalpb.GetTimeTickChannelRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryNode_GetStatisticsChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.GetStatisticsChannelRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).GetStatisticsChannel(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_GetStatisticsChannel_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).GetStatisticsChannel(ctx, req.(*internalpb.GetStatisticsChannelRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryNode_WatchDmChannels_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(WatchDmChannelsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).WatchDmChannels(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_WatchDmChannels_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).WatchDmChannels(ctx, req.(*WatchDmChannelsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryNode_UnsubDmChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(UnsubDmChannelRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).UnsubDmChannel(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_UnsubDmChannel_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).UnsubDmChannel(ctx, req.(*UnsubDmChannelRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryNode_LoadSegments_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(LoadSegmentsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).LoadSegments(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_LoadSegments_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).LoadSegments(ctx, req.(*LoadSegmentsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryNode_ReleaseCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ReleaseCollectionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).ReleaseCollection(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_ReleaseCollection_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).ReleaseCollection(ctx, req.(*ReleaseCollectionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryNode_LoadPartitions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(LoadPartitionsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).LoadPartitions(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_LoadPartitions_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).LoadPartitions(ctx, req.(*LoadPartitionsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryNode_ReleasePartitions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ReleasePartitionsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).ReleasePartitions(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_ReleasePartitions_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).ReleasePartitions(ctx, req.(*ReleasePartitionsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryNode_ReleaseSegments_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ReleaseSegmentsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).ReleaseSegments(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_ReleaseSegments_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).ReleaseSegments(ctx, req.(*ReleaseSegmentsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryNode_GetSegmentInfo_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetSegmentInfoRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).GetSegmentInfo(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_GetSegmentInfo_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).GetSegmentInfo(ctx, req.(*GetSegmentInfoRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryNode_SyncReplicaSegments_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(SyncReplicaSegmentsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).SyncReplicaSegments(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_SyncReplicaSegments_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).SyncReplicaSegments(ctx, req.(*SyncReplicaSegmentsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryNode_GetStatistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetStatisticsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).GetStatistics(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_GetStatistics_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).GetStatistics(ctx, req.(*GetStatisticsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryNode_Search_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(SearchRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).Search(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_Search_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).Search(ctx, req.(*SearchRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryNode_SearchSegments_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(SearchRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).SearchSegments(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_SearchSegments_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).SearchSegments(ctx, req.(*SearchRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryNode_Query_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).Query(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_Query_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).Query(ctx, req.(*QueryRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryNode_QueryStream_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(QueryRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(QueryNodeServer).QueryStream(m, &queryNodeQueryStreamServer{stream}) +} + +type QueryNode_QueryStreamServer interface { + Send(*internalpb.RetrieveResults) error + grpc.ServerStream +} + +type queryNodeQueryStreamServer struct { + grpc.ServerStream +} + +func (x *queryNodeQueryStreamServer) Send(m *internalpb.RetrieveResults) error { + return x.ServerStream.SendMsg(m) +} + +func _QueryNode_QuerySegments_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).QuerySegments(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_QuerySegments_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).QuerySegments(ctx, req.(*QueryRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryNode_QueryStreamSegments_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(QueryRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(QueryNodeServer).QueryStreamSegments(m, &queryNodeQueryStreamSegmentsServer{stream}) +} + +type QueryNode_QueryStreamSegmentsServer interface { + Send(*internalpb.RetrieveResults) error + grpc.ServerStream +} + +type queryNodeQueryStreamSegmentsServer struct { + grpc.ServerStream +} + +func (x *queryNodeQueryStreamSegmentsServer) Send(m *internalpb.RetrieveResults) error { + return x.ServerStream.SendMsg(m) +} + +func _QueryNode_ShowConfigurations_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.ShowConfigurationsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).ShowConfigurations(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_ShowConfigurations_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).ShowConfigurations(ctx, req.(*internalpb.ShowConfigurationsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryNode_GetMetrics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.GetMetricsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).GetMetrics(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_GetMetrics_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).GetMetrics(ctx, req.(*milvuspb.GetMetricsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryNode_GetDataDistribution_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetDataDistributionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).GetDataDistribution(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_GetDataDistribution_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).GetDataDistribution(ctx, req.(*GetDataDistributionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryNode_SyncDistribution_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(SyncDistributionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).SyncDistribution(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_SyncDistribution_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).SyncDistribution(ctx, req.(*SyncDistributionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryNode_Delete_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DeleteRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).Delete(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_Delete_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).Delete(ctx, req.(*DeleteRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _QueryNode_DeleteBatch_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DeleteBatchRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).DeleteBatch(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_DeleteBatch_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).DeleteBatch(ctx, req.(*DeleteBatchRequest)) + } + 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) +var QueryNode_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "milvus.proto.query.QueryNode", + HandlerType: (*QueryNodeServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "GetComponentStates", + Handler: _QueryNode_GetComponentStates_Handler, + }, + { + MethodName: "GetTimeTickChannel", + Handler: _QueryNode_GetTimeTickChannel_Handler, + }, + { + MethodName: "GetStatisticsChannel", + Handler: _QueryNode_GetStatisticsChannel_Handler, + }, + { + MethodName: "WatchDmChannels", + Handler: _QueryNode_WatchDmChannels_Handler, + }, + { + MethodName: "UnsubDmChannel", + Handler: _QueryNode_UnsubDmChannel_Handler, + }, + { + MethodName: "LoadSegments", + Handler: _QueryNode_LoadSegments_Handler, + }, + { + MethodName: "ReleaseCollection", + Handler: _QueryNode_ReleaseCollection_Handler, + }, + { + MethodName: "LoadPartitions", + Handler: _QueryNode_LoadPartitions_Handler, + }, + { + MethodName: "ReleasePartitions", + Handler: _QueryNode_ReleasePartitions_Handler, + }, + { + MethodName: "ReleaseSegments", + Handler: _QueryNode_ReleaseSegments_Handler, + }, + { + MethodName: "GetSegmentInfo", + Handler: _QueryNode_GetSegmentInfo_Handler, + }, + { + MethodName: "SyncReplicaSegments", + Handler: _QueryNode_SyncReplicaSegments_Handler, + }, + { + MethodName: "GetStatistics", + Handler: _QueryNode_GetStatistics_Handler, + }, + { + MethodName: "Search", + Handler: _QueryNode_Search_Handler, + }, + { + MethodName: "SearchSegments", + Handler: _QueryNode_SearchSegments_Handler, + }, + { + MethodName: "Query", + Handler: _QueryNode_Query_Handler, + }, + { + MethodName: "QuerySegments", + Handler: _QueryNode_QuerySegments_Handler, + }, + { + MethodName: "ShowConfigurations", + Handler: _QueryNode_ShowConfigurations_Handler, + }, + { + MethodName: "GetMetrics", + Handler: _QueryNode_GetMetrics_Handler, + }, + { + MethodName: "GetDataDistribution", + Handler: _QueryNode_GetDataDistribution_Handler, + }, + { + MethodName: "SyncDistribution", + Handler: _QueryNode_SyncDistribution_Handler, + }, + { + MethodName: "Delete", + Handler: _QueryNode_Delete_Handler, + }, + { + MethodName: "DeleteBatch", + Handler: _QueryNode_DeleteBatch_Handler, + }, + }, + Streams: []grpc.StreamDesc{ + { + StreamName: "QueryStream", + Handler: _QueryNode_QueryStream_Handler, + ServerStreams: true, + }, + { + StreamName: "QueryStreamSegments", + Handler: _QueryNode_QueryStreamSegments_Handler, + ServerStreams: true, + }, + }, + Metadata: "query_coord.proto", +} diff --git a/internal/proto/root_coord.proto b/pkg/proto/root_coord.proto similarity index 99% rename from internal/proto/root_coord.proto rename to pkg/proto/root_coord.proto index 4fdd208ebc418..7aa99db7c92e5 100644 --- a/internal/proto/root_coord.proto +++ b/pkg/proto/root_coord.proto @@ -1,7 +1,7 @@ syntax = "proto3"; package milvus.proto.rootcoord; -option go_package="github.com/milvus-io/milvus/internal/proto/rootcoordpb"; +option go_package="github.com/milvus-io/milvus/pkg/proto/rootcoordpb"; import "common.proto"; import "milvus.proto"; diff --git a/pkg/proto/rootcoordpb/root_coord.pb.go b/pkg/proto/rootcoordpb/root_coord.pb.go new file mode 100644 index 0000000000000..e6ddcbb94f923 --- /dev/null +++ b/pkg/proto/rootcoordpb/root_coord.pb.go @@ -0,0 +1,2093 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.33.0 +// protoc v3.21.4 +// source: root_coord.proto + +package rootcoordpb + +import ( + commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + etcdpb "github.com/milvus-io/milvus/pkg/proto/etcdpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" + proxypb "github.com/milvus-io/milvus/pkg/proto/proxypb" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type AllocTimestampRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + Count uint32 `protobuf:"varint,3,opt,name=count,proto3" json:"count,omitempty"` + BlockTimestamp uint64 `protobuf:"varint,4,opt,name=blockTimestamp,proto3" json:"blockTimestamp,omitempty"` +} + +func (x *AllocTimestampRequest) Reset() { + *x = AllocTimestampRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_root_coord_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AllocTimestampRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AllocTimestampRequest) ProtoMessage() {} + +func (x *AllocTimestampRequest) ProtoReflect() protoreflect.Message { + mi := &file_root_coord_proto_msgTypes[0] + 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 AllocTimestampRequest.ProtoReflect.Descriptor instead. +func (*AllocTimestampRequest) Descriptor() ([]byte, []int) { + return file_root_coord_proto_rawDescGZIP(), []int{0} +} + +func (x *AllocTimestampRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *AllocTimestampRequest) GetCount() uint32 { + if x != nil { + return x.Count + } + return 0 +} + +func (x *AllocTimestampRequest) GetBlockTimestamp() uint64 { + if x != nil { + return x.BlockTimestamp + } + return 0 +} + +type AllocTimestampResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + Timestamp uint64 `protobuf:"varint,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + Count uint32 `protobuf:"varint,3,opt,name=count,proto3" json:"count,omitempty"` +} + +func (x *AllocTimestampResponse) Reset() { + *x = AllocTimestampResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_root_coord_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AllocTimestampResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AllocTimestampResponse) ProtoMessage() {} + +func (x *AllocTimestampResponse) ProtoReflect() protoreflect.Message { + mi := &file_root_coord_proto_msgTypes[1] + 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 AllocTimestampResponse.ProtoReflect.Descriptor instead. +func (*AllocTimestampResponse) Descriptor() ([]byte, []int) { + return file_root_coord_proto_rawDescGZIP(), []int{1} +} + +func (x *AllocTimestampResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *AllocTimestampResponse) GetTimestamp() uint64 { + if x != nil { + return x.Timestamp + } + return 0 +} + +func (x *AllocTimestampResponse) GetCount() uint32 { + if x != nil { + return x.Count + } + return 0 +} + +type AllocIDRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + Count uint32 `protobuf:"varint,2,opt,name=count,proto3" json:"count,omitempty"` +} + +func (x *AllocIDRequest) Reset() { + *x = AllocIDRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_root_coord_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AllocIDRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AllocIDRequest) ProtoMessage() {} + +func (x *AllocIDRequest) ProtoReflect() protoreflect.Message { + mi := &file_root_coord_proto_msgTypes[2] + 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 AllocIDRequest.ProtoReflect.Descriptor instead. +func (*AllocIDRequest) Descriptor() ([]byte, []int) { + return file_root_coord_proto_rawDescGZIP(), []int{2} +} + +func (x *AllocIDRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *AllocIDRequest) GetCount() uint32 { + if x != nil { + return x.Count + } + return 0 +} + +type AllocIDResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + ID int64 `protobuf:"varint,2,opt,name=ID,proto3" json:"ID,omitempty"` + Count uint32 `protobuf:"varint,3,opt,name=count,proto3" json:"count,omitempty"` +} + +func (x *AllocIDResponse) Reset() { + *x = AllocIDResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_root_coord_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AllocIDResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AllocIDResponse) ProtoMessage() {} + +func (x *AllocIDResponse) ProtoReflect() protoreflect.Message { + mi := &file_root_coord_proto_msgTypes[3] + 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 AllocIDResponse.ProtoReflect.Descriptor instead. +func (*AllocIDResponse) Descriptor() ([]byte, []int) { + return file_root_coord_proto_rawDescGZIP(), []int{3} +} + +func (x *AllocIDResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *AllocIDResponse) GetID() int64 { + if x != nil { + return x.ID + } + return 0 +} + +func (x *AllocIDResponse) GetCount() uint32 { + if x != nil { + return x.Count + } + return 0 +} + +type DescribeSegmentsRequest 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"` + SegmentIDs []int64 `protobuf:"varint,3,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,omitempty"` +} + +func (x *DescribeSegmentsRequest) Reset() { + *x = DescribeSegmentsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_root_coord_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DescribeSegmentsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DescribeSegmentsRequest) ProtoMessage() {} + +func (x *DescribeSegmentsRequest) ProtoReflect() protoreflect.Message { + mi := &file_root_coord_proto_msgTypes[4] + 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 DescribeSegmentsRequest.ProtoReflect.Descriptor instead. +func (*DescribeSegmentsRequest) Descriptor() ([]byte, []int) { + return file_root_coord_proto_rawDescGZIP(), []int{4} +} + +func (x *DescribeSegmentsRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *DescribeSegmentsRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *DescribeSegmentsRequest) GetSegmentIDs() []int64 { + if x != nil { + return x.SegmentIDs + } + return nil +} + +type SegmentBaseInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` // in which collection. + PartitionID int64 `protobuf:"varint,2,opt,name=partitionID,proto3" json:"partitionID,omitempty"` // in which partition. + SegmentID int64 `protobuf:"varint,3,opt,name=segmentID,proto3" json:"segmentID,omitempty"` +} + +func (x *SegmentBaseInfo) Reset() { + *x = SegmentBaseInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_root_coord_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentBaseInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentBaseInfo) ProtoMessage() {} + +func (x *SegmentBaseInfo) ProtoReflect() protoreflect.Message { + mi := &file_root_coord_proto_msgTypes[5] + 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 SegmentBaseInfo.ProtoReflect.Descriptor instead. +func (*SegmentBaseInfo) Descriptor() ([]byte, []int) { + return file_root_coord_proto_rawDescGZIP(), []int{5} +} + +func (x *SegmentBaseInfo) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *SegmentBaseInfo) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *SegmentBaseInfo) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +type SegmentInfos struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + BaseInfo *SegmentBaseInfo `protobuf:"bytes,1,opt,name=base_info,json=baseInfo,proto3" json:"base_info,omitempty"` // base information. + IndexInfos []*etcdpb.SegmentIndexInfo `protobuf:"bytes,2,rep,name=index_infos,json=indexInfos,proto3" json:"index_infos,omitempty"` // multiple index infos. + ExtraIndexInfos map[int64]*etcdpb.IndexInfo `protobuf:"bytes,3,rep,name=extra_index_infos,json=extraIndexInfos,proto3" json:"extra_index_infos,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // index_id -> IndexInfo{index_name, index_id, index_params} +} + +func (x *SegmentInfos) Reset() { + *x = SegmentInfos{} + if protoimpl.UnsafeEnabled { + mi := &file_root_coord_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentInfos) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentInfos) ProtoMessage() {} + +func (x *SegmentInfos) ProtoReflect() protoreflect.Message { + mi := &file_root_coord_proto_msgTypes[6] + 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 SegmentInfos.ProtoReflect.Descriptor instead. +func (*SegmentInfos) Descriptor() ([]byte, []int) { + return file_root_coord_proto_rawDescGZIP(), []int{6} +} + +func (x *SegmentInfos) GetBaseInfo() *SegmentBaseInfo { + if x != nil { + return x.BaseInfo + } + return nil +} + +func (x *SegmentInfos) GetIndexInfos() []*etcdpb.SegmentIndexInfo { + if x != nil { + return x.IndexInfos + } + return nil +} + +func (x *SegmentInfos) GetExtraIndexInfos() map[int64]*etcdpb.IndexInfo { + if x != nil { + return x.ExtraIndexInfos + } + return nil +} + +type DescribeSegmentsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + CollectionID int64 `protobuf:"varint,2,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + SegmentInfos map[int64]*SegmentInfos `protobuf:"bytes,3,rep,name=segment_infos,json=segmentInfos,proto3" json:"segment_infos,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // segment_id -> segment infos +} + +func (x *DescribeSegmentsResponse) Reset() { + *x = DescribeSegmentsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_root_coord_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DescribeSegmentsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DescribeSegmentsResponse) ProtoMessage() {} + +func (x *DescribeSegmentsResponse) ProtoReflect() protoreflect.Message { + mi := &file_root_coord_proto_msgTypes[7] + 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 DescribeSegmentsResponse.ProtoReflect.Descriptor instead. +func (*DescribeSegmentsResponse) Descriptor() ([]byte, []int) { + return file_root_coord_proto_rawDescGZIP(), []int{7} +} + +func (x *DescribeSegmentsResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *DescribeSegmentsResponse) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *DescribeSegmentsResponse) GetSegmentInfos() map[int64]*SegmentInfos { + if x != nil { + return x.SegmentInfos + } + return nil +} + +type GetCredentialRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Not useful for now + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + // username + Username string `protobuf:"bytes,2,opt,name=username,proto3" json:"username,omitempty"` +} + +func (x *GetCredentialRequest) Reset() { + *x = GetCredentialRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_root_coord_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetCredentialRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetCredentialRequest) ProtoMessage() {} + +func (x *GetCredentialRequest) ProtoReflect() protoreflect.Message { + mi := &file_root_coord_proto_msgTypes[8] + 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 GetCredentialRequest.ProtoReflect.Descriptor instead. +func (*GetCredentialRequest) Descriptor() ([]byte, []int) { + return file_root_coord_proto_rawDescGZIP(), []int{8} +} + +func (x *GetCredentialRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *GetCredentialRequest) GetUsername() string { + if x != nil { + return x.Username + } + return "" +} + +type GetCredentialResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Contain error_code and reason + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + // username + Username string `protobuf:"bytes,2,opt,name=username,proto3" json:"username,omitempty"` + // password stored in etcd/mysql + Password string `protobuf:"bytes,3,opt,name=password,proto3" json:"password,omitempty"` +} + +func (x *GetCredentialResponse) Reset() { + *x = GetCredentialResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_root_coord_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetCredentialResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetCredentialResponse) ProtoMessage() {} + +func (x *GetCredentialResponse) ProtoReflect() protoreflect.Message { + mi := &file_root_coord_proto_msgTypes[9] + 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 GetCredentialResponse.ProtoReflect.Descriptor instead. +func (*GetCredentialResponse) Descriptor() ([]byte, []int) { + return file_root_coord_proto_rawDescGZIP(), []int{9} +} + +func (x *GetCredentialResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *GetCredentialResponse) GetUsername() string { + if x != nil { + return x.Username + } + return "" +} + +func (x *GetCredentialResponse) GetPassword() string { + if x != nil { + return x.Password + } + return "" +} + +type DescribeDatabaseRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + DbName string `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` +} + +func (x *DescribeDatabaseRequest) Reset() { + *x = DescribeDatabaseRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_root_coord_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DescribeDatabaseRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DescribeDatabaseRequest) ProtoMessage() {} + +func (x *DescribeDatabaseRequest) ProtoReflect() protoreflect.Message { + mi := &file_root_coord_proto_msgTypes[10] + 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 DescribeDatabaseRequest.ProtoReflect.Descriptor instead. +func (*DescribeDatabaseRequest) Descriptor() ([]byte, []int) { + return file_root_coord_proto_rawDescGZIP(), []int{10} +} + +func (x *DescribeDatabaseRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *DescribeDatabaseRequest) GetDbName() string { + if x != nil { + return x.DbName + } + return "" +} + +type DescribeDatabaseResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + DbName string `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` + DbID int64 `protobuf:"varint,3,opt,name=dbID,proto3" json:"dbID,omitempty"` + CreatedTimestamp uint64 `protobuf:"varint,4,opt,name=created_timestamp,json=createdTimestamp,proto3" json:"created_timestamp,omitempty"` + Properties []*commonpb.KeyValuePair `protobuf:"bytes,5,rep,name=properties,proto3" json:"properties,omitempty"` +} + +func (x *DescribeDatabaseResponse) Reset() { + *x = DescribeDatabaseResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_root_coord_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DescribeDatabaseResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DescribeDatabaseResponse) ProtoMessage() {} + +func (x *DescribeDatabaseResponse) ProtoReflect() protoreflect.Message { + mi := &file_root_coord_proto_msgTypes[11] + 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 DescribeDatabaseResponse.ProtoReflect.Descriptor instead. +func (*DescribeDatabaseResponse) Descriptor() ([]byte, []int) { + return file_root_coord_proto_rawDescGZIP(), []int{11} +} + +func (x *DescribeDatabaseResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *DescribeDatabaseResponse) GetDbName() string { + if x != nil { + return x.DbName + } + return "" +} + +func (x *DescribeDatabaseResponse) GetDbID() int64 { + if x != nil { + return x.DbID + } + return 0 +} + +func (x *DescribeDatabaseResponse) GetCreatedTimestamp() uint64 { + if x != nil { + return x.CreatedTimestamp + } + return 0 +} + +func (x *DescribeDatabaseResponse) GetProperties() []*commonpb.KeyValuePair { + if x != nil { + return x.Properties + } + return nil +} + +type AlterDatabaseRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + DbName string `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` + DbId string `protobuf:"bytes,3,opt,name=db_id,json=dbId,proto3" json:"db_id,omitempty"` + Properties []*commonpb.KeyValuePair `protobuf:"bytes,4,rep,name=properties,proto3" json:"properties,omitempty"` + DeleteKeys []string `protobuf:"bytes,5,rep,name=delete_keys,json=deleteKeys,proto3" json:"delete_keys,omitempty"` +} + +func (x *AlterDatabaseRequest) Reset() { + *x = AlterDatabaseRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_root_coord_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AlterDatabaseRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AlterDatabaseRequest) ProtoMessage() {} + +func (x *AlterDatabaseRequest) ProtoReflect() protoreflect.Message { + mi := &file_root_coord_proto_msgTypes[12] + 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 AlterDatabaseRequest.ProtoReflect.Descriptor instead. +func (*AlterDatabaseRequest) Descriptor() ([]byte, []int) { + return file_root_coord_proto_rawDescGZIP(), []int{12} +} + +func (x *AlterDatabaseRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *AlterDatabaseRequest) GetDbName() string { + if x != nil { + return x.DbName + } + return "" +} + +func (x *AlterDatabaseRequest) GetDbId() string { + if x != nil { + return x.DbId + } + return "" +} + +func (x *AlterDatabaseRequest) GetProperties() []*commonpb.KeyValuePair { + if x != nil { + return x.Properties + } + return nil +} + +func (x *AlterDatabaseRequest) GetDeleteKeys() []string { + if x != nil { + return x.DeleteKeys + } + return nil +} + +type GetPChannelInfoRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + Pchannel string `protobuf:"bytes,2,opt,name=pchannel,proto3" json:"pchannel,omitempty"` +} + +func (x *GetPChannelInfoRequest) Reset() { + *x = GetPChannelInfoRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_root_coord_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetPChannelInfoRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetPChannelInfoRequest) ProtoMessage() {} + +func (x *GetPChannelInfoRequest) ProtoReflect() protoreflect.Message { + mi := &file_root_coord_proto_msgTypes[13] + 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 GetPChannelInfoRequest.ProtoReflect.Descriptor instead. +func (*GetPChannelInfoRequest) Descriptor() ([]byte, []int) { + return file_root_coord_proto_rawDescGZIP(), []int{13} +} + +func (x *GetPChannelInfoRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *GetPChannelInfoRequest) GetPchannel() string { + if x != nil { + return x.Pchannel + } + return "" +} + +type GetPChannelInfoResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + Collections []*CollectionInfoOnPChannel `protobuf:"bytes,2,rep,name=collections,proto3" json:"collections,omitempty"` +} + +func (x *GetPChannelInfoResponse) Reset() { + *x = GetPChannelInfoResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_root_coord_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetPChannelInfoResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetPChannelInfoResponse) ProtoMessage() {} + +func (x *GetPChannelInfoResponse) ProtoReflect() protoreflect.Message { + mi := &file_root_coord_proto_msgTypes[14] + 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 GetPChannelInfoResponse.ProtoReflect.Descriptor instead. +func (*GetPChannelInfoResponse) Descriptor() ([]byte, []int) { + return file_root_coord_proto_rawDescGZIP(), []int{14} +} + +func (x *GetPChannelInfoResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *GetPChannelInfoResponse) GetCollections() []*CollectionInfoOnPChannel { + if x != nil { + return x.Collections + } + return nil +} + +type CollectionInfoOnPChannel 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"` + Partitions []*PartitionInfoOnPChannel `protobuf:"bytes,2,rep,name=partitions,proto3" json:"partitions,omitempty"` + Vchannel string `protobuf:"bytes,3,opt,name=vchannel,proto3" json:"vchannel,omitempty"` +} + +func (x *CollectionInfoOnPChannel) Reset() { + *x = CollectionInfoOnPChannel{} + if protoimpl.UnsafeEnabled { + mi := &file_root_coord_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CollectionInfoOnPChannel) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CollectionInfoOnPChannel) ProtoMessage() {} + +func (x *CollectionInfoOnPChannel) ProtoReflect() protoreflect.Message { + mi := &file_root_coord_proto_msgTypes[15] + 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 CollectionInfoOnPChannel.ProtoReflect.Descriptor instead. +func (*CollectionInfoOnPChannel) Descriptor() ([]byte, []int) { + return file_root_coord_proto_rawDescGZIP(), []int{15} +} + +func (x *CollectionInfoOnPChannel) GetCollectionId() int64 { + if x != nil { + return x.CollectionId + } + return 0 +} + +func (x *CollectionInfoOnPChannel) GetPartitions() []*PartitionInfoOnPChannel { + if x != nil { + return x.Partitions + } + return nil +} + +func (x *CollectionInfoOnPChannel) GetVchannel() string { + if x != nil { + return x.Vchannel + } + return "" +} + +type PartitionInfoOnPChannel struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + PartitionId int64 `protobuf:"varint,1,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"` +} + +func (x *PartitionInfoOnPChannel) Reset() { + *x = PartitionInfoOnPChannel{} + if protoimpl.UnsafeEnabled { + mi := &file_root_coord_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PartitionInfoOnPChannel) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PartitionInfoOnPChannel) ProtoMessage() {} + +func (x *PartitionInfoOnPChannel) ProtoReflect() protoreflect.Message { + mi := &file_root_coord_proto_msgTypes[16] + 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 PartitionInfoOnPChannel.ProtoReflect.Descriptor instead. +func (*PartitionInfoOnPChannel) Descriptor() ([]byte, []int) { + return file_root_coord_proto_rawDescGZIP(), []int{16} +} + +func (x *PartitionInfoOnPChannel) GetPartitionId() int64 { + if x != nil { + return x.PartitionId + } + return 0 +} + +var File_root_coord_proto protoreflect.FileDescriptor + +var file_root_coord_proto_rawDesc = []byte{ + 0x0a, 0x10, 0x72, 0x6f, 0x6f, 0x74, 0x5f, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x12, 0x16, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x1a, 0x0c, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x0c, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x0e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x0b, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x1a, 0x0f, 0x65, 0x74, 0x63, 0x64, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x87, 0x01, 0x0a, 0x15, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x54, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 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, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, + 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x26, 0x0a, 0x0e, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x54, + 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0e, + 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x22, 0x81, + 0x01, 0x0a, 0x16, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x1c, + 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x04, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x14, 0x0a, 0x05, + 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x63, 0x6f, 0x75, + 0x6e, 0x74, 0x22, 0x58, 0x0a, 0x0e, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x49, 0x44, 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, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x6c, 0x0a, 0x0f, + 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x49, 0x44, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x02, 0x49, 0x44, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0d, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x8f, 0x01, 0x0a, 0x17, 0x44, + 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, + 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 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, 0x1e, 0x0a, 0x0a, + 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, + 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x22, 0x75, 0x0a, 0x0f, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x42, 0x61, 0x73, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, + 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x44, 0x22, 0xe3, 0x02, 0x0a, 0x0c, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, + 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x44, 0x0a, 0x09, 0x62, 0x61, 0x73, 0x65, 0x5f, 0x69, 0x6e, 0x66, + 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, + 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x42, 0x61, 0x73, 0x65, 0x49, 0x6e, 0x66, 0x6f, + 0x52, 0x08, 0x62, 0x61, 0x73, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x44, 0x0a, 0x0b, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x65, + 0x74, 0x63, 0x64, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, + 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x73, + 0x12, 0x65, 0x0a, 0x11, 0x65, 0x78, 0x74, 0x72, 0x61, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, + 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, + 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, + 0x73, 0x2e, 0x45, 0x78, 0x74, 0x72, 0x61, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, + 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0f, 0x65, 0x78, 0x74, 0x72, 0x61, 0x49, 0x6e, 0x64, + 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x1a, 0x60, 0x0a, 0x14, 0x45, 0x78, 0x74, 0x72, 0x61, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, + 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, + 0x79, 0x12, 0x32, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x65, 0x74, 0x63, 0x64, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xc3, 0x02, 0x0a, 0x18, 0x44, 0x65, + 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 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, + 0x67, 0x0a, 0x0d, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, + 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, + 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x73, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x1a, 0x65, 0x0a, 0x11, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, + 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, + 0x3a, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, + 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, + 0x6e, 0x66, 0x6f, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, + 0x64, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, + 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, 0x1a, 0x0a, 0x08, 0x75, 0x73, 0x65, + 0x72, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x75, 0x73, 0x65, + 0x72, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x84, 0x01, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x43, 0x72, 0x65, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, 0x65, + 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x22, 0x64, 0x0a, 0x17, + 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, + 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, 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, + 0x6d, 0x65, 0x22, 0xec, 0x01, 0x0a, 0x18, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x44, + 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x12, 0x0a, + 0x04, 0x64, 0x62, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, + 0x44, 0x12, 0x2b, 0x0a, 0x11, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x10, 0x63, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x41, + 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, + 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, + 0x73, 0x22, 0xda, 0x01, 0x0a, 0x14, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x44, 0x61, 0x74, 0x61, 0x62, + 0x61, 0x73, 0x65, 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, 0x17, 0x0a, 0x07, + 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, + 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x13, 0x0a, 0x05, 0x64, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x64, 0x62, 0x49, 0x64, 0x12, 0x41, 0x0a, 0x0a, 0x70, 0x72, + 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, + 0x72, 0x52, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x12, 0x1f, 0x0a, + 0x0b, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x05, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x0a, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x22, 0x66, + 0x0a, 0x16, 0x47, 0x65, 0x74, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, + 0x6f, 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, 0x1a, 0x0a, 0x08, 0x70, 0x63, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x63, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x22, 0xa2, 0x01, 0x0a, 0x17, 0x47, 0x65, 0x74, 0x50, 0x43, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, + 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x52, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, + 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x6e, 0x66, 0x6f, 0x4f, 0x6e, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x0b, + 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xac, 0x01, 0x0a, 0x18, + 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x4f, 0x6e, + 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 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, 0x4f, 0x0a, + 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x4f, 0x6e, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1a, + 0x0a, 0x08, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x22, 0x3c, 0x0a, 0x17, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x4f, 0x6e, 0x50, 0x43, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 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, 0x32, 0xec, 0x2b, 0x0a, 0x09, 0x52, 0x6f, 0x6f, + 0x74, 0x43, 0x6f, 0x6f, 0x72, 0x64, 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, 0x5f, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61, 0x74, 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, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 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, 0x5b, 0x0a, 0x0e, 0x44, 0x72, 0x6f, 0x70, 0x43, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2a, 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, 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, 0x5f, 0x0a, 0x0d, 0x48, 0x61, 0x73, 0x43, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x48, 0x61, 0x73, 0x43, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x42, 0x6f, 0x6f, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x77, 0x0a, 0x12, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, + 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2e, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 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, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7f, + 0x0a, 0x1a, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x2e, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 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, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x55, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x41, 0x6c, 0x69, 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, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x41, 0x6c, 0x69, 0x61, 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, 0x51, 0x0a, 0x09, 0x44, 0x72, 0x6f, 0x70, 0x41, 0x6c, + 0x69, 0x61, 0x73, 0x12, 0x25, 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, 0x41, 0x6c, + 0x69, 0x61, 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, 0x53, 0x0a, 0x0a, 0x41, 0x6c, 0x74, + 0x65, 0x72, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x41, 0x6c, + 0x74, 0x65, 0x72, 0x41, 0x6c, 0x69, 0x61, 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, 0x68, + 0x0a, 0x0d, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, + 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x41, 0x6c, + 0x69, 0x61, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x62, 0x0a, 0x0b, 0x4c, 0x69, 0x73, 0x74, + 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x12, 0x27, 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, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 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, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, + 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6e, 0x0a, 0x0f, + 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, + 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 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, 0x5d, 0x0a, 0x0f, + 0x41, 0x6c, 0x74, 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x67, 0x0a, 0x14, 0x41, + 0x6c, 0x74, 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x69, + 0x65, 0x6c, 0x64, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x43, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x22, 0x00, 0x12, 0x5d, 0x0a, 0x0f, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2b, 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, 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, 0x59, 0x0a, 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x29, 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, 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, 0x5d, + 0x0a, 0x0c, 0x48, 0x61, 0x73, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x28, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x48, 0x61, 0x73, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x42, + 0x6f, 0x6f, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6b, 0x0a, + 0x0e, 0x53, 0x68, 0x6f, 0x77, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, + 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x50, 0x61, 0x72, 0x74, 0x69, 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, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 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, 0x73, 0x0a, 0x16, 0x53, 0x68, + 0x6f, 0x77, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x49, 0x6e, 0x74, 0x65, + 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x50, + 0x61, 0x72, 0x74, 0x69, 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, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 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, + 0x65, 0x0a, 0x0c, 0x53, 0x68, 0x6f, 0x77, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, + 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x53, 0x68, 0x6f, 0x77, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x74, 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x50, 0x43, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, + 0x72, 0x64, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, + 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, + 0x72, 0x64, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, + 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x71, 0x0a, 0x0e, + 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x2d, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, + 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x54, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, + 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x54, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x5c, 0x0a, 0x07, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x49, 0x44, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, + 0x6f, 0x72, 0x64, 0x2e, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x49, 0x44, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x41, 0x6c, 0x6c, 0x6f, + 0x63, 0x49, 0x44, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x61, 0x0a, + 0x15, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x54, 0x69, + 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x43, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x4d, 0x73, + 0x67, 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, 0x1d, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x43, 0x61, 0x63, 0x68, + 0x65, 0x12, 0x32, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x2e, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, + 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x43, 0x61, 0x63, 0x68, 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, 0x58, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x72, 0x65, 0x64, + 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x43, + 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 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, 0x10, + 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, + 0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, + 0x69, 0x61, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 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, 0x5f, 0x0a, 0x10, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, + 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, + 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, 0x68, 0x0a, 0x0d, 0x4c, 0x69, 0x73, 0x74, 0x43, + 0x72, 0x65, 0x64, 0x55, 0x73, 0x65, 0x72, 0x73, 0x12, 0x29, 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, 0x43, 0x72, 0x65, 0x64, 0x55, 0x73, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x72, + 0x65, 0x64, 0x55, 0x73, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x12, 0x6e, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, + 0x61, 0x6c, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x47, 0x65, 0x74, 0x43, + 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x72, 0x65, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x12, 0x53, 0x0a, 0x0a, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x6f, 0x6c, 0x65, 0x12, + 0x26, 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, 0x6f, 0x6c, 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, 0x4f, 0x0a, 0x08, 0x44, 0x72, 0x6f, 0x70, 0x52, 0x6f, + 0x6c, 0x65, 0x12, 0x24, 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, 0x6f, 0x6c, + 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, 0x5d, 0x0a, 0x0f, 0x4f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, 0x52, 0x6f, 0x6c, 0x65, 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, 0x52, 0x6f, 0x6c, 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, 0x5f, 0x0a, 0x0a, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, + 0x52, 0x6f, 0x6c, 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, + 0x74, 0x52, 0x6f, 0x6c, 0x65, 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, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x52, 0x6f, 0x6c, 0x65, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x0a, 0x53, 0x65, 0x6c, 0x65, 0x63, + 0x74, 0x55, 0x73, 0x65, 0x72, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x65, 0x6c, 0x65, + 0x63, 0x74, 0x55, 0x73, 0x65, 0x72, 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, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x55, 0x73, 0x65, 0x72, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x10, 0x4f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x65, 0x50, 0x72, 0x69, 0x76, 0x69, 0x6c, 0x65, 0x67, 0x65, 0x12, 0x2c, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x65, 0x50, 0x72, 0x69, 0x76, 0x69, 0x6c, + 0x65, 0x67, 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, 0x62, 0x0a, 0x0b, 0x53, 0x65, 0x6c, + 0x65, 0x63, 0x74, 0x47, 0x72, 0x61, 0x6e, 0x74, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, + 0x65, 0x6c, 0x65, 0x63, 0x74, 0x47, 0x72, 0x61, 0x6e, 0x74, 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, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x47, 0x72, + 0x61, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x63, 0x0a, + 0x0a, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x28, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, + 0x6e, 0x61, 0x6c, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x4c, 0x69, + 0x73, 0x74, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x67, 0x0a, 0x0a, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x42, 0x41, 0x43, + 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x42, 0x41, + 0x43, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x42, 0x41, 0x43, 0x4d, 0x65, 0x74, + 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x59, 0x0a, 0x0b, 0x52, + 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x52, 0x42, 0x41, 0x43, 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x52, 0x42, 0x41, 0x43, 0x4d, 0x65, 0x74, 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, 0x67, 0x0a, 0x14, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x50, 0x72, 0x69, 0x76, 0x69, 0x6c, 0x65, 0x67, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x30, + 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, 0x50, 0x72, 0x69, 0x76, 0x69, + 0x6c, 0x65, 0x67, 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, + 0x63, 0x0a, 0x12, 0x44, 0x72, 0x6f, 0x70, 0x50, 0x72, 0x69, 0x76, 0x69, 0x6c, 0x65, 0x67, 0x65, + 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x2e, 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, + 0x50, 0x72, 0x69, 0x76, 0x69, 0x6c, 0x65, 0x67, 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, 0x7a, 0x0a, 0x13, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x72, 0x69, 0x76, + 0x69, 0x6c, 0x65, 0x67, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x12, 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, 0x50, 0x72, 0x69, 0x76, 0x69, 0x6c, 0x65, 0x67, 0x65, 0x47, + 0x72, 0x6f, 0x75, 0x70, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x30, 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, 0x50, 0x72, 0x69, 0x76, 0x69, 0x6c, 0x65, 0x67, 0x65, + 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x69, 0x0a, 0x15, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x65, 0x50, 0x72, 0x69, 0x76, 0x69, + 0x6c, 0x65, 0x67, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x65, 0x50, 0x72, 0x69, 0x76, 0x69, 0x6c, 0x65, 0x67, 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, 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, + 0x5f, 0x0a, 0x10, 0x52, 0x65, 0x6e, 0x61, 0x6d, 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, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x52, 0x65, 0x6e, 0x61, 0x6d, 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, 0x5b, 0x0a, 0x0e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, + 0x73, 0x65, 0x12, 0x2a, 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, 0x44, + 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 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, 0x57, 0x0a, + 0x0c, 0x44, 0x72, 0x6f, 0x70, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x12, 0x28, 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, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 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, 0x68, 0x0a, 0x0d, 0x4c, 0x69, 0x73, 0x74, 0x44, 0x61, + 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x73, 0x12, 0x29, 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, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x44, 0x61, 0x74, + 0x61, 0x62, 0x61, 0x73, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x77, 0x0a, 0x10, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x44, 0x61, 0x74, 0x61, + 0x62, 0x61, 0x73, 0x65, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x44, 0x65, + 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x44, + 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5c, 0x0a, 0x0d, 0x41, 0x6c, 0x74, + 0x65, 0x72, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, + 0x6f, 0x72, 0x64, 0x2e, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x42, 0x33, 0x5a, 0x31, 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, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2f, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_root_coord_proto_rawDescOnce sync.Once + file_root_coord_proto_rawDescData = file_root_coord_proto_rawDesc +) + +func file_root_coord_proto_rawDescGZIP() []byte { + file_root_coord_proto_rawDescOnce.Do(func() { + file_root_coord_proto_rawDescData = protoimpl.X.CompressGZIP(file_root_coord_proto_rawDescData) + }) + return file_root_coord_proto_rawDescData +} + +var file_root_coord_proto_msgTypes = make([]protoimpl.MessageInfo, 19) +var file_root_coord_proto_goTypes = []interface{}{ + (*AllocTimestampRequest)(nil), // 0: milvus.proto.rootcoord.AllocTimestampRequest + (*AllocTimestampResponse)(nil), // 1: milvus.proto.rootcoord.AllocTimestampResponse + (*AllocIDRequest)(nil), // 2: milvus.proto.rootcoord.AllocIDRequest + (*AllocIDResponse)(nil), // 3: milvus.proto.rootcoord.AllocIDResponse + (*DescribeSegmentsRequest)(nil), // 4: milvus.proto.rootcoord.DescribeSegmentsRequest + (*SegmentBaseInfo)(nil), // 5: milvus.proto.rootcoord.SegmentBaseInfo + (*SegmentInfos)(nil), // 6: milvus.proto.rootcoord.SegmentInfos + (*DescribeSegmentsResponse)(nil), // 7: milvus.proto.rootcoord.DescribeSegmentsResponse + (*GetCredentialRequest)(nil), // 8: milvus.proto.rootcoord.GetCredentialRequest + (*GetCredentialResponse)(nil), // 9: milvus.proto.rootcoord.GetCredentialResponse + (*DescribeDatabaseRequest)(nil), // 10: milvus.proto.rootcoord.DescribeDatabaseRequest + (*DescribeDatabaseResponse)(nil), // 11: milvus.proto.rootcoord.DescribeDatabaseResponse + (*AlterDatabaseRequest)(nil), // 12: milvus.proto.rootcoord.AlterDatabaseRequest + (*GetPChannelInfoRequest)(nil), // 13: milvus.proto.rootcoord.GetPChannelInfoRequest + (*GetPChannelInfoResponse)(nil), // 14: milvus.proto.rootcoord.GetPChannelInfoResponse + (*CollectionInfoOnPChannel)(nil), // 15: milvus.proto.rootcoord.CollectionInfoOnPChannel + (*PartitionInfoOnPChannel)(nil), // 16: milvus.proto.rootcoord.PartitionInfoOnPChannel + nil, // 17: milvus.proto.rootcoord.SegmentInfos.ExtraIndexInfosEntry + nil, // 18: milvus.proto.rootcoord.DescribeSegmentsResponse.SegmentInfosEntry + (*commonpb.MsgBase)(nil), // 19: milvus.proto.common.MsgBase + (*commonpb.Status)(nil), // 20: milvus.proto.common.Status + (*etcdpb.SegmentIndexInfo)(nil), // 21: milvus.proto.etcd.SegmentIndexInfo + (*commonpb.KeyValuePair)(nil), // 22: milvus.proto.common.KeyValuePair + (*etcdpb.IndexInfo)(nil), // 23: milvus.proto.etcd.IndexInfo + (*milvuspb.GetComponentStatesRequest)(nil), // 24: milvus.proto.milvus.GetComponentStatesRequest + (*internalpb.GetTimeTickChannelRequest)(nil), // 25: milvus.proto.internal.GetTimeTickChannelRequest + (*internalpb.GetStatisticsChannelRequest)(nil), // 26: milvus.proto.internal.GetStatisticsChannelRequest + (*milvuspb.CreateCollectionRequest)(nil), // 27: milvus.proto.milvus.CreateCollectionRequest + (*milvuspb.DropCollectionRequest)(nil), // 28: milvus.proto.milvus.DropCollectionRequest + (*milvuspb.HasCollectionRequest)(nil), // 29: milvus.proto.milvus.HasCollectionRequest + (*milvuspb.DescribeCollectionRequest)(nil), // 30: milvus.proto.milvus.DescribeCollectionRequest + (*milvuspb.CreateAliasRequest)(nil), // 31: milvus.proto.milvus.CreateAliasRequest + (*milvuspb.DropAliasRequest)(nil), // 32: milvus.proto.milvus.DropAliasRequest + (*milvuspb.AlterAliasRequest)(nil), // 33: milvus.proto.milvus.AlterAliasRequest + (*milvuspb.DescribeAliasRequest)(nil), // 34: milvus.proto.milvus.DescribeAliasRequest + (*milvuspb.ListAliasesRequest)(nil), // 35: milvus.proto.milvus.ListAliasesRequest + (*milvuspb.ShowCollectionsRequest)(nil), // 36: milvus.proto.milvus.ShowCollectionsRequest + (*milvuspb.AlterCollectionRequest)(nil), // 37: milvus.proto.milvus.AlterCollectionRequest + (*milvuspb.AlterCollectionFieldRequest)(nil), // 38: milvus.proto.milvus.AlterCollectionFieldRequest + (*milvuspb.CreatePartitionRequest)(nil), // 39: milvus.proto.milvus.CreatePartitionRequest + (*milvuspb.DropPartitionRequest)(nil), // 40: milvus.proto.milvus.DropPartitionRequest + (*milvuspb.HasPartitionRequest)(nil), // 41: milvus.proto.milvus.HasPartitionRequest + (*milvuspb.ShowPartitionsRequest)(nil), // 42: milvus.proto.milvus.ShowPartitionsRequest + (*milvuspb.ShowSegmentsRequest)(nil), // 43: milvus.proto.milvus.ShowSegmentsRequest + (*internalpb.ChannelTimeTickMsg)(nil), // 44: milvus.proto.internal.ChannelTimeTickMsg + (*proxypb.InvalidateCollMetaCacheRequest)(nil), // 45: milvus.proto.proxy.InvalidateCollMetaCacheRequest + (*internalpb.ShowConfigurationsRequest)(nil), // 46: milvus.proto.internal.ShowConfigurationsRequest + (*milvuspb.GetMetricsRequest)(nil), // 47: milvus.proto.milvus.GetMetricsRequest + (*internalpb.CredentialInfo)(nil), // 48: milvus.proto.internal.CredentialInfo + (*milvuspb.DeleteCredentialRequest)(nil), // 49: milvus.proto.milvus.DeleteCredentialRequest + (*milvuspb.ListCredUsersRequest)(nil), // 50: milvus.proto.milvus.ListCredUsersRequest + (*milvuspb.CreateRoleRequest)(nil), // 51: milvus.proto.milvus.CreateRoleRequest + (*milvuspb.DropRoleRequest)(nil), // 52: milvus.proto.milvus.DropRoleRequest + (*milvuspb.OperateUserRoleRequest)(nil), // 53: milvus.proto.milvus.OperateUserRoleRequest + (*milvuspb.SelectRoleRequest)(nil), // 54: milvus.proto.milvus.SelectRoleRequest + (*milvuspb.SelectUserRequest)(nil), // 55: milvus.proto.milvus.SelectUserRequest + (*milvuspb.OperatePrivilegeRequest)(nil), // 56: milvus.proto.milvus.OperatePrivilegeRequest + (*milvuspb.SelectGrantRequest)(nil), // 57: milvus.proto.milvus.SelectGrantRequest + (*internalpb.ListPolicyRequest)(nil), // 58: milvus.proto.internal.ListPolicyRequest + (*milvuspb.BackupRBACMetaRequest)(nil), // 59: milvus.proto.milvus.BackupRBACMetaRequest + (*milvuspb.RestoreRBACMetaRequest)(nil), // 60: milvus.proto.milvus.RestoreRBACMetaRequest + (*milvuspb.CreatePrivilegeGroupRequest)(nil), // 61: milvus.proto.milvus.CreatePrivilegeGroupRequest + (*milvuspb.DropPrivilegeGroupRequest)(nil), // 62: milvus.proto.milvus.DropPrivilegeGroupRequest + (*milvuspb.ListPrivilegeGroupsRequest)(nil), // 63: milvus.proto.milvus.ListPrivilegeGroupsRequest + (*milvuspb.OperatePrivilegeGroupRequest)(nil), // 64: milvus.proto.milvus.OperatePrivilegeGroupRequest + (*milvuspb.CheckHealthRequest)(nil), // 65: milvus.proto.milvus.CheckHealthRequest + (*milvuspb.RenameCollectionRequest)(nil), // 66: milvus.proto.milvus.RenameCollectionRequest + (*milvuspb.CreateDatabaseRequest)(nil), // 67: milvus.proto.milvus.CreateDatabaseRequest + (*milvuspb.DropDatabaseRequest)(nil), // 68: milvus.proto.milvus.DropDatabaseRequest + (*milvuspb.ListDatabasesRequest)(nil), // 69: milvus.proto.milvus.ListDatabasesRequest + (*milvuspb.ComponentStates)(nil), // 70: milvus.proto.milvus.ComponentStates + (*milvuspb.StringResponse)(nil), // 71: milvus.proto.milvus.StringResponse + (*milvuspb.BoolResponse)(nil), // 72: milvus.proto.milvus.BoolResponse + (*milvuspb.DescribeCollectionResponse)(nil), // 73: milvus.proto.milvus.DescribeCollectionResponse + (*milvuspb.DescribeAliasResponse)(nil), // 74: milvus.proto.milvus.DescribeAliasResponse + (*milvuspb.ListAliasesResponse)(nil), // 75: milvus.proto.milvus.ListAliasesResponse + (*milvuspb.ShowCollectionsResponse)(nil), // 76: milvus.proto.milvus.ShowCollectionsResponse + (*milvuspb.ShowPartitionsResponse)(nil), // 77: milvus.proto.milvus.ShowPartitionsResponse + (*milvuspb.ShowSegmentsResponse)(nil), // 78: milvus.proto.milvus.ShowSegmentsResponse + (*internalpb.ShowConfigurationsResponse)(nil), // 79: milvus.proto.internal.ShowConfigurationsResponse + (*milvuspb.GetMetricsResponse)(nil), // 80: milvus.proto.milvus.GetMetricsResponse + (*milvuspb.ListCredUsersResponse)(nil), // 81: milvus.proto.milvus.ListCredUsersResponse + (*milvuspb.SelectRoleResponse)(nil), // 82: milvus.proto.milvus.SelectRoleResponse + (*milvuspb.SelectUserResponse)(nil), // 83: milvus.proto.milvus.SelectUserResponse + (*milvuspb.SelectGrantResponse)(nil), // 84: milvus.proto.milvus.SelectGrantResponse + (*internalpb.ListPolicyResponse)(nil), // 85: milvus.proto.internal.ListPolicyResponse + (*milvuspb.BackupRBACMetaResponse)(nil), // 86: milvus.proto.milvus.BackupRBACMetaResponse + (*milvuspb.ListPrivilegeGroupsResponse)(nil), // 87: milvus.proto.milvus.ListPrivilegeGroupsResponse + (*milvuspb.CheckHealthResponse)(nil), // 88: milvus.proto.milvus.CheckHealthResponse + (*milvuspb.ListDatabasesResponse)(nil), // 89: milvus.proto.milvus.ListDatabasesResponse +} +var file_root_coord_proto_depIdxs = []int32{ + 19, // 0: milvus.proto.rootcoord.AllocTimestampRequest.base:type_name -> milvus.proto.common.MsgBase + 20, // 1: milvus.proto.rootcoord.AllocTimestampResponse.status:type_name -> milvus.proto.common.Status + 19, // 2: milvus.proto.rootcoord.AllocIDRequest.base:type_name -> milvus.proto.common.MsgBase + 20, // 3: milvus.proto.rootcoord.AllocIDResponse.status:type_name -> milvus.proto.common.Status + 19, // 4: milvus.proto.rootcoord.DescribeSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase + 5, // 5: milvus.proto.rootcoord.SegmentInfos.base_info:type_name -> milvus.proto.rootcoord.SegmentBaseInfo + 21, // 6: milvus.proto.rootcoord.SegmentInfos.index_infos:type_name -> milvus.proto.etcd.SegmentIndexInfo + 17, // 7: milvus.proto.rootcoord.SegmentInfos.extra_index_infos:type_name -> milvus.proto.rootcoord.SegmentInfos.ExtraIndexInfosEntry + 20, // 8: milvus.proto.rootcoord.DescribeSegmentsResponse.status:type_name -> milvus.proto.common.Status + 18, // 9: milvus.proto.rootcoord.DescribeSegmentsResponse.segment_infos:type_name -> milvus.proto.rootcoord.DescribeSegmentsResponse.SegmentInfosEntry + 19, // 10: milvus.proto.rootcoord.GetCredentialRequest.base:type_name -> milvus.proto.common.MsgBase + 20, // 11: milvus.proto.rootcoord.GetCredentialResponse.status:type_name -> milvus.proto.common.Status + 19, // 12: milvus.proto.rootcoord.DescribeDatabaseRequest.base:type_name -> milvus.proto.common.MsgBase + 20, // 13: milvus.proto.rootcoord.DescribeDatabaseResponse.status:type_name -> milvus.proto.common.Status + 22, // 14: milvus.proto.rootcoord.DescribeDatabaseResponse.properties:type_name -> milvus.proto.common.KeyValuePair + 19, // 15: milvus.proto.rootcoord.AlterDatabaseRequest.base:type_name -> milvus.proto.common.MsgBase + 22, // 16: milvus.proto.rootcoord.AlterDatabaseRequest.properties:type_name -> milvus.proto.common.KeyValuePair + 19, // 17: milvus.proto.rootcoord.GetPChannelInfoRequest.base:type_name -> milvus.proto.common.MsgBase + 20, // 18: milvus.proto.rootcoord.GetPChannelInfoResponse.status:type_name -> milvus.proto.common.Status + 15, // 19: milvus.proto.rootcoord.GetPChannelInfoResponse.collections:type_name -> milvus.proto.rootcoord.CollectionInfoOnPChannel + 16, // 20: milvus.proto.rootcoord.CollectionInfoOnPChannel.partitions:type_name -> milvus.proto.rootcoord.PartitionInfoOnPChannel + 23, // 21: milvus.proto.rootcoord.SegmentInfos.ExtraIndexInfosEntry.value:type_name -> milvus.proto.etcd.IndexInfo + 6, // 22: milvus.proto.rootcoord.DescribeSegmentsResponse.SegmentInfosEntry.value:type_name -> milvus.proto.rootcoord.SegmentInfos + 24, // 23: milvus.proto.rootcoord.RootCoord.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest + 25, // 24: milvus.proto.rootcoord.RootCoord.GetTimeTickChannel:input_type -> milvus.proto.internal.GetTimeTickChannelRequest + 26, // 25: milvus.proto.rootcoord.RootCoord.GetStatisticsChannel:input_type -> milvus.proto.internal.GetStatisticsChannelRequest + 27, // 26: milvus.proto.rootcoord.RootCoord.CreateCollection:input_type -> milvus.proto.milvus.CreateCollectionRequest + 28, // 27: milvus.proto.rootcoord.RootCoord.DropCollection:input_type -> milvus.proto.milvus.DropCollectionRequest + 29, // 28: milvus.proto.rootcoord.RootCoord.HasCollection:input_type -> milvus.proto.milvus.HasCollectionRequest + 30, // 29: milvus.proto.rootcoord.RootCoord.DescribeCollection:input_type -> milvus.proto.milvus.DescribeCollectionRequest + 30, // 30: milvus.proto.rootcoord.RootCoord.DescribeCollectionInternal:input_type -> milvus.proto.milvus.DescribeCollectionRequest + 31, // 31: milvus.proto.rootcoord.RootCoord.CreateAlias:input_type -> milvus.proto.milvus.CreateAliasRequest + 32, // 32: milvus.proto.rootcoord.RootCoord.DropAlias:input_type -> milvus.proto.milvus.DropAliasRequest + 33, // 33: milvus.proto.rootcoord.RootCoord.AlterAlias:input_type -> milvus.proto.milvus.AlterAliasRequest + 34, // 34: milvus.proto.rootcoord.RootCoord.DescribeAlias:input_type -> milvus.proto.milvus.DescribeAliasRequest + 35, // 35: milvus.proto.rootcoord.RootCoord.ListAliases:input_type -> milvus.proto.milvus.ListAliasesRequest + 36, // 36: milvus.proto.rootcoord.RootCoord.ShowCollections:input_type -> milvus.proto.milvus.ShowCollectionsRequest + 37, // 37: milvus.proto.rootcoord.RootCoord.AlterCollection:input_type -> milvus.proto.milvus.AlterCollectionRequest + 38, // 38: milvus.proto.rootcoord.RootCoord.AlterCollectionField:input_type -> milvus.proto.milvus.AlterCollectionFieldRequest + 39, // 39: milvus.proto.rootcoord.RootCoord.CreatePartition:input_type -> milvus.proto.milvus.CreatePartitionRequest + 40, // 40: milvus.proto.rootcoord.RootCoord.DropPartition:input_type -> milvus.proto.milvus.DropPartitionRequest + 41, // 41: milvus.proto.rootcoord.RootCoord.HasPartition:input_type -> milvus.proto.milvus.HasPartitionRequest + 42, // 42: milvus.proto.rootcoord.RootCoord.ShowPartitions:input_type -> milvus.proto.milvus.ShowPartitionsRequest + 42, // 43: milvus.proto.rootcoord.RootCoord.ShowPartitionsInternal:input_type -> milvus.proto.milvus.ShowPartitionsRequest + 43, // 44: milvus.proto.rootcoord.RootCoord.ShowSegments:input_type -> milvus.proto.milvus.ShowSegmentsRequest + 13, // 45: milvus.proto.rootcoord.RootCoord.GetPChannelInfo:input_type -> milvus.proto.rootcoord.GetPChannelInfoRequest + 0, // 46: milvus.proto.rootcoord.RootCoord.AllocTimestamp:input_type -> milvus.proto.rootcoord.AllocTimestampRequest + 2, // 47: milvus.proto.rootcoord.RootCoord.AllocID:input_type -> milvus.proto.rootcoord.AllocIDRequest + 44, // 48: milvus.proto.rootcoord.RootCoord.UpdateChannelTimeTick:input_type -> milvus.proto.internal.ChannelTimeTickMsg + 45, // 49: milvus.proto.rootcoord.RootCoord.InvalidateCollectionMetaCache:input_type -> milvus.proto.proxy.InvalidateCollMetaCacheRequest + 46, // 50: milvus.proto.rootcoord.RootCoord.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest + 47, // 51: milvus.proto.rootcoord.RootCoord.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest + 48, // 52: milvus.proto.rootcoord.RootCoord.CreateCredential:input_type -> milvus.proto.internal.CredentialInfo + 48, // 53: milvus.proto.rootcoord.RootCoord.UpdateCredential:input_type -> milvus.proto.internal.CredentialInfo + 49, // 54: milvus.proto.rootcoord.RootCoord.DeleteCredential:input_type -> milvus.proto.milvus.DeleteCredentialRequest + 50, // 55: milvus.proto.rootcoord.RootCoord.ListCredUsers:input_type -> milvus.proto.milvus.ListCredUsersRequest + 8, // 56: milvus.proto.rootcoord.RootCoord.GetCredential:input_type -> milvus.proto.rootcoord.GetCredentialRequest + 51, // 57: milvus.proto.rootcoord.RootCoord.CreateRole:input_type -> milvus.proto.milvus.CreateRoleRequest + 52, // 58: milvus.proto.rootcoord.RootCoord.DropRole:input_type -> milvus.proto.milvus.DropRoleRequest + 53, // 59: milvus.proto.rootcoord.RootCoord.OperateUserRole:input_type -> milvus.proto.milvus.OperateUserRoleRequest + 54, // 60: milvus.proto.rootcoord.RootCoord.SelectRole:input_type -> milvus.proto.milvus.SelectRoleRequest + 55, // 61: milvus.proto.rootcoord.RootCoord.SelectUser:input_type -> milvus.proto.milvus.SelectUserRequest + 56, // 62: milvus.proto.rootcoord.RootCoord.OperatePrivilege:input_type -> milvus.proto.milvus.OperatePrivilegeRequest + 57, // 63: milvus.proto.rootcoord.RootCoord.SelectGrant:input_type -> milvus.proto.milvus.SelectGrantRequest + 58, // 64: milvus.proto.rootcoord.RootCoord.ListPolicy:input_type -> milvus.proto.internal.ListPolicyRequest + 59, // 65: milvus.proto.rootcoord.RootCoord.BackupRBAC:input_type -> milvus.proto.milvus.BackupRBACMetaRequest + 60, // 66: milvus.proto.rootcoord.RootCoord.RestoreRBAC:input_type -> milvus.proto.milvus.RestoreRBACMetaRequest + 61, // 67: milvus.proto.rootcoord.RootCoord.CreatePrivilegeGroup:input_type -> milvus.proto.milvus.CreatePrivilegeGroupRequest + 62, // 68: milvus.proto.rootcoord.RootCoord.DropPrivilegeGroup:input_type -> milvus.proto.milvus.DropPrivilegeGroupRequest + 63, // 69: milvus.proto.rootcoord.RootCoord.ListPrivilegeGroups:input_type -> milvus.proto.milvus.ListPrivilegeGroupsRequest + 64, // 70: milvus.proto.rootcoord.RootCoord.OperatePrivilegeGroup:input_type -> milvus.proto.milvus.OperatePrivilegeGroupRequest + 65, // 71: milvus.proto.rootcoord.RootCoord.CheckHealth:input_type -> milvus.proto.milvus.CheckHealthRequest + 66, // 72: milvus.proto.rootcoord.RootCoord.RenameCollection:input_type -> milvus.proto.milvus.RenameCollectionRequest + 67, // 73: milvus.proto.rootcoord.RootCoord.CreateDatabase:input_type -> milvus.proto.milvus.CreateDatabaseRequest + 68, // 74: milvus.proto.rootcoord.RootCoord.DropDatabase:input_type -> milvus.proto.milvus.DropDatabaseRequest + 69, // 75: milvus.proto.rootcoord.RootCoord.ListDatabases:input_type -> milvus.proto.milvus.ListDatabasesRequest + 10, // 76: milvus.proto.rootcoord.RootCoord.DescribeDatabase:input_type -> milvus.proto.rootcoord.DescribeDatabaseRequest + 12, // 77: milvus.proto.rootcoord.RootCoord.AlterDatabase:input_type -> milvus.proto.rootcoord.AlterDatabaseRequest + 70, // 78: milvus.proto.rootcoord.RootCoord.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates + 71, // 79: milvus.proto.rootcoord.RootCoord.GetTimeTickChannel:output_type -> milvus.proto.milvus.StringResponse + 71, // 80: milvus.proto.rootcoord.RootCoord.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse + 20, // 81: milvus.proto.rootcoord.RootCoord.CreateCollection:output_type -> milvus.proto.common.Status + 20, // 82: milvus.proto.rootcoord.RootCoord.DropCollection:output_type -> milvus.proto.common.Status + 72, // 83: milvus.proto.rootcoord.RootCoord.HasCollection:output_type -> milvus.proto.milvus.BoolResponse + 73, // 84: milvus.proto.rootcoord.RootCoord.DescribeCollection:output_type -> milvus.proto.milvus.DescribeCollectionResponse + 73, // 85: milvus.proto.rootcoord.RootCoord.DescribeCollectionInternal:output_type -> milvus.proto.milvus.DescribeCollectionResponse + 20, // 86: milvus.proto.rootcoord.RootCoord.CreateAlias:output_type -> milvus.proto.common.Status + 20, // 87: milvus.proto.rootcoord.RootCoord.DropAlias:output_type -> milvus.proto.common.Status + 20, // 88: milvus.proto.rootcoord.RootCoord.AlterAlias:output_type -> milvus.proto.common.Status + 74, // 89: milvus.proto.rootcoord.RootCoord.DescribeAlias:output_type -> milvus.proto.milvus.DescribeAliasResponse + 75, // 90: milvus.proto.rootcoord.RootCoord.ListAliases:output_type -> milvus.proto.milvus.ListAliasesResponse + 76, // 91: milvus.proto.rootcoord.RootCoord.ShowCollections:output_type -> milvus.proto.milvus.ShowCollectionsResponse + 20, // 92: milvus.proto.rootcoord.RootCoord.AlterCollection:output_type -> milvus.proto.common.Status + 20, // 93: milvus.proto.rootcoord.RootCoord.AlterCollectionField:output_type -> milvus.proto.common.Status + 20, // 94: milvus.proto.rootcoord.RootCoord.CreatePartition:output_type -> milvus.proto.common.Status + 20, // 95: milvus.proto.rootcoord.RootCoord.DropPartition:output_type -> milvus.proto.common.Status + 72, // 96: milvus.proto.rootcoord.RootCoord.HasPartition:output_type -> milvus.proto.milvus.BoolResponse + 77, // 97: milvus.proto.rootcoord.RootCoord.ShowPartitions:output_type -> milvus.proto.milvus.ShowPartitionsResponse + 77, // 98: milvus.proto.rootcoord.RootCoord.ShowPartitionsInternal:output_type -> milvus.proto.milvus.ShowPartitionsResponse + 78, // 99: milvus.proto.rootcoord.RootCoord.ShowSegments:output_type -> milvus.proto.milvus.ShowSegmentsResponse + 14, // 100: milvus.proto.rootcoord.RootCoord.GetPChannelInfo:output_type -> milvus.proto.rootcoord.GetPChannelInfoResponse + 1, // 101: milvus.proto.rootcoord.RootCoord.AllocTimestamp:output_type -> milvus.proto.rootcoord.AllocTimestampResponse + 3, // 102: milvus.proto.rootcoord.RootCoord.AllocID:output_type -> milvus.proto.rootcoord.AllocIDResponse + 20, // 103: milvus.proto.rootcoord.RootCoord.UpdateChannelTimeTick:output_type -> milvus.proto.common.Status + 20, // 104: milvus.proto.rootcoord.RootCoord.InvalidateCollectionMetaCache:output_type -> milvus.proto.common.Status + 79, // 105: milvus.proto.rootcoord.RootCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse + 80, // 106: milvus.proto.rootcoord.RootCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse + 20, // 107: milvus.proto.rootcoord.RootCoord.CreateCredential:output_type -> milvus.proto.common.Status + 20, // 108: milvus.proto.rootcoord.RootCoord.UpdateCredential:output_type -> milvus.proto.common.Status + 20, // 109: milvus.proto.rootcoord.RootCoord.DeleteCredential:output_type -> milvus.proto.common.Status + 81, // 110: milvus.proto.rootcoord.RootCoord.ListCredUsers:output_type -> milvus.proto.milvus.ListCredUsersResponse + 9, // 111: milvus.proto.rootcoord.RootCoord.GetCredential:output_type -> milvus.proto.rootcoord.GetCredentialResponse + 20, // 112: milvus.proto.rootcoord.RootCoord.CreateRole:output_type -> milvus.proto.common.Status + 20, // 113: milvus.proto.rootcoord.RootCoord.DropRole:output_type -> milvus.proto.common.Status + 20, // 114: milvus.proto.rootcoord.RootCoord.OperateUserRole:output_type -> milvus.proto.common.Status + 82, // 115: milvus.proto.rootcoord.RootCoord.SelectRole:output_type -> milvus.proto.milvus.SelectRoleResponse + 83, // 116: milvus.proto.rootcoord.RootCoord.SelectUser:output_type -> milvus.proto.milvus.SelectUserResponse + 20, // 117: milvus.proto.rootcoord.RootCoord.OperatePrivilege:output_type -> milvus.proto.common.Status + 84, // 118: milvus.proto.rootcoord.RootCoord.SelectGrant:output_type -> milvus.proto.milvus.SelectGrantResponse + 85, // 119: milvus.proto.rootcoord.RootCoord.ListPolicy:output_type -> milvus.proto.internal.ListPolicyResponse + 86, // 120: milvus.proto.rootcoord.RootCoord.BackupRBAC:output_type -> milvus.proto.milvus.BackupRBACMetaResponse + 20, // 121: milvus.proto.rootcoord.RootCoord.RestoreRBAC:output_type -> milvus.proto.common.Status + 20, // 122: milvus.proto.rootcoord.RootCoord.CreatePrivilegeGroup:output_type -> milvus.proto.common.Status + 20, // 123: milvus.proto.rootcoord.RootCoord.DropPrivilegeGroup:output_type -> milvus.proto.common.Status + 87, // 124: milvus.proto.rootcoord.RootCoord.ListPrivilegeGroups:output_type -> milvus.proto.milvus.ListPrivilegeGroupsResponse + 20, // 125: milvus.proto.rootcoord.RootCoord.OperatePrivilegeGroup:output_type -> milvus.proto.common.Status + 88, // 126: milvus.proto.rootcoord.RootCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse + 20, // 127: milvus.proto.rootcoord.RootCoord.RenameCollection:output_type -> milvus.proto.common.Status + 20, // 128: milvus.proto.rootcoord.RootCoord.CreateDatabase:output_type -> milvus.proto.common.Status + 20, // 129: milvus.proto.rootcoord.RootCoord.DropDatabase:output_type -> milvus.proto.common.Status + 89, // 130: milvus.proto.rootcoord.RootCoord.ListDatabases:output_type -> milvus.proto.milvus.ListDatabasesResponse + 11, // 131: milvus.proto.rootcoord.RootCoord.DescribeDatabase:output_type -> milvus.proto.rootcoord.DescribeDatabaseResponse + 20, // 132: milvus.proto.rootcoord.RootCoord.AlterDatabase:output_type -> milvus.proto.common.Status + 78, // [78:133] is the sub-list for method output_type + 23, // [23:78] is the sub-list for method input_type + 23, // [23:23] is the sub-list for extension type_name + 23, // [23:23] is the sub-list for extension extendee + 0, // [0:23] is the sub-list for field type_name +} + +func init() { file_root_coord_proto_init() } +func file_root_coord_proto_init() { + if File_root_coord_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_root_coord_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AllocTimestampRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_root_coord_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AllocTimestampResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_root_coord_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AllocIDRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_root_coord_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AllocIDResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_root_coord_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DescribeSegmentsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_root_coord_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentBaseInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_root_coord_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentInfos); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_root_coord_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DescribeSegmentsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_root_coord_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetCredentialRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_root_coord_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetCredentialResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_root_coord_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DescribeDatabaseRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_root_coord_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DescribeDatabaseResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_root_coord_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AlterDatabaseRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_root_coord_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetPChannelInfoRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_root_coord_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetPChannelInfoResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_root_coord_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CollectionInfoOnPChannel); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_root_coord_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PartitionInfoOnPChannel); 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{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_root_coord_proto_rawDesc, + NumEnums: 0, + NumMessages: 19, + NumExtensions: 0, + NumServices: 1, + }, + GoTypes: file_root_coord_proto_goTypes, + DependencyIndexes: file_root_coord_proto_depIdxs, + MessageInfos: file_root_coord_proto_msgTypes, + }.Build() + File_root_coord_proto = out.File + file_root_coord_proto_rawDesc = nil + file_root_coord_proto_goTypes = nil + file_root_coord_proto_depIdxs = nil +} diff --git a/pkg/proto/rootcoordpb/root_coord_grpc.pb.go b/pkg/proto/rootcoordpb/root_coord_grpc.pb.go new file mode 100644 index 0000000000000..57260ee8ffb7e --- /dev/null +++ b/pkg/proto/rootcoordpb/root_coord_grpc.pb.go @@ -0,0 +1,2209 @@ +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. +// versions: +// - protoc-gen-go-grpc v1.3.0 +// - protoc v3.21.4 +// source: root_coord.proto + +package rootcoordpb + +import ( + context "context" + commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" + proxypb "github.com/milvus-io/milvus/pkg/proto/proxypb" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" +) + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +// Requires gRPC-Go v1.32.0 or later. +const _ = grpc.SupportPackageIsVersion7 + +const ( + RootCoord_GetComponentStates_FullMethodName = "/milvus.proto.rootcoord.RootCoord/GetComponentStates" + RootCoord_GetTimeTickChannel_FullMethodName = "/milvus.proto.rootcoord.RootCoord/GetTimeTickChannel" + RootCoord_GetStatisticsChannel_FullMethodName = "/milvus.proto.rootcoord.RootCoord/GetStatisticsChannel" + RootCoord_CreateCollection_FullMethodName = "/milvus.proto.rootcoord.RootCoord/CreateCollection" + RootCoord_DropCollection_FullMethodName = "/milvus.proto.rootcoord.RootCoord/DropCollection" + RootCoord_HasCollection_FullMethodName = "/milvus.proto.rootcoord.RootCoord/HasCollection" + RootCoord_DescribeCollection_FullMethodName = "/milvus.proto.rootcoord.RootCoord/DescribeCollection" + RootCoord_DescribeCollectionInternal_FullMethodName = "/milvus.proto.rootcoord.RootCoord/DescribeCollectionInternal" + RootCoord_CreateAlias_FullMethodName = "/milvus.proto.rootcoord.RootCoord/CreateAlias" + RootCoord_DropAlias_FullMethodName = "/milvus.proto.rootcoord.RootCoord/DropAlias" + RootCoord_AlterAlias_FullMethodName = "/milvus.proto.rootcoord.RootCoord/AlterAlias" + RootCoord_DescribeAlias_FullMethodName = "/milvus.proto.rootcoord.RootCoord/DescribeAlias" + RootCoord_ListAliases_FullMethodName = "/milvus.proto.rootcoord.RootCoord/ListAliases" + RootCoord_ShowCollections_FullMethodName = "/milvus.proto.rootcoord.RootCoord/ShowCollections" + RootCoord_AlterCollection_FullMethodName = "/milvus.proto.rootcoord.RootCoord/AlterCollection" + RootCoord_AlterCollectionField_FullMethodName = "/milvus.proto.rootcoord.RootCoord/AlterCollectionField" + RootCoord_CreatePartition_FullMethodName = "/milvus.proto.rootcoord.RootCoord/CreatePartition" + RootCoord_DropPartition_FullMethodName = "/milvus.proto.rootcoord.RootCoord/DropPartition" + RootCoord_HasPartition_FullMethodName = "/milvus.proto.rootcoord.RootCoord/HasPartition" + RootCoord_ShowPartitions_FullMethodName = "/milvus.proto.rootcoord.RootCoord/ShowPartitions" + RootCoord_ShowPartitionsInternal_FullMethodName = "/milvus.proto.rootcoord.RootCoord/ShowPartitionsInternal" + RootCoord_ShowSegments_FullMethodName = "/milvus.proto.rootcoord.RootCoord/ShowSegments" + RootCoord_GetPChannelInfo_FullMethodName = "/milvus.proto.rootcoord.RootCoord/GetPChannelInfo" + RootCoord_AllocTimestamp_FullMethodName = "/milvus.proto.rootcoord.RootCoord/AllocTimestamp" + RootCoord_AllocID_FullMethodName = "/milvus.proto.rootcoord.RootCoord/AllocID" + RootCoord_UpdateChannelTimeTick_FullMethodName = "/milvus.proto.rootcoord.RootCoord/UpdateChannelTimeTick" + RootCoord_InvalidateCollectionMetaCache_FullMethodName = "/milvus.proto.rootcoord.RootCoord/InvalidateCollectionMetaCache" + RootCoord_ShowConfigurations_FullMethodName = "/milvus.proto.rootcoord.RootCoord/ShowConfigurations" + RootCoord_GetMetrics_FullMethodName = "/milvus.proto.rootcoord.RootCoord/GetMetrics" + RootCoord_CreateCredential_FullMethodName = "/milvus.proto.rootcoord.RootCoord/CreateCredential" + RootCoord_UpdateCredential_FullMethodName = "/milvus.proto.rootcoord.RootCoord/UpdateCredential" + RootCoord_DeleteCredential_FullMethodName = "/milvus.proto.rootcoord.RootCoord/DeleteCredential" + RootCoord_ListCredUsers_FullMethodName = "/milvus.proto.rootcoord.RootCoord/ListCredUsers" + RootCoord_GetCredential_FullMethodName = "/milvus.proto.rootcoord.RootCoord/GetCredential" + RootCoord_CreateRole_FullMethodName = "/milvus.proto.rootcoord.RootCoord/CreateRole" + RootCoord_DropRole_FullMethodName = "/milvus.proto.rootcoord.RootCoord/DropRole" + RootCoord_OperateUserRole_FullMethodName = "/milvus.proto.rootcoord.RootCoord/OperateUserRole" + RootCoord_SelectRole_FullMethodName = "/milvus.proto.rootcoord.RootCoord/SelectRole" + RootCoord_SelectUser_FullMethodName = "/milvus.proto.rootcoord.RootCoord/SelectUser" + RootCoord_OperatePrivilege_FullMethodName = "/milvus.proto.rootcoord.RootCoord/OperatePrivilege" + RootCoord_SelectGrant_FullMethodName = "/milvus.proto.rootcoord.RootCoord/SelectGrant" + RootCoord_ListPolicy_FullMethodName = "/milvus.proto.rootcoord.RootCoord/ListPolicy" + RootCoord_BackupRBAC_FullMethodName = "/milvus.proto.rootcoord.RootCoord/BackupRBAC" + RootCoord_RestoreRBAC_FullMethodName = "/milvus.proto.rootcoord.RootCoord/RestoreRBAC" + RootCoord_CreatePrivilegeGroup_FullMethodName = "/milvus.proto.rootcoord.RootCoord/CreatePrivilegeGroup" + RootCoord_DropPrivilegeGroup_FullMethodName = "/milvus.proto.rootcoord.RootCoord/DropPrivilegeGroup" + RootCoord_ListPrivilegeGroups_FullMethodName = "/milvus.proto.rootcoord.RootCoord/ListPrivilegeGroups" + RootCoord_OperatePrivilegeGroup_FullMethodName = "/milvus.proto.rootcoord.RootCoord/OperatePrivilegeGroup" + RootCoord_CheckHealth_FullMethodName = "/milvus.proto.rootcoord.RootCoord/CheckHealth" + RootCoord_RenameCollection_FullMethodName = "/milvus.proto.rootcoord.RootCoord/RenameCollection" + RootCoord_CreateDatabase_FullMethodName = "/milvus.proto.rootcoord.RootCoord/CreateDatabase" + RootCoord_DropDatabase_FullMethodName = "/milvus.proto.rootcoord.RootCoord/DropDatabase" + RootCoord_ListDatabases_FullMethodName = "/milvus.proto.rootcoord.RootCoord/ListDatabases" + RootCoord_DescribeDatabase_FullMethodName = "/milvus.proto.rootcoord.RootCoord/DescribeDatabase" + RootCoord_AlterDatabase_FullMethodName = "/milvus.proto.rootcoord.RootCoord/AlterDatabase" +) + +// RootCoordClient is the client API for RootCoord service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +type RootCoordClient interface { + GetComponentStates(ctx context.Context, in *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) + GetTimeTickChannel(ctx context.Context, in *internalpb.GetTimeTickChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) + GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) + // * + // @brief This method is used to create collection + // + // @param CreateCollectionRequest, use to provide collection information to be created. + // + // @return Status + CreateCollection(ctx context.Context, in *milvuspb.CreateCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + // * + // @brief This method is used to delete collection. + // + // @param DropCollectionRequest, collection name is going to be deleted. + // + // @return Status + DropCollection(ctx context.Context, in *milvuspb.DropCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + // * + // @brief This method is used to test collection existence. + // + // @param HasCollectionRequest, collection name is going to be tested. + // + // @return BoolResponse + HasCollection(ctx context.Context, in *milvuspb.HasCollectionRequest, opts ...grpc.CallOption) (*milvuspb.BoolResponse, error) + // * + // @brief This method is used to get collection schema. + // + // @param DescribeCollectionRequest, target collection name. + // + // @return CollectionSchema + DescribeCollection(ctx context.Context, in *milvuspb.DescribeCollectionRequest, opts ...grpc.CallOption) (*milvuspb.DescribeCollectionResponse, error) + DescribeCollectionInternal(ctx context.Context, in *milvuspb.DescribeCollectionRequest, opts ...grpc.CallOption) (*milvuspb.DescribeCollectionResponse, error) + CreateAlias(ctx context.Context, in *milvuspb.CreateAliasRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + DropAlias(ctx context.Context, in *milvuspb.DropAliasRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + AlterAlias(ctx context.Context, in *milvuspb.AlterAliasRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + DescribeAlias(ctx context.Context, in *milvuspb.DescribeAliasRequest, opts ...grpc.CallOption) (*milvuspb.DescribeAliasResponse, error) + ListAliases(ctx context.Context, in *milvuspb.ListAliasesRequest, opts ...grpc.CallOption) (*milvuspb.ListAliasesResponse, error) + // * + // @brief This method is used to list all collections. + // + // @return StringListResponse, collection name list + ShowCollections(ctx context.Context, in *milvuspb.ShowCollectionsRequest, opts ...grpc.CallOption) (*milvuspb.ShowCollectionsResponse, error) + AlterCollection(ctx context.Context, in *milvuspb.AlterCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + AlterCollectionField(ctx context.Context, in *milvuspb.AlterCollectionFieldRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + // * + // @brief This method is used to create partition + // + // @return Status + CreatePartition(ctx context.Context, in *milvuspb.CreatePartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + // * + // @brief This method is used to drop partition + // + // @return Status + DropPartition(ctx context.Context, in *milvuspb.DropPartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + // * + // @brief This method is used to test partition existence. + // + // @return BoolResponse + HasPartition(ctx context.Context, in *milvuspb.HasPartitionRequest, opts ...grpc.CallOption) (*milvuspb.BoolResponse, error) + // * + // @brief This method is used to show partition information + // + // @param ShowPartitionRequest, target collection name. + // + // @return StringListResponse + ShowPartitions(ctx context.Context, in *milvuspb.ShowPartitionsRequest, opts ...grpc.CallOption) (*milvuspb.ShowPartitionsResponse, error) + ShowPartitionsInternal(ctx context.Context, in *milvuspb.ShowPartitionsRequest, opts ...grpc.CallOption) (*milvuspb.ShowPartitionsResponse, error) + ShowSegments(ctx context.Context, in *milvuspb.ShowSegmentsRequest, opts ...grpc.CallOption) (*milvuspb.ShowSegmentsResponse, error) + GetPChannelInfo(ctx context.Context, in *GetPChannelInfoRequest, opts ...grpc.CallOption) (*GetPChannelInfoResponse, error) + AllocTimestamp(ctx context.Context, in *AllocTimestampRequest, opts ...grpc.CallOption) (*AllocTimestampResponse, error) + AllocID(ctx context.Context, in *AllocIDRequest, opts ...grpc.CallOption) (*AllocIDResponse, error) + UpdateChannelTimeTick(ctx context.Context, in *internalpb.ChannelTimeTickMsg, opts ...grpc.CallOption) (*commonpb.Status, error) + InvalidateCollectionMetaCache(ctx context.Context, in *proxypb.InvalidateCollMetaCacheRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + ShowConfigurations(ctx context.Context, in *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error) + // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy + GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) + // https://wiki.lfaidata.foundation/display/MIL/MEP+27+--+Support+Basic+Authentication + CreateCredential(ctx context.Context, in *internalpb.CredentialInfo, opts ...grpc.CallOption) (*commonpb.Status, error) + UpdateCredential(ctx context.Context, in *internalpb.CredentialInfo, opts ...grpc.CallOption) (*commonpb.Status, error) + DeleteCredential(ctx context.Context, in *milvuspb.DeleteCredentialRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + ListCredUsers(ctx context.Context, in *milvuspb.ListCredUsersRequest, opts ...grpc.CallOption) (*milvuspb.ListCredUsersResponse, error) + // userd by proxy, not exposed to sdk + GetCredential(ctx context.Context, in *GetCredentialRequest, opts ...grpc.CallOption) (*GetCredentialResponse, error) + // https://wiki.lfaidata.foundation/display/MIL/MEP+29+--+Support+Role-Based+Access+Control + CreateRole(ctx context.Context, in *milvuspb.CreateRoleRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + DropRole(ctx context.Context, in *milvuspb.DropRoleRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + OperateUserRole(ctx context.Context, in *milvuspb.OperateUserRoleRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + SelectRole(ctx context.Context, in *milvuspb.SelectRoleRequest, opts ...grpc.CallOption) (*milvuspb.SelectRoleResponse, error) + SelectUser(ctx context.Context, in *milvuspb.SelectUserRequest, opts ...grpc.CallOption) (*milvuspb.SelectUserResponse, error) + OperatePrivilege(ctx context.Context, in *milvuspb.OperatePrivilegeRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + SelectGrant(ctx context.Context, in *milvuspb.SelectGrantRequest, opts ...grpc.CallOption) (*milvuspb.SelectGrantResponse, error) + ListPolicy(ctx context.Context, in *internalpb.ListPolicyRequest, opts ...grpc.CallOption) (*internalpb.ListPolicyResponse, error) + BackupRBAC(ctx context.Context, in *milvuspb.BackupRBACMetaRequest, opts ...grpc.CallOption) (*milvuspb.BackupRBACMetaResponse, error) + RestoreRBAC(ctx context.Context, in *milvuspb.RestoreRBACMetaRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + CreatePrivilegeGroup(ctx context.Context, in *milvuspb.CreatePrivilegeGroupRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + DropPrivilegeGroup(ctx context.Context, in *milvuspb.DropPrivilegeGroupRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + ListPrivilegeGroups(ctx context.Context, in *milvuspb.ListPrivilegeGroupsRequest, opts ...grpc.CallOption) (*milvuspb.ListPrivilegeGroupsResponse, error) + OperatePrivilegeGroup(ctx context.Context, in *milvuspb.OperatePrivilegeGroupRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + CheckHealth(ctx context.Context, in *milvuspb.CheckHealthRequest, opts ...grpc.CallOption) (*milvuspb.CheckHealthResponse, error) + RenameCollection(ctx context.Context, in *milvuspb.RenameCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + CreateDatabase(ctx context.Context, in *milvuspb.CreateDatabaseRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + DropDatabase(ctx context.Context, in *milvuspb.DropDatabaseRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + ListDatabases(ctx context.Context, in *milvuspb.ListDatabasesRequest, opts ...grpc.CallOption) (*milvuspb.ListDatabasesResponse, error) + DescribeDatabase(ctx context.Context, in *DescribeDatabaseRequest, opts ...grpc.CallOption) (*DescribeDatabaseResponse, error) + AlterDatabase(ctx context.Context, in *AlterDatabaseRequest, opts ...grpc.CallOption) (*commonpb.Status, error) +} + +type rootCoordClient struct { + cc grpc.ClientConnInterface +} + +func NewRootCoordClient(cc grpc.ClientConnInterface) RootCoordClient { + return &rootCoordClient{cc} +} + +func (c *rootCoordClient) GetComponentStates(ctx context.Context, in *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) { + out := new(milvuspb.ComponentStates) + err := c.cc.Invoke(ctx, RootCoord_GetComponentStates_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) GetTimeTickChannel(ctx context.Context, in *internalpb.GetTimeTickChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) { + out := new(milvuspb.StringResponse) + err := c.cc.Invoke(ctx, RootCoord_GetTimeTickChannel_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) { + out := new(milvuspb.StringResponse) + err := c.cc.Invoke(ctx, RootCoord_GetStatisticsChannel_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) CreateCollection(ctx context.Context, in *milvuspb.CreateCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_CreateCollection_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) DropCollection(ctx context.Context, in *milvuspb.DropCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_DropCollection_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) HasCollection(ctx context.Context, in *milvuspb.HasCollectionRequest, opts ...grpc.CallOption) (*milvuspb.BoolResponse, error) { + out := new(milvuspb.BoolResponse) + err := c.cc.Invoke(ctx, RootCoord_HasCollection_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) DescribeCollection(ctx context.Context, in *milvuspb.DescribeCollectionRequest, opts ...grpc.CallOption) (*milvuspb.DescribeCollectionResponse, error) { + out := new(milvuspb.DescribeCollectionResponse) + err := c.cc.Invoke(ctx, RootCoord_DescribeCollection_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) DescribeCollectionInternal(ctx context.Context, in *milvuspb.DescribeCollectionRequest, opts ...grpc.CallOption) (*milvuspb.DescribeCollectionResponse, error) { + out := new(milvuspb.DescribeCollectionResponse) + err := c.cc.Invoke(ctx, RootCoord_DescribeCollectionInternal_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) CreateAlias(ctx context.Context, in *milvuspb.CreateAliasRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_CreateAlias_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) DropAlias(ctx context.Context, in *milvuspb.DropAliasRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_DropAlias_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) AlterAlias(ctx context.Context, in *milvuspb.AlterAliasRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_AlterAlias_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) DescribeAlias(ctx context.Context, in *milvuspb.DescribeAliasRequest, opts ...grpc.CallOption) (*milvuspb.DescribeAliasResponse, error) { + out := new(milvuspb.DescribeAliasResponse) + err := c.cc.Invoke(ctx, RootCoord_DescribeAlias_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) ListAliases(ctx context.Context, in *milvuspb.ListAliasesRequest, opts ...grpc.CallOption) (*milvuspb.ListAliasesResponse, error) { + out := new(milvuspb.ListAliasesResponse) + err := c.cc.Invoke(ctx, RootCoord_ListAliases_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) ShowCollections(ctx context.Context, in *milvuspb.ShowCollectionsRequest, opts ...grpc.CallOption) (*milvuspb.ShowCollectionsResponse, error) { + out := new(milvuspb.ShowCollectionsResponse) + err := c.cc.Invoke(ctx, RootCoord_ShowCollections_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) AlterCollection(ctx context.Context, in *milvuspb.AlterCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_AlterCollection_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) AlterCollectionField(ctx context.Context, in *milvuspb.AlterCollectionFieldRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_AlterCollectionField_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) CreatePartition(ctx context.Context, in *milvuspb.CreatePartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_CreatePartition_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) DropPartition(ctx context.Context, in *milvuspb.DropPartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_DropPartition_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) HasPartition(ctx context.Context, in *milvuspb.HasPartitionRequest, opts ...grpc.CallOption) (*milvuspb.BoolResponse, error) { + out := new(milvuspb.BoolResponse) + err := c.cc.Invoke(ctx, RootCoord_HasPartition_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) ShowPartitions(ctx context.Context, in *milvuspb.ShowPartitionsRequest, opts ...grpc.CallOption) (*milvuspb.ShowPartitionsResponse, error) { + out := new(milvuspb.ShowPartitionsResponse) + err := c.cc.Invoke(ctx, RootCoord_ShowPartitions_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) ShowPartitionsInternal(ctx context.Context, in *milvuspb.ShowPartitionsRequest, opts ...grpc.CallOption) (*milvuspb.ShowPartitionsResponse, error) { + out := new(milvuspb.ShowPartitionsResponse) + err := c.cc.Invoke(ctx, RootCoord_ShowPartitionsInternal_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) ShowSegments(ctx context.Context, in *milvuspb.ShowSegmentsRequest, opts ...grpc.CallOption) (*milvuspb.ShowSegmentsResponse, error) { + out := new(milvuspb.ShowSegmentsResponse) + err := c.cc.Invoke(ctx, RootCoord_ShowSegments_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) GetPChannelInfo(ctx context.Context, in *GetPChannelInfoRequest, opts ...grpc.CallOption) (*GetPChannelInfoResponse, error) { + out := new(GetPChannelInfoResponse) + err := c.cc.Invoke(ctx, RootCoord_GetPChannelInfo_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) AllocTimestamp(ctx context.Context, in *AllocTimestampRequest, opts ...grpc.CallOption) (*AllocTimestampResponse, error) { + out := new(AllocTimestampResponse) + err := c.cc.Invoke(ctx, RootCoord_AllocTimestamp_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) AllocID(ctx context.Context, in *AllocIDRequest, opts ...grpc.CallOption) (*AllocIDResponse, error) { + out := new(AllocIDResponse) + err := c.cc.Invoke(ctx, RootCoord_AllocID_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) UpdateChannelTimeTick(ctx context.Context, in *internalpb.ChannelTimeTickMsg, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_UpdateChannelTimeTick_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) InvalidateCollectionMetaCache(ctx context.Context, in *proxypb.InvalidateCollMetaCacheRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_InvalidateCollectionMetaCache_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) ShowConfigurations(ctx context.Context, in *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error) { + out := new(internalpb.ShowConfigurationsResponse) + err := c.cc.Invoke(ctx, RootCoord_ShowConfigurations_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) { + out := new(milvuspb.GetMetricsResponse) + err := c.cc.Invoke(ctx, RootCoord_GetMetrics_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) CreateCredential(ctx context.Context, in *internalpb.CredentialInfo, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_CreateCredential_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) UpdateCredential(ctx context.Context, in *internalpb.CredentialInfo, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_UpdateCredential_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) DeleteCredential(ctx context.Context, in *milvuspb.DeleteCredentialRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_DeleteCredential_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) ListCredUsers(ctx context.Context, in *milvuspb.ListCredUsersRequest, opts ...grpc.CallOption) (*milvuspb.ListCredUsersResponse, error) { + out := new(milvuspb.ListCredUsersResponse) + err := c.cc.Invoke(ctx, RootCoord_ListCredUsers_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) GetCredential(ctx context.Context, in *GetCredentialRequest, opts ...grpc.CallOption) (*GetCredentialResponse, error) { + out := new(GetCredentialResponse) + err := c.cc.Invoke(ctx, RootCoord_GetCredential_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) CreateRole(ctx context.Context, in *milvuspb.CreateRoleRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_CreateRole_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) DropRole(ctx context.Context, in *milvuspb.DropRoleRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_DropRole_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) OperateUserRole(ctx context.Context, in *milvuspb.OperateUserRoleRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_OperateUserRole_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) SelectRole(ctx context.Context, in *milvuspb.SelectRoleRequest, opts ...grpc.CallOption) (*milvuspb.SelectRoleResponse, error) { + out := new(milvuspb.SelectRoleResponse) + err := c.cc.Invoke(ctx, RootCoord_SelectRole_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) SelectUser(ctx context.Context, in *milvuspb.SelectUserRequest, opts ...grpc.CallOption) (*milvuspb.SelectUserResponse, error) { + out := new(milvuspb.SelectUserResponse) + err := c.cc.Invoke(ctx, RootCoord_SelectUser_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) OperatePrivilege(ctx context.Context, in *milvuspb.OperatePrivilegeRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_OperatePrivilege_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) SelectGrant(ctx context.Context, in *milvuspb.SelectGrantRequest, opts ...grpc.CallOption) (*milvuspb.SelectGrantResponse, error) { + out := new(milvuspb.SelectGrantResponse) + err := c.cc.Invoke(ctx, RootCoord_SelectGrant_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) ListPolicy(ctx context.Context, in *internalpb.ListPolicyRequest, opts ...grpc.CallOption) (*internalpb.ListPolicyResponse, error) { + out := new(internalpb.ListPolicyResponse) + err := c.cc.Invoke(ctx, RootCoord_ListPolicy_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) BackupRBAC(ctx context.Context, in *milvuspb.BackupRBACMetaRequest, opts ...grpc.CallOption) (*milvuspb.BackupRBACMetaResponse, error) { + out := new(milvuspb.BackupRBACMetaResponse) + err := c.cc.Invoke(ctx, RootCoord_BackupRBAC_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) RestoreRBAC(ctx context.Context, in *milvuspb.RestoreRBACMetaRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_RestoreRBAC_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) CreatePrivilegeGroup(ctx context.Context, in *milvuspb.CreatePrivilegeGroupRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_CreatePrivilegeGroup_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) DropPrivilegeGroup(ctx context.Context, in *milvuspb.DropPrivilegeGroupRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_DropPrivilegeGroup_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) ListPrivilegeGroups(ctx context.Context, in *milvuspb.ListPrivilegeGroupsRequest, opts ...grpc.CallOption) (*milvuspb.ListPrivilegeGroupsResponse, error) { + out := new(milvuspb.ListPrivilegeGroupsResponse) + err := c.cc.Invoke(ctx, RootCoord_ListPrivilegeGroups_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) OperatePrivilegeGroup(ctx context.Context, in *milvuspb.OperatePrivilegeGroupRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_OperatePrivilegeGroup_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) CheckHealth(ctx context.Context, in *milvuspb.CheckHealthRequest, opts ...grpc.CallOption) (*milvuspb.CheckHealthResponse, error) { + out := new(milvuspb.CheckHealthResponse) + err := c.cc.Invoke(ctx, RootCoord_CheckHealth_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) RenameCollection(ctx context.Context, in *milvuspb.RenameCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_RenameCollection_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) CreateDatabase(ctx context.Context, in *milvuspb.CreateDatabaseRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_CreateDatabase_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) DropDatabase(ctx context.Context, in *milvuspb.DropDatabaseRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_DropDatabase_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) ListDatabases(ctx context.Context, in *milvuspb.ListDatabasesRequest, opts ...grpc.CallOption) (*milvuspb.ListDatabasesResponse, error) { + out := new(milvuspb.ListDatabasesResponse) + err := c.cc.Invoke(ctx, RootCoord_ListDatabases_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) DescribeDatabase(ctx context.Context, in *DescribeDatabaseRequest, opts ...grpc.CallOption) (*DescribeDatabaseResponse, error) { + out := new(DescribeDatabaseResponse) + err := c.cc.Invoke(ctx, RootCoord_DescribeDatabase_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rootCoordClient) AlterDatabase(ctx context.Context, in *AlterDatabaseRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, RootCoord_AlterDatabase_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// RootCoordServer is the server API for RootCoord service. +// All implementations should embed UnimplementedRootCoordServer +// for forward compatibility +type RootCoordServer interface { + GetComponentStates(context.Context, *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) + GetTimeTickChannel(context.Context, *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error) + GetStatisticsChannel(context.Context, *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) + // * + // @brief This method is used to create collection + // + // @param CreateCollectionRequest, use to provide collection information to be created. + // + // @return Status + CreateCollection(context.Context, *milvuspb.CreateCollectionRequest) (*commonpb.Status, error) + // * + // @brief This method is used to delete collection. + // + // @param DropCollectionRequest, collection name is going to be deleted. + // + // @return Status + DropCollection(context.Context, *milvuspb.DropCollectionRequest) (*commonpb.Status, error) + // * + // @brief This method is used to test collection existence. + // + // @param HasCollectionRequest, collection name is going to be tested. + // + // @return BoolResponse + HasCollection(context.Context, *milvuspb.HasCollectionRequest) (*milvuspb.BoolResponse, error) + // * + // @brief This method is used to get collection schema. + // + // @param DescribeCollectionRequest, target collection name. + // + // @return CollectionSchema + DescribeCollection(context.Context, *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) + DescribeCollectionInternal(context.Context, *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) + CreateAlias(context.Context, *milvuspb.CreateAliasRequest) (*commonpb.Status, error) + DropAlias(context.Context, *milvuspb.DropAliasRequest) (*commonpb.Status, error) + AlterAlias(context.Context, *milvuspb.AlterAliasRequest) (*commonpb.Status, error) + DescribeAlias(context.Context, *milvuspb.DescribeAliasRequest) (*milvuspb.DescribeAliasResponse, error) + ListAliases(context.Context, *milvuspb.ListAliasesRequest) (*milvuspb.ListAliasesResponse, error) + // * + // @brief This method is used to list all collections. + // + // @return StringListResponse, collection name list + ShowCollections(context.Context, *milvuspb.ShowCollectionsRequest) (*milvuspb.ShowCollectionsResponse, error) + AlterCollection(context.Context, *milvuspb.AlterCollectionRequest) (*commonpb.Status, error) + AlterCollectionField(context.Context, *milvuspb.AlterCollectionFieldRequest) (*commonpb.Status, error) + // * + // @brief This method is used to create partition + // + // @return Status + CreatePartition(context.Context, *milvuspb.CreatePartitionRequest) (*commonpb.Status, error) + // * + // @brief This method is used to drop partition + // + // @return Status + DropPartition(context.Context, *milvuspb.DropPartitionRequest) (*commonpb.Status, error) + // * + // @brief This method is used to test partition existence. + // + // @return BoolResponse + HasPartition(context.Context, *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error) + // * + // @brief This method is used to show partition information + // + // @param ShowPartitionRequest, target collection name. + // + // @return StringListResponse + ShowPartitions(context.Context, *milvuspb.ShowPartitionsRequest) (*milvuspb.ShowPartitionsResponse, error) + ShowPartitionsInternal(context.Context, *milvuspb.ShowPartitionsRequest) (*milvuspb.ShowPartitionsResponse, error) + ShowSegments(context.Context, *milvuspb.ShowSegmentsRequest) (*milvuspb.ShowSegmentsResponse, error) + GetPChannelInfo(context.Context, *GetPChannelInfoRequest) (*GetPChannelInfoResponse, error) + AllocTimestamp(context.Context, *AllocTimestampRequest) (*AllocTimestampResponse, error) + AllocID(context.Context, *AllocIDRequest) (*AllocIDResponse, error) + UpdateChannelTimeTick(context.Context, *internalpb.ChannelTimeTickMsg) (*commonpb.Status, error) + InvalidateCollectionMetaCache(context.Context, *proxypb.InvalidateCollMetaCacheRequest) (*commonpb.Status, error) + ShowConfigurations(context.Context, *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) + // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy + GetMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) + // https://wiki.lfaidata.foundation/display/MIL/MEP+27+--+Support+Basic+Authentication + CreateCredential(context.Context, *internalpb.CredentialInfo) (*commonpb.Status, error) + UpdateCredential(context.Context, *internalpb.CredentialInfo) (*commonpb.Status, error) + DeleteCredential(context.Context, *milvuspb.DeleteCredentialRequest) (*commonpb.Status, error) + ListCredUsers(context.Context, *milvuspb.ListCredUsersRequest) (*milvuspb.ListCredUsersResponse, error) + // userd by proxy, not exposed to sdk + GetCredential(context.Context, *GetCredentialRequest) (*GetCredentialResponse, error) + // https://wiki.lfaidata.foundation/display/MIL/MEP+29+--+Support+Role-Based+Access+Control + CreateRole(context.Context, *milvuspb.CreateRoleRequest) (*commonpb.Status, error) + DropRole(context.Context, *milvuspb.DropRoleRequest) (*commonpb.Status, error) + OperateUserRole(context.Context, *milvuspb.OperateUserRoleRequest) (*commonpb.Status, error) + SelectRole(context.Context, *milvuspb.SelectRoleRequest) (*milvuspb.SelectRoleResponse, error) + SelectUser(context.Context, *milvuspb.SelectUserRequest) (*milvuspb.SelectUserResponse, error) + OperatePrivilege(context.Context, *milvuspb.OperatePrivilegeRequest) (*commonpb.Status, error) + SelectGrant(context.Context, *milvuspb.SelectGrantRequest) (*milvuspb.SelectGrantResponse, error) + ListPolicy(context.Context, *internalpb.ListPolicyRequest) (*internalpb.ListPolicyResponse, error) + BackupRBAC(context.Context, *milvuspb.BackupRBACMetaRequest) (*milvuspb.BackupRBACMetaResponse, error) + RestoreRBAC(context.Context, *milvuspb.RestoreRBACMetaRequest) (*commonpb.Status, error) + CreatePrivilegeGroup(context.Context, *milvuspb.CreatePrivilegeGroupRequest) (*commonpb.Status, error) + DropPrivilegeGroup(context.Context, *milvuspb.DropPrivilegeGroupRequest) (*commonpb.Status, error) + ListPrivilegeGroups(context.Context, *milvuspb.ListPrivilegeGroupsRequest) (*milvuspb.ListPrivilegeGroupsResponse, error) + OperatePrivilegeGroup(context.Context, *milvuspb.OperatePrivilegeGroupRequest) (*commonpb.Status, error) + CheckHealth(context.Context, *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error) + RenameCollection(context.Context, *milvuspb.RenameCollectionRequest) (*commonpb.Status, error) + CreateDatabase(context.Context, *milvuspb.CreateDatabaseRequest) (*commonpb.Status, error) + DropDatabase(context.Context, *milvuspb.DropDatabaseRequest) (*commonpb.Status, error) + ListDatabases(context.Context, *milvuspb.ListDatabasesRequest) (*milvuspb.ListDatabasesResponse, error) + DescribeDatabase(context.Context, *DescribeDatabaseRequest) (*DescribeDatabaseResponse, error) + AlterDatabase(context.Context, *AlterDatabaseRequest) (*commonpb.Status, error) +} + +// UnimplementedRootCoordServer should be embedded to have forward compatible implementations. +type UnimplementedRootCoordServer struct { +} + +func (UnimplementedRootCoordServer) GetComponentStates(context.Context, *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetComponentStates not implemented") +} +func (UnimplementedRootCoordServer) GetTimeTickChannel(context.Context, *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetTimeTickChannel not implemented") +} +func (UnimplementedRootCoordServer) GetStatisticsChannel(context.Context, *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetStatisticsChannel not implemented") +} +func (UnimplementedRootCoordServer) CreateCollection(context.Context, *milvuspb.CreateCollectionRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method CreateCollection not implemented") +} +func (UnimplementedRootCoordServer) DropCollection(context.Context, *milvuspb.DropCollectionRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method DropCollection not implemented") +} +func (UnimplementedRootCoordServer) HasCollection(context.Context, *milvuspb.HasCollectionRequest) (*milvuspb.BoolResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method HasCollection not implemented") +} +func (UnimplementedRootCoordServer) DescribeCollection(context.Context, *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DescribeCollection not implemented") +} +func (UnimplementedRootCoordServer) DescribeCollectionInternal(context.Context, *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DescribeCollectionInternal not implemented") +} +func (UnimplementedRootCoordServer) CreateAlias(context.Context, *milvuspb.CreateAliasRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method CreateAlias not implemented") +} +func (UnimplementedRootCoordServer) DropAlias(context.Context, *milvuspb.DropAliasRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method DropAlias not implemented") +} +func (UnimplementedRootCoordServer) AlterAlias(context.Context, *milvuspb.AlterAliasRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method AlterAlias not implemented") +} +func (UnimplementedRootCoordServer) DescribeAlias(context.Context, *milvuspb.DescribeAliasRequest) (*milvuspb.DescribeAliasResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DescribeAlias not implemented") +} +func (UnimplementedRootCoordServer) ListAliases(context.Context, *milvuspb.ListAliasesRequest) (*milvuspb.ListAliasesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListAliases not implemented") +} +func (UnimplementedRootCoordServer) ShowCollections(context.Context, *milvuspb.ShowCollectionsRequest) (*milvuspb.ShowCollectionsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ShowCollections not implemented") +} +func (UnimplementedRootCoordServer) AlterCollection(context.Context, *milvuspb.AlterCollectionRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method AlterCollection not implemented") +} +func (UnimplementedRootCoordServer) AlterCollectionField(context.Context, *milvuspb.AlterCollectionFieldRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method AlterCollectionField not implemented") +} +func (UnimplementedRootCoordServer) CreatePartition(context.Context, *milvuspb.CreatePartitionRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method CreatePartition not implemented") +} +func (UnimplementedRootCoordServer) DropPartition(context.Context, *milvuspb.DropPartitionRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method DropPartition not implemented") +} +func (UnimplementedRootCoordServer) HasPartition(context.Context, *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method HasPartition not implemented") +} +func (UnimplementedRootCoordServer) ShowPartitions(context.Context, *milvuspb.ShowPartitionsRequest) (*milvuspb.ShowPartitionsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ShowPartitions not implemented") +} +func (UnimplementedRootCoordServer) ShowPartitionsInternal(context.Context, *milvuspb.ShowPartitionsRequest) (*milvuspb.ShowPartitionsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ShowPartitionsInternal not implemented") +} +func (UnimplementedRootCoordServer) ShowSegments(context.Context, *milvuspb.ShowSegmentsRequest) (*milvuspb.ShowSegmentsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ShowSegments not implemented") +} +func (UnimplementedRootCoordServer) GetPChannelInfo(context.Context, *GetPChannelInfoRequest) (*GetPChannelInfoResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetPChannelInfo not implemented") +} +func (UnimplementedRootCoordServer) AllocTimestamp(context.Context, *AllocTimestampRequest) (*AllocTimestampResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method AllocTimestamp not implemented") +} +func (UnimplementedRootCoordServer) AllocID(context.Context, *AllocIDRequest) (*AllocIDResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method AllocID not implemented") +} +func (UnimplementedRootCoordServer) UpdateChannelTimeTick(context.Context, *internalpb.ChannelTimeTickMsg) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateChannelTimeTick not implemented") +} +func (UnimplementedRootCoordServer) InvalidateCollectionMetaCache(context.Context, *proxypb.InvalidateCollMetaCacheRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method InvalidateCollectionMetaCache not implemented") +} +func (UnimplementedRootCoordServer) ShowConfigurations(context.Context, *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ShowConfigurations not implemented") +} +func (UnimplementedRootCoordServer) GetMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetMetrics not implemented") +} +func (UnimplementedRootCoordServer) CreateCredential(context.Context, *internalpb.CredentialInfo) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method CreateCredential not implemented") +} +func (UnimplementedRootCoordServer) UpdateCredential(context.Context, *internalpb.CredentialInfo) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateCredential not implemented") +} +func (UnimplementedRootCoordServer) DeleteCredential(context.Context, *milvuspb.DeleteCredentialRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method DeleteCredential not implemented") +} +func (UnimplementedRootCoordServer) ListCredUsers(context.Context, *milvuspb.ListCredUsersRequest) (*milvuspb.ListCredUsersResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListCredUsers not implemented") +} +func (UnimplementedRootCoordServer) GetCredential(context.Context, *GetCredentialRequest) (*GetCredentialResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetCredential not implemented") +} +func (UnimplementedRootCoordServer) CreateRole(context.Context, *milvuspb.CreateRoleRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method CreateRole not implemented") +} +func (UnimplementedRootCoordServer) DropRole(context.Context, *milvuspb.DropRoleRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method DropRole not implemented") +} +func (UnimplementedRootCoordServer) OperateUserRole(context.Context, *milvuspb.OperateUserRoleRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method OperateUserRole not implemented") +} +func (UnimplementedRootCoordServer) SelectRole(context.Context, *milvuspb.SelectRoleRequest) (*milvuspb.SelectRoleResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method SelectRole not implemented") +} +func (UnimplementedRootCoordServer) SelectUser(context.Context, *milvuspb.SelectUserRequest) (*milvuspb.SelectUserResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method SelectUser not implemented") +} +func (UnimplementedRootCoordServer) OperatePrivilege(context.Context, *milvuspb.OperatePrivilegeRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method OperatePrivilege not implemented") +} +func (UnimplementedRootCoordServer) SelectGrant(context.Context, *milvuspb.SelectGrantRequest) (*milvuspb.SelectGrantResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method SelectGrant not implemented") +} +func (UnimplementedRootCoordServer) ListPolicy(context.Context, *internalpb.ListPolicyRequest) (*internalpb.ListPolicyResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListPolicy not implemented") +} +func (UnimplementedRootCoordServer) BackupRBAC(context.Context, *milvuspb.BackupRBACMetaRequest) (*milvuspb.BackupRBACMetaResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method BackupRBAC not implemented") +} +func (UnimplementedRootCoordServer) RestoreRBAC(context.Context, *milvuspb.RestoreRBACMetaRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method RestoreRBAC not implemented") +} +func (UnimplementedRootCoordServer) CreatePrivilegeGroup(context.Context, *milvuspb.CreatePrivilegeGroupRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method CreatePrivilegeGroup not implemented") +} +func (UnimplementedRootCoordServer) DropPrivilegeGroup(context.Context, *milvuspb.DropPrivilegeGroupRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method DropPrivilegeGroup not implemented") +} +func (UnimplementedRootCoordServer) ListPrivilegeGroups(context.Context, *milvuspb.ListPrivilegeGroupsRequest) (*milvuspb.ListPrivilegeGroupsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListPrivilegeGroups not implemented") +} +func (UnimplementedRootCoordServer) OperatePrivilegeGroup(context.Context, *milvuspb.OperatePrivilegeGroupRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method OperatePrivilegeGroup not implemented") +} +func (UnimplementedRootCoordServer) CheckHealth(context.Context, *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method CheckHealth not implemented") +} +func (UnimplementedRootCoordServer) RenameCollection(context.Context, *milvuspb.RenameCollectionRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method RenameCollection not implemented") +} +func (UnimplementedRootCoordServer) CreateDatabase(context.Context, *milvuspb.CreateDatabaseRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method CreateDatabase not implemented") +} +func (UnimplementedRootCoordServer) DropDatabase(context.Context, *milvuspb.DropDatabaseRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method DropDatabase not implemented") +} +func (UnimplementedRootCoordServer) ListDatabases(context.Context, *milvuspb.ListDatabasesRequest) (*milvuspb.ListDatabasesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListDatabases not implemented") +} +func (UnimplementedRootCoordServer) DescribeDatabase(context.Context, *DescribeDatabaseRequest) (*DescribeDatabaseResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DescribeDatabase not implemented") +} +func (UnimplementedRootCoordServer) AlterDatabase(context.Context, *AlterDatabaseRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method AlterDatabase not implemented") +} + +// UnsafeRootCoordServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to RootCoordServer will +// result in compilation errors. +type UnsafeRootCoordServer interface { + mustEmbedUnimplementedRootCoordServer() +} + +func RegisterRootCoordServer(s grpc.ServiceRegistrar, srv RootCoordServer) { + s.RegisterService(&RootCoord_ServiceDesc, srv) +} + +func _RootCoord_GetComponentStates_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.GetComponentStatesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).GetComponentStates(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_GetComponentStates_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).GetComponentStates(ctx, req.(*milvuspb.GetComponentStatesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_GetTimeTickChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.GetTimeTickChannelRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).GetTimeTickChannel(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_GetTimeTickChannel_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).GetTimeTickChannel(ctx, req.(*internalpb.GetTimeTickChannelRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_GetStatisticsChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.GetStatisticsChannelRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).GetStatisticsChannel(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_GetStatisticsChannel_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).GetStatisticsChannel(ctx, req.(*internalpb.GetStatisticsChannelRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_CreateCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.CreateCollectionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).CreateCollection(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_CreateCollection_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).CreateCollection(ctx, req.(*milvuspb.CreateCollectionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_DropCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.DropCollectionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).DropCollection(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_DropCollection_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).DropCollection(ctx, req.(*milvuspb.DropCollectionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_HasCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.HasCollectionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).HasCollection(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_HasCollection_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).HasCollection(ctx, req.(*milvuspb.HasCollectionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_DescribeCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.DescribeCollectionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).DescribeCollection(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_DescribeCollection_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).DescribeCollection(ctx, req.(*milvuspb.DescribeCollectionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_DescribeCollectionInternal_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.DescribeCollectionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).DescribeCollectionInternal(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_DescribeCollectionInternal_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).DescribeCollectionInternal(ctx, req.(*milvuspb.DescribeCollectionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_CreateAlias_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.CreateAliasRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).CreateAlias(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_CreateAlias_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).CreateAlias(ctx, req.(*milvuspb.CreateAliasRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_DropAlias_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.DropAliasRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).DropAlias(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_DropAlias_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).DropAlias(ctx, req.(*milvuspb.DropAliasRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_AlterAlias_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.AlterAliasRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).AlterAlias(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_AlterAlias_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).AlterAlias(ctx, req.(*milvuspb.AlterAliasRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_DescribeAlias_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.DescribeAliasRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).DescribeAlias(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_DescribeAlias_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).DescribeAlias(ctx, req.(*milvuspb.DescribeAliasRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_ListAliases_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.ListAliasesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).ListAliases(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_ListAliases_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).ListAliases(ctx, req.(*milvuspb.ListAliasesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_ShowCollections_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.ShowCollectionsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).ShowCollections(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_ShowCollections_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).ShowCollections(ctx, req.(*milvuspb.ShowCollectionsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_AlterCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.AlterCollectionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).AlterCollection(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_AlterCollection_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).AlterCollection(ctx, req.(*milvuspb.AlterCollectionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_AlterCollectionField_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.AlterCollectionFieldRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).AlterCollectionField(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_AlterCollectionField_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).AlterCollectionField(ctx, req.(*milvuspb.AlterCollectionFieldRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_CreatePartition_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.CreatePartitionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).CreatePartition(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_CreatePartition_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).CreatePartition(ctx, req.(*milvuspb.CreatePartitionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_DropPartition_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.DropPartitionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).DropPartition(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_DropPartition_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).DropPartition(ctx, req.(*milvuspb.DropPartitionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_HasPartition_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.HasPartitionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).HasPartition(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_HasPartition_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).HasPartition(ctx, req.(*milvuspb.HasPartitionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_ShowPartitions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.ShowPartitionsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).ShowPartitions(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_ShowPartitions_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).ShowPartitions(ctx, req.(*milvuspb.ShowPartitionsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_ShowPartitionsInternal_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.ShowPartitionsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).ShowPartitionsInternal(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_ShowPartitionsInternal_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).ShowPartitionsInternal(ctx, req.(*milvuspb.ShowPartitionsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_ShowSegments_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.ShowSegmentsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).ShowSegments(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_ShowSegments_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).ShowSegments(ctx, req.(*milvuspb.ShowSegmentsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_GetPChannelInfo_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetPChannelInfoRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).GetPChannelInfo(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_GetPChannelInfo_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).GetPChannelInfo(ctx, req.(*GetPChannelInfoRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_AllocTimestamp_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(AllocTimestampRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).AllocTimestamp(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_AllocTimestamp_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).AllocTimestamp(ctx, req.(*AllocTimestampRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_AllocID_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(AllocIDRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).AllocID(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_AllocID_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).AllocID(ctx, req.(*AllocIDRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_UpdateChannelTimeTick_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.ChannelTimeTickMsg) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).UpdateChannelTimeTick(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_UpdateChannelTimeTick_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).UpdateChannelTimeTick(ctx, req.(*internalpb.ChannelTimeTickMsg)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_InvalidateCollectionMetaCache_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(proxypb.InvalidateCollMetaCacheRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).InvalidateCollectionMetaCache(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_InvalidateCollectionMetaCache_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).InvalidateCollectionMetaCache(ctx, req.(*proxypb.InvalidateCollMetaCacheRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_ShowConfigurations_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.ShowConfigurationsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).ShowConfigurations(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_ShowConfigurations_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).ShowConfigurations(ctx, req.(*internalpb.ShowConfigurationsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_GetMetrics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.GetMetricsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).GetMetrics(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_GetMetrics_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).GetMetrics(ctx, req.(*milvuspb.GetMetricsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_CreateCredential_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.CredentialInfo) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).CreateCredential(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_CreateCredential_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).CreateCredential(ctx, req.(*internalpb.CredentialInfo)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_UpdateCredential_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.CredentialInfo) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).UpdateCredential(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_UpdateCredential_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).UpdateCredential(ctx, req.(*internalpb.CredentialInfo)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_DeleteCredential_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.DeleteCredentialRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).DeleteCredential(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_DeleteCredential_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).DeleteCredential(ctx, req.(*milvuspb.DeleteCredentialRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_ListCredUsers_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.ListCredUsersRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).ListCredUsers(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_ListCredUsers_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).ListCredUsers(ctx, req.(*milvuspb.ListCredUsersRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_GetCredential_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetCredentialRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).GetCredential(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_GetCredential_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).GetCredential(ctx, req.(*GetCredentialRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_CreateRole_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.CreateRoleRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).CreateRole(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_CreateRole_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).CreateRole(ctx, req.(*milvuspb.CreateRoleRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_DropRole_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.DropRoleRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).DropRole(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_DropRole_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).DropRole(ctx, req.(*milvuspb.DropRoleRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_OperateUserRole_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.OperateUserRoleRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).OperateUserRole(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_OperateUserRole_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).OperateUserRole(ctx, req.(*milvuspb.OperateUserRoleRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_SelectRole_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.SelectRoleRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).SelectRole(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_SelectRole_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).SelectRole(ctx, req.(*milvuspb.SelectRoleRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_SelectUser_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.SelectUserRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).SelectUser(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_SelectUser_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).SelectUser(ctx, req.(*milvuspb.SelectUserRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_OperatePrivilege_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.OperatePrivilegeRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).OperatePrivilege(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_OperatePrivilege_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).OperatePrivilege(ctx, req.(*milvuspb.OperatePrivilegeRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_SelectGrant_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.SelectGrantRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).SelectGrant(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_SelectGrant_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).SelectGrant(ctx, req.(*milvuspb.SelectGrantRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_ListPolicy_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.ListPolicyRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).ListPolicy(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_ListPolicy_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).ListPolicy(ctx, req.(*internalpb.ListPolicyRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_BackupRBAC_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.BackupRBACMetaRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).BackupRBAC(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_BackupRBAC_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).BackupRBAC(ctx, req.(*milvuspb.BackupRBACMetaRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_RestoreRBAC_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.RestoreRBACMetaRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).RestoreRBAC(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_RestoreRBAC_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).RestoreRBAC(ctx, req.(*milvuspb.RestoreRBACMetaRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_CreatePrivilegeGroup_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.CreatePrivilegeGroupRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).CreatePrivilegeGroup(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_CreatePrivilegeGroup_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).CreatePrivilegeGroup(ctx, req.(*milvuspb.CreatePrivilegeGroupRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_DropPrivilegeGroup_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.DropPrivilegeGroupRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).DropPrivilegeGroup(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_DropPrivilegeGroup_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).DropPrivilegeGroup(ctx, req.(*milvuspb.DropPrivilegeGroupRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_ListPrivilegeGroups_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.ListPrivilegeGroupsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).ListPrivilegeGroups(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_ListPrivilegeGroups_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).ListPrivilegeGroups(ctx, req.(*milvuspb.ListPrivilegeGroupsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_OperatePrivilegeGroup_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.OperatePrivilegeGroupRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).OperatePrivilegeGroup(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_OperatePrivilegeGroup_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).OperatePrivilegeGroup(ctx, req.(*milvuspb.OperatePrivilegeGroupRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_CheckHealth_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.CheckHealthRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).CheckHealth(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_CheckHealth_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).CheckHealth(ctx, req.(*milvuspb.CheckHealthRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_RenameCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.RenameCollectionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).RenameCollection(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_RenameCollection_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).RenameCollection(ctx, req.(*milvuspb.RenameCollectionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_CreateDatabase_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.CreateDatabaseRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).CreateDatabase(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_CreateDatabase_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).CreateDatabase(ctx, req.(*milvuspb.CreateDatabaseRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_DropDatabase_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.DropDatabaseRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).DropDatabase(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_DropDatabase_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).DropDatabase(ctx, req.(*milvuspb.DropDatabaseRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_ListDatabases_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.ListDatabasesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).ListDatabases(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_ListDatabases_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).ListDatabases(ctx, req.(*milvuspb.ListDatabasesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_DescribeDatabase_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DescribeDatabaseRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).DescribeDatabase(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_DescribeDatabase_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).DescribeDatabase(ctx, req.(*DescribeDatabaseRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RootCoord_AlterDatabase_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(AlterDatabaseRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).AlterDatabase(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_AlterDatabase_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).AlterDatabase(ctx, req.(*AlterDatabaseRequest)) + } + return interceptor(ctx, in, info, handler) +} + +// RootCoord_ServiceDesc is the grpc.ServiceDesc for RootCoord service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var RootCoord_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "milvus.proto.rootcoord.RootCoord", + HandlerType: (*RootCoordServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "GetComponentStates", + Handler: _RootCoord_GetComponentStates_Handler, + }, + { + MethodName: "GetTimeTickChannel", + Handler: _RootCoord_GetTimeTickChannel_Handler, + }, + { + MethodName: "GetStatisticsChannel", + Handler: _RootCoord_GetStatisticsChannel_Handler, + }, + { + MethodName: "CreateCollection", + Handler: _RootCoord_CreateCollection_Handler, + }, + { + MethodName: "DropCollection", + Handler: _RootCoord_DropCollection_Handler, + }, + { + MethodName: "HasCollection", + Handler: _RootCoord_HasCollection_Handler, + }, + { + MethodName: "DescribeCollection", + Handler: _RootCoord_DescribeCollection_Handler, + }, + { + MethodName: "DescribeCollectionInternal", + Handler: _RootCoord_DescribeCollectionInternal_Handler, + }, + { + MethodName: "CreateAlias", + Handler: _RootCoord_CreateAlias_Handler, + }, + { + MethodName: "DropAlias", + Handler: _RootCoord_DropAlias_Handler, + }, + { + MethodName: "AlterAlias", + Handler: _RootCoord_AlterAlias_Handler, + }, + { + MethodName: "DescribeAlias", + Handler: _RootCoord_DescribeAlias_Handler, + }, + { + MethodName: "ListAliases", + Handler: _RootCoord_ListAliases_Handler, + }, + { + MethodName: "ShowCollections", + Handler: _RootCoord_ShowCollections_Handler, + }, + { + MethodName: "AlterCollection", + Handler: _RootCoord_AlterCollection_Handler, + }, + { + MethodName: "AlterCollectionField", + Handler: _RootCoord_AlterCollectionField_Handler, + }, + { + MethodName: "CreatePartition", + Handler: _RootCoord_CreatePartition_Handler, + }, + { + MethodName: "DropPartition", + Handler: _RootCoord_DropPartition_Handler, + }, + { + MethodName: "HasPartition", + Handler: _RootCoord_HasPartition_Handler, + }, + { + MethodName: "ShowPartitions", + Handler: _RootCoord_ShowPartitions_Handler, + }, + { + MethodName: "ShowPartitionsInternal", + Handler: _RootCoord_ShowPartitionsInternal_Handler, + }, + { + MethodName: "ShowSegments", + Handler: _RootCoord_ShowSegments_Handler, + }, + { + MethodName: "GetPChannelInfo", + Handler: _RootCoord_GetPChannelInfo_Handler, + }, + { + MethodName: "AllocTimestamp", + Handler: _RootCoord_AllocTimestamp_Handler, + }, + { + MethodName: "AllocID", + Handler: _RootCoord_AllocID_Handler, + }, + { + MethodName: "UpdateChannelTimeTick", + Handler: _RootCoord_UpdateChannelTimeTick_Handler, + }, + { + MethodName: "InvalidateCollectionMetaCache", + Handler: _RootCoord_InvalidateCollectionMetaCache_Handler, + }, + { + MethodName: "ShowConfigurations", + Handler: _RootCoord_ShowConfigurations_Handler, + }, + { + MethodName: "GetMetrics", + Handler: _RootCoord_GetMetrics_Handler, + }, + { + MethodName: "CreateCredential", + Handler: _RootCoord_CreateCredential_Handler, + }, + { + MethodName: "UpdateCredential", + Handler: _RootCoord_UpdateCredential_Handler, + }, + { + MethodName: "DeleteCredential", + Handler: _RootCoord_DeleteCredential_Handler, + }, + { + MethodName: "ListCredUsers", + Handler: _RootCoord_ListCredUsers_Handler, + }, + { + MethodName: "GetCredential", + Handler: _RootCoord_GetCredential_Handler, + }, + { + MethodName: "CreateRole", + Handler: _RootCoord_CreateRole_Handler, + }, + { + MethodName: "DropRole", + Handler: _RootCoord_DropRole_Handler, + }, + { + MethodName: "OperateUserRole", + Handler: _RootCoord_OperateUserRole_Handler, + }, + { + MethodName: "SelectRole", + Handler: _RootCoord_SelectRole_Handler, + }, + { + MethodName: "SelectUser", + Handler: _RootCoord_SelectUser_Handler, + }, + { + MethodName: "OperatePrivilege", + Handler: _RootCoord_OperatePrivilege_Handler, + }, + { + MethodName: "SelectGrant", + Handler: _RootCoord_SelectGrant_Handler, + }, + { + MethodName: "ListPolicy", + Handler: _RootCoord_ListPolicy_Handler, + }, + { + MethodName: "BackupRBAC", + Handler: _RootCoord_BackupRBAC_Handler, + }, + { + MethodName: "RestoreRBAC", + Handler: _RootCoord_RestoreRBAC_Handler, + }, + { + MethodName: "CreatePrivilegeGroup", + Handler: _RootCoord_CreatePrivilegeGroup_Handler, + }, + { + MethodName: "DropPrivilegeGroup", + Handler: _RootCoord_DropPrivilegeGroup_Handler, + }, + { + MethodName: "ListPrivilegeGroups", + Handler: _RootCoord_ListPrivilegeGroups_Handler, + }, + { + MethodName: "OperatePrivilegeGroup", + Handler: _RootCoord_OperatePrivilegeGroup_Handler, + }, + { + MethodName: "CheckHealth", + Handler: _RootCoord_CheckHealth_Handler, + }, + { + MethodName: "RenameCollection", + Handler: _RootCoord_RenameCollection_Handler, + }, + { + MethodName: "CreateDatabase", + Handler: _RootCoord_CreateDatabase_Handler, + }, + { + MethodName: "DropDatabase", + Handler: _RootCoord_DropDatabase_Handler, + }, + { + MethodName: "ListDatabases", + Handler: _RootCoord_ListDatabases_Handler, + }, + { + MethodName: "DescribeDatabase", + Handler: _RootCoord_DescribeDatabase_Handler, + }, + { + MethodName: "AlterDatabase", + Handler: _RootCoord_AlterDatabase_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "root_coord.proto", +} diff --git a/internal/proto/segcore.proto b/pkg/proto/segcore.proto similarity index 92% rename from internal/proto/segcore.proto rename to pkg/proto/segcore.proto index 3e419a23f6751..f11ac17bc85a4 100644 --- a/internal/proto/segcore.proto +++ b/pkg/proto/segcore.proto @@ -1,7 +1,7 @@ syntax = "proto3"; package milvus.proto.segcore; -option go_package = "github.com/milvus-io/milvus/internal/proto/segcorepb"; +option go_package = "github.com/milvus-io/milvus/pkg/proto/segcorepb"; import "schema.proto"; import "common.proto"; diff --git a/pkg/proto/segcorepb/segcore.pb.go b/pkg/proto/segcorepb/segcore.pb.go new file mode 100644 index 0000000000000..ef4c536797c02 --- /dev/null +++ b/pkg/proto/segcorepb/segcore.pb.go @@ -0,0 +1,646 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.33.0 +// protoc v3.21.4 +// source: segcore.proto + +package segcorepb + +import ( + commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + 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" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type RetrieveResults struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Ids *schemapb.IDs `protobuf:"bytes,1,opt,name=ids,proto3" json:"ids,omitempty"` + Offset []int64 `protobuf:"varint,2,rep,packed,name=offset,proto3" json:"offset,omitempty"` + FieldsData []*schemapb.FieldData `protobuf:"bytes,3,rep,name=fields_data,json=fieldsData,proto3" json:"fields_data,omitempty"` + AllRetrieveCount int64 `protobuf:"varint,4,opt,name=all_retrieve_count,json=allRetrieveCount,proto3" json:"all_retrieve_count,omitempty"` + HasMoreResult bool `protobuf:"varint,5,opt,name=has_more_result,json=hasMoreResult,proto3" json:"has_more_result,omitempty"` +} + +func (x *RetrieveResults) Reset() { + *x = RetrieveResults{} + if protoimpl.UnsafeEnabled { + mi := &file_segcore_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RetrieveResults) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RetrieveResults) ProtoMessage() {} + +func (x *RetrieveResults) ProtoReflect() protoreflect.Message { + mi := &file_segcore_proto_msgTypes[0] + 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 RetrieveResults.ProtoReflect.Descriptor instead. +func (*RetrieveResults) Descriptor() ([]byte, []int) { + return file_segcore_proto_rawDescGZIP(), []int{0} +} + +func (x *RetrieveResults) GetIds() *schemapb.IDs { + if x != nil { + return x.Ids + } + return nil +} + +func (x *RetrieveResults) GetOffset() []int64 { + if x != nil { + return x.Offset + } + return nil +} + +func (x *RetrieveResults) GetFieldsData() []*schemapb.FieldData { + if x != nil { + return x.FieldsData + } + return nil +} + +func (x *RetrieveResults) GetAllRetrieveCount() int64 { + if x != nil { + return x.AllRetrieveCount + } + return 0 +} + +func (x *RetrieveResults) GetHasMoreResult() bool { + if x != nil { + return x.HasMoreResult + } + return false +} + +type LoadFieldMeta struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + MinTimestamp int64 `protobuf:"varint,1,opt,name=min_timestamp,json=minTimestamp,proto3" json:"min_timestamp,omitempty"` + MaxTimestamp int64 `protobuf:"varint,2,opt,name=max_timestamp,json=maxTimestamp,proto3" json:"max_timestamp,omitempty"` + RowCount int64 `protobuf:"varint,3,opt,name=row_count,json=rowCount,proto3" json:"row_count,omitempty"` +} + +func (x *LoadFieldMeta) Reset() { + *x = LoadFieldMeta{} + if protoimpl.UnsafeEnabled { + mi := &file_segcore_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *LoadFieldMeta) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LoadFieldMeta) ProtoMessage() {} + +func (x *LoadFieldMeta) ProtoReflect() protoreflect.Message { + mi := &file_segcore_proto_msgTypes[1] + 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 LoadFieldMeta.ProtoReflect.Descriptor instead. +func (*LoadFieldMeta) Descriptor() ([]byte, []int) { + return file_segcore_proto_rawDescGZIP(), []int{1} +} + +func (x *LoadFieldMeta) GetMinTimestamp() int64 { + if x != nil { + return x.MinTimestamp + } + return 0 +} + +func (x *LoadFieldMeta) GetMaxTimestamp() int64 { + if x != nil { + return x.MaxTimestamp + } + return 0 +} + +func (x *LoadFieldMeta) GetRowCount() int64 { + if x != nil { + return x.RowCount + } + return 0 +} + +type LoadSegmentMeta struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // TODOs + Metas []*LoadFieldMeta `protobuf:"bytes,1,rep,name=metas,proto3" json:"metas,omitempty"` + TotalSize int64 `protobuf:"varint,2,opt,name=total_size,json=totalSize,proto3" json:"total_size,omitempty"` +} + +func (x *LoadSegmentMeta) Reset() { + *x = LoadSegmentMeta{} + if protoimpl.UnsafeEnabled { + mi := &file_segcore_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *LoadSegmentMeta) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LoadSegmentMeta) ProtoMessage() {} + +func (x *LoadSegmentMeta) ProtoReflect() protoreflect.Message { + mi := &file_segcore_proto_msgTypes[2] + 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 LoadSegmentMeta.ProtoReflect.Descriptor instead. +func (*LoadSegmentMeta) Descriptor() ([]byte, []int) { + return file_segcore_proto_rawDescGZIP(), []int{2} +} + +func (x *LoadSegmentMeta) GetMetas() []*LoadFieldMeta { + if x != nil { + return x.Metas + } + return nil +} + +func (x *LoadSegmentMeta) GetTotalSize() int64 { + if x != nil { + return x.TotalSize + } + return 0 +} + +type InsertRecord struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + FieldsData []*schemapb.FieldData `protobuf:"bytes,1,rep,name=fields_data,json=fieldsData,proto3" json:"fields_data,omitempty"` + NumRows int64 `protobuf:"varint,2,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"` +} + +func (x *InsertRecord) Reset() { + *x = InsertRecord{} + if protoimpl.UnsafeEnabled { + mi := &file_segcore_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *InsertRecord) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*InsertRecord) ProtoMessage() {} + +func (x *InsertRecord) ProtoReflect() protoreflect.Message { + mi := &file_segcore_proto_msgTypes[3] + 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 InsertRecord.ProtoReflect.Descriptor instead. +func (*InsertRecord) Descriptor() ([]byte, []int) { + return file_segcore_proto_rawDescGZIP(), []int{3} +} + +func (x *InsertRecord) GetFieldsData() []*schemapb.FieldData { + if x != nil { + return x.FieldsData + } + return nil +} + +func (x *InsertRecord) GetNumRows() int64 { + if x != nil { + return x.NumRows + } + return 0 +} + +type FieldIndexMeta struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + FieldID int64 `protobuf:"varint,1,opt,name=fieldID,proto3" json:"fieldID,omitempty"` + CollectionID int64 `protobuf:"varint,2,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + IndexName string `protobuf:"bytes,3,opt,name=index_name,json=indexName,proto3" json:"index_name,omitempty"` + TypeParams []*commonpb.KeyValuePair `protobuf:"bytes,4,rep,name=type_params,json=typeParams,proto3" json:"type_params,omitempty"` + IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,5,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"` + IsAutoIndex bool `protobuf:"varint,6,opt,name=is_auto_index,json=isAutoIndex,proto3" json:"is_auto_index,omitempty"` + UserIndexParams []*commonpb.KeyValuePair `protobuf:"bytes,7,rep,name=user_index_params,json=userIndexParams,proto3" json:"user_index_params,omitempty"` +} + +func (x *FieldIndexMeta) Reset() { + *x = FieldIndexMeta{} + if protoimpl.UnsafeEnabled { + mi := &file_segcore_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *FieldIndexMeta) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FieldIndexMeta) ProtoMessage() {} + +func (x *FieldIndexMeta) ProtoReflect() protoreflect.Message { + mi := &file_segcore_proto_msgTypes[4] + 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 FieldIndexMeta.ProtoReflect.Descriptor instead. +func (*FieldIndexMeta) Descriptor() ([]byte, []int) { + return file_segcore_proto_rawDescGZIP(), []int{4} +} + +func (x *FieldIndexMeta) GetFieldID() int64 { + if x != nil { + return x.FieldID + } + return 0 +} + +func (x *FieldIndexMeta) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *FieldIndexMeta) GetIndexName() string { + if x != nil { + return x.IndexName + } + return "" +} + +func (x *FieldIndexMeta) GetTypeParams() []*commonpb.KeyValuePair { + if x != nil { + return x.TypeParams + } + return nil +} + +func (x *FieldIndexMeta) GetIndexParams() []*commonpb.KeyValuePair { + if x != nil { + return x.IndexParams + } + return nil +} + +func (x *FieldIndexMeta) GetIsAutoIndex() bool { + if x != nil { + return x.IsAutoIndex + } + return false +} + +func (x *FieldIndexMeta) GetUserIndexParams() []*commonpb.KeyValuePair { + if x != nil { + return x.UserIndexParams + } + return nil +} + +type CollectionIndexMeta struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + MaxIndexRowCount int64 `protobuf:"varint,1,opt,name=maxIndexRowCount,proto3" json:"maxIndexRowCount,omitempty"` + IndexMetas []*FieldIndexMeta `protobuf:"bytes,2,rep,name=index_metas,json=indexMetas,proto3" json:"index_metas,omitempty"` +} + +func (x *CollectionIndexMeta) Reset() { + *x = CollectionIndexMeta{} + if protoimpl.UnsafeEnabled { + mi := &file_segcore_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CollectionIndexMeta) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CollectionIndexMeta) ProtoMessage() {} + +func (x *CollectionIndexMeta) ProtoReflect() protoreflect.Message { + mi := &file_segcore_proto_msgTypes[5] + 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 CollectionIndexMeta.ProtoReflect.Descriptor instead. +func (*CollectionIndexMeta) Descriptor() ([]byte, []int) { + return file_segcore_proto_rawDescGZIP(), []int{5} +} + +func (x *CollectionIndexMeta) GetMaxIndexRowCount() int64 { + if x != nil { + return x.MaxIndexRowCount + } + return 0 +} + +func (x *CollectionIndexMeta) GetIndexMetas() []*FieldIndexMeta { + if x != nil { + return x.IndexMetas + } + return nil +} + +var File_segcore_proto protoreflect.FileDescriptor + +var file_segcore_proto_rawDesc = []byte{ + 0x0a, 0x0d, 0x73, 0x65, 0x67, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, + 0x14, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x65, + 0x67, 0x63, 0x6f, 0x72, 0x65, 0x1a, 0x0c, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x1a, 0x0c, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x22, 0xec, 0x01, 0x0a, 0x0f, 0x52, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x65, 0x52, 0x65, + 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x2a, 0x0a, 0x03, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x49, 0x44, 0x73, 0x52, 0x03, 0x69, 0x64, + 0x73, 0x12, 0x16, 0x0a, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x02, 0x20, 0x03, 0x28, + 0x03, 0x52, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x3f, 0x0a, 0x0b, 0x66, 0x69, 0x65, + 0x6c, 0x64, 0x73, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x44, 0x61, 0x74, 0x61, 0x52, 0x0a, + 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x44, 0x61, 0x74, 0x61, 0x12, 0x2c, 0x0a, 0x12, 0x61, 0x6c, + 0x6c, 0x5f, 0x72, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x65, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x61, 0x6c, 0x6c, 0x52, 0x65, 0x74, 0x72, 0x69, + 0x65, 0x76, 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x26, 0x0a, 0x0f, 0x68, 0x61, 0x73, 0x5f, + 0x6d, 0x6f, 0x72, 0x65, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x0d, 0x68, 0x61, 0x73, 0x4d, 0x6f, 0x72, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, + 0x22, 0x76, 0x0a, 0x0d, 0x4c, 0x6f, 0x61, 0x64, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x4d, 0x65, 0x74, + 0x61, 0x12, 0x23, 0x0a, 0x0d, 0x6d, 0x69, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, + 0x6d, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x6d, 0x69, 0x6e, 0x54, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x23, 0x0a, 0x0d, 0x6d, 0x61, 0x78, 0x5f, 0x74, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x6d, + 0x61, 0x78, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x1b, 0x0a, 0x09, 0x72, + 0x6f, 0x77, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, + 0x72, 0x6f, 0x77, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x6b, 0x0a, 0x0f, 0x4c, 0x6f, 0x61, 0x64, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x39, 0x0a, 0x05, 0x6d, + 0x65, 0x74, 0x61, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x65, 0x67, 0x63, 0x6f, 0x72, + 0x65, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x4d, 0x65, 0x74, 0x61, 0x52, + 0x05, 0x6d, 0x65, 0x74, 0x61, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, + 0x73, 0x69, 0x7a, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x6f, 0x74, 0x61, + 0x6c, 0x53, 0x69, 0x7a, 0x65, 0x22, 0x6a, 0x0a, 0x0c, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x52, + 0x65, 0x63, 0x6f, 0x72, 0x64, 0x12, 0x3f, 0x0a, 0x0b, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x5f, + 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x44, 0x61, 0x74, 0x61, 0x52, 0x0a, 0x66, 0x69, 0x65, 0x6c, + 0x64, 0x73, 0x44, 0x61, 0x74, 0x61, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, + 0x77, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, + 0x73, 0x22, 0xea, 0x02, 0x0a, 0x0e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, + 0x4d, 0x65, 0x74, 0x61, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 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, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, + 0x65, 0x12, 0x42, 0x0a, 0x0b, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, + 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, + 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0a, 0x74, 0x79, 0x70, 0x65, 0x50, + 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x44, 0x0a, 0x0c, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x70, + 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0b, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x22, 0x0a, 0x0d, 0x69, + 0x73, 0x5f, 0x61, 0x75, 0x74, 0x6f, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0b, 0x69, 0x73, 0x41, 0x75, 0x74, 0x6f, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, + 0x4d, 0x0a, 0x11, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x70, 0x61, + 0x72, 0x61, 0x6d, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0f, 0x75, + 0x73, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x22, 0x88, + 0x01, 0x0a, 0x13, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x64, + 0x65, 0x78, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x2a, 0x0a, 0x10, 0x6d, 0x61, 0x78, 0x49, 0x6e, 0x64, + 0x65, 0x78, 0x52, 0x6f, 0x77, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x10, 0x6d, 0x61, 0x78, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x6f, 0x77, 0x43, 0x6f, 0x75, + 0x6e, 0x74, 0x12, 0x45, 0x0a, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6d, 0x65, 0x74, 0x61, + 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x65, 0x67, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x46, + 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x0a, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x4d, 0x65, 0x74, 0x61, 0x73, 0x42, 0x31, 0x5a, 0x2f, 0x67, 0x69, 0x74, + 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, + 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2f, 0x73, 0x65, 0x67, 0x63, 0x6f, 0x72, 0x65, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_segcore_proto_rawDescOnce sync.Once + file_segcore_proto_rawDescData = file_segcore_proto_rawDesc +) + +func file_segcore_proto_rawDescGZIP() []byte { + file_segcore_proto_rawDescOnce.Do(func() { + file_segcore_proto_rawDescData = protoimpl.X.CompressGZIP(file_segcore_proto_rawDescData) + }) + return file_segcore_proto_rawDescData +} + +var file_segcore_proto_msgTypes = make([]protoimpl.MessageInfo, 6) +var file_segcore_proto_goTypes = []interface{}{ + (*RetrieveResults)(nil), // 0: milvus.proto.segcore.RetrieveResults + (*LoadFieldMeta)(nil), // 1: milvus.proto.segcore.LoadFieldMeta + (*LoadSegmentMeta)(nil), // 2: milvus.proto.segcore.LoadSegmentMeta + (*InsertRecord)(nil), // 3: milvus.proto.segcore.InsertRecord + (*FieldIndexMeta)(nil), // 4: milvus.proto.segcore.FieldIndexMeta + (*CollectionIndexMeta)(nil), // 5: milvus.proto.segcore.CollectionIndexMeta + (*schemapb.IDs)(nil), // 6: milvus.proto.schema.IDs + (*schemapb.FieldData)(nil), // 7: milvus.proto.schema.FieldData + (*commonpb.KeyValuePair)(nil), // 8: milvus.proto.common.KeyValuePair +} +var file_segcore_proto_depIdxs = []int32{ + 6, // 0: milvus.proto.segcore.RetrieveResults.ids:type_name -> milvus.proto.schema.IDs + 7, // 1: milvus.proto.segcore.RetrieveResults.fields_data:type_name -> milvus.proto.schema.FieldData + 1, // 2: milvus.proto.segcore.LoadSegmentMeta.metas:type_name -> milvus.proto.segcore.LoadFieldMeta + 7, // 3: milvus.proto.segcore.InsertRecord.fields_data:type_name -> milvus.proto.schema.FieldData + 8, // 4: milvus.proto.segcore.FieldIndexMeta.type_params:type_name -> milvus.proto.common.KeyValuePair + 8, // 5: milvus.proto.segcore.FieldIndexMeta.index_params:type_name -> milvus.proto.common.KeyValuePair + 8, // 6: milvus.proto.segcore.FieldIndexMeta.user_index_params:type_name -> milvus.proto.common.KeyValuePair + 4, // 7: milvus.proto.segcore.CollectionIndexMeta.index_metas:type_name -> milvus.proto.segcore.FieldIndexMeta + 8, // [8:8] is the sub-list for method output_type + 8, // [8:8] is the sub-list for method input_type + 8, // [8:8] is the sub-list for extension type_name + 8, // [8:8] is the sub-list for extension extendee + 0, // [0:8] is the sub-list for field type_name +} + +func init() { file_segcore_proto_init() } +func file_segcore_proto_init() { + if File_segcore_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_segcore_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RetrieveResults); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_segcore_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*LoadFieldMeta); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_segcore_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*LoadSegmentMeta); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_segcore_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*InsertRecord); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_segcore_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*FieldIndexMeta); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_segcore_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CollectionIndexMeta); 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{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_segcore_proto_rawDesc, + NumEnums: 0, + NumMessages: 6, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_segcore_proto_goTypes, + DependencyIndexes: file_segcore_proto_depIdxs, + MessageInfos: file_segcore_proto_msgTypes, + }.Build() + File_segcore_proto = out.File + file_segcore_proto_rawDesc = nil + file_segcore_proto_goTypes = nil + file_segcore_proto_depIdxs = nil +} diff --git a/pkg/streaming/proto/streaming.proto b/pkg/proto/streaming.proto similarity index 99% rename from pkg/streaming/proto/streaming.proto rename to pkg/proto/streaming.proto index 0a7debc9dad5c..4539bf34395ce 100644 --- a/pkg/streaming/proto/streaming.proto +++ b/pkg/proto/streaming.proto @@ -2,7 +2,7 @@ syntax = "proto3"; package milvus.proto.streaming; -option go_package = "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"; +option go_package = "github.com/milvus-io/milvus/pkg/proto/streamingpb"; import "messages.proto"; import "milvus.proto"; diff --git a/pkg/streaming/proto/streamingpb/extends.go b/pkg/proto/streamingpb/extends.go similarity index 100% rename from pkg/streaming/proto/streamingpb/extends.go rename to pkg/proto/streamingpb/extends.go diff --git a/pkg/proto/streamingpb/streaming.pb.go b/pkg/proto/streamingpb/streaming.pb.go new file mode 100644 index 0000000000000..850806e5d41bb --- /dev/null +++ b/pkg/proto/streamingpb/streaming.pb.go @@ -0,0 +1,4932 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.33.0 +// protoc v3.21.4 +// source: streaming.proto + +package streamingpb + +import ( + milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + messagespb "github.com/milvus-io/milvus/pkg/proto/messagespb" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + anypb "google.golang.org/protobuf/types/known/anypb" + emptypb "google.golang.org/protobuf/types/known/emptypb" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// PChannelMetaState +type PChannelMetaState int32 + +const ( + PChannelMetaState_PCHANNEL_META_STATE_UNKNOWN PChannelMetaState = 0 // should never used. + PChannelMetaState_PCHANNEL_META_STATE_UNINITIALIZED PChannelMetaState = 1 // channel is uninitialized, never assgined to any streaming node. + PChannelMetaState_PCHANNEL_META_STATE_ASSIGNING PChannelMetaState = 2 // new term is allocated, but not determined to be assgined. + PChannelMetaState_PCHANNEL_META_STATE_ASSIGNED PChannelMetaState = 3 // channel is assigned to a streaming node. + PChannelMetaState_PCHANNEL_META_STATE_UNAVAILABLE PChannelMetaState = 4 // channel is unavailable at this term. +) + +// Enum value maps for PChannelMetaState. +var ( + PChannelMetaState_name = map[int32]string{ + 0: "PCHANNEL_META_STATE_UNKNOWN", + 1: "PCHANNEL_META_STATE_UNINITIALIZED", + 2: "PCHANNEL_META_STATE_ASSIGNING", + 3: "PCHANNEL_META_STATE_ASSIGNED", + 4: "PCHANNEL_META_STATE_UNAVAILABLE", + } + PChannelMetaState_value = map[string]int32{ + "PCHANNEL_META_STATE_UNKNOWN": 0, + "PCHANNEL_META_STATE_UNINITIALIZED": 1, + "PCHANNEL_META_STATE_ASSIGNING": 2, + "PCHANNEL_META_STATE_ASSIGNED": 3, + "PCHANNEL_META_STATE_UNAVAILABLE": 4, + } +) + +func (x PChannelMetaState) Enum() *PChannelMetaState { + p := new(PChannelMetaState) + *p = x + return p +} + +func (x PChannelMetaState) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (PChannelMetaState) Descriptor() protoreflect.EnumDescriptor { + return file_streaming_proto_enumTypes[0].Descriptor() +} + +func (PChannelMetaState) Type() protoreflect.EnumType { + return &file_streaming_proto_enumTypes[0] +} + +func (x PChannelMetaState) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use PChannelMetaState.Descriptor instead. +func (PChannelMetaState) EnumDescriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{0} +} + +// BroadcastTaskState is the state of the broadcast task. +type BroadcastTaskState int32 + +const ( + BroadcastTaskState_BROADCAST_TASK_STATE_UNKNOWN BroadcastTaskState = 0 // should never used. + BroadcastTaskState_BROADCAST_TASK_STATE_PENDING BroadcastTaskState = 1 // task is pending. + BroadcastTaskState_BROADCAST_TASK_STATE_DONE BroadcastTaskState = 2 // task is done, the message is broadcasted, and the persisted task can be cleared. +) + +// Enum value maps for BroadcastTaskState. +var ( + BroadcastTaskState_name = map[int32]string{ + 0: "BROADCAST_TASK_STATE_UNKNOWN", + 1: "BROADCAST_TASK_STATE_PENDING", + 2: "BROADCAST_TASK_STATE_DONE", + } + BroadcastTaskState_value = map[string]int32{ + "BROADCAST_TASK_STATE_UNKNOWN": 0, + "BROADCAST_TASK_STATE_PENDING": 1, + "BROADCAST_TASK_STATE_DONE": 2, + } +) + +func (x BroadcastTaskState) Enum() *BroadcastTaskState { + p := new(BroadcastTaskState) + *p = x + return p +} + +func (x BroadcastTaskState) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (BroadcastTaskState) Descriptor() protoreflect.EnumDescriptor { + return file_streaming_proto_enumTypes[1].Descriptor() +} + +func (BroadcastTaskState) Type() protoreflect.EnumType { + return &file_streaming_proto_enumTypes[1] +} + +func (x BroadcastTaskState) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use BroadcastTaskState.Descriptor instead. +func (BroadcastTaskState) EnumDescriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{1} +} + +// StreamingCode is the error code for log internal component. +type StreamingCode int32 + +const ( + StreamingCode_STREAMING_CODE_OK StreamingCode = 0 + StreamingCode_STREAMING_CODE_CHANNEL_NOT_EXIST StreamingCode = 1 // channel not exist + StreamingCode_STREAMING_CODE_CHANNEL_FENCED StreamingCode = 2 // channel is fenced + StreamingCode_STREAMING_CODE_ON_SHUTDOWN StreamingCode = 3 // component is on shutdown + StreamingCode_STREAMING_CODE_INVALID_REQUEST_SEQ StreamingCode = 4 // invalid request sequence + StreamingCode_STREAMING_CODE_UNMATCHED_CHANNEL_TERM StreamingCode = 5 // unmatched channel term + StreamingCode_STREAMING_CODE_IGNORED_OPERATION StreamingCode = 6 // ignored operation + StreamingCode_STREAMING_CODE_INNER StreamingCode = 7 // underlying service failure. + StreamingCode_STREAMING_CODE_INVAILD_ARGUMENT StreamingCode = 8 // invalid argument + StreamingCode_STREAMING_CODE_TRANSACTION_EXPIRED StreamingCode = 9 // transaction expired + StreamingCode_STREAMING_CODE_INVALID_TRANSACTION_STATE StreamingCode = 10 // invalid transaction state + StreamingCode_STREAMING_CODE_UNRECOVERABLE StreamingCode = 11 // unrecoverable error + StreamingCode_STREAMING_CODE_UNKNOWN StreamingCode = 999 // unknown error +) + +// Enum value maps for StreamingCode. +var ( + StreamingCode_name = map[int32]string{ + 0: "STREAMING_CODE_OK", + 1: "STREAMING_CODE_CHANNEL_NOT_EXIST", + 2: "STREAMING_CODE_CHANNEL_FENCED", + 3: "STREAMING_CODE_ON_SHUTDOWN", + 4: "STREAMING_CODE_INVALID_REQUEST_SEQ", + 5: "STREAMING_CODE_UNMATCHED_CHANNEL_TERM", + 6: "STREAMING_CODE_IGNORED_OPERATION", + 7: "STREAMING_CODE_INNER", + 8: "STREAMING_CODE_INVAILD_ARGUMENT", + 9: "STREAMING_CODE_TRANSACTION_EXPIRED", + 10: "STREAMING_CODE_INVALID_TRANSACTION_STATE", + 11: "STREAMING_CODE_UNRECOVERABLE", + 999: "STREAMING_CODE_UNKNOWN", + } + StreamingCode_value = map[string]int32{ + "STREAMING_CODE_OK": 0, + "STREAMING_CODE_CHANNEL_NOT_EXIST": 1, + "STREAMING_CODE_CHANNEL_FENCED": 2, + "STREAMING_CODE_ON_SHUTDOWN": 3, + "STREAMING_CODE_INVALID_REQUEST_SEQ": 4, + "STREAMING_CODE_UNMATCHED_CHANNEL_TERM": 5, + "STREAMING_CODE_IGNORED_OPERATION": 6, + "STREAMING_CODE_INNER": 7, + "STREAMING_CODE_INVAILD_ARGUMENT": 8, + "STREAMING_CODE_TRANSACTION_EXPIRED": 9, + "STREAMING_CODE_INVALID_TRANSACTION_STATE": 10, + "STREAMING_CODE_UNRECOVERABLE": 11, + "STREAMING_CODE_UNKNOWN": 999, + } +) + +func (x StreamingCode) Enum() *StreamingCode { + p := new(StreamingCode) + *p = x + return p +} + +func (x StreamingCode) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (StreamingCode) Descriptor() protoreflect.EnumDescriptor { + return file_streaming_proto_enumTypes[2].Descriptor() +} + +func (StreamingCode) Type() protoreflect.EnumType { + return &file_streaming_proto_enumTypes[2] +} + +func (x StreamingCode) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use StreamingCode.Descriptor instead. +func (StreamingCode) EnumDescriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{2} +} + +// SegmentAssignmentState is the state of segment assignment. +// The state machine can be described as following: +// 1. PENDING -> GROWING -> SEALED -> FLUSHED +type SegmentAssignmentState int32 + +const ( + SegmentAssignmentState_SEGMENT_ASSIGNMENT_STATE_UNKNOWN SegmentAssignmentState = 0 // should never used. + SegmentAssignmentState_SEGMENT_ASSIGNMENT_STATE_PENDING SegmentAssignmentState = 1 + SegmentAssignmentState_SEGMENT_ASSIGNMENT_STATE_GROWING SegmentAssignmentState = 2 + SegmentAssignmentState_SEGMENT_ASSIGNMENT_STATE_SEALED SegmentAssignmentState = 3 + SegmentAssignmentState_SEGMENT_ASSIGNMENT_STATE_FLUSHED SegmentAssignmentState = 4 // can never be seen, because it's +) + +// Enum value maps for SegmentAssignmentState. +var ( + SegmentAssignmentState_name = map[int32]string{ + 0: "SEGMENT_ASSIGNMENT_STATE_UNKNOWN", + 1: "SEGMENT_ASSIGNMENT_STATE_PENDING", + 2: "SEGMENT_ASSIGNMENT_STATE_GROWING", + 3: "SEGMENT_ASSIGNMENT_STATE_SEALED", + 4: "SEGMENT_ASSIGNMENT_STATE_FLUSHED", + } + SegmentAssignmentState_value = map[string]int32{ + "SEGMENT_ASSIGNMENT_STATE_UNKNOWN": 0, + "SEGMENT_ASSIGNMENT_STATE_PENDING": 1, + "SEGMENT_ASSIGNMENT_STATE_GROWING": 2, + "SEGMENT_ASSIGNMENT_STATE_SEALED": 3, + "SEGMENT_ASSIGNMENT_STATE_FLUSHED": 4, + } +) + +func (x SegmentAssignmentState) Enum() *SegmentAssignmentState { + p := new(SegmentAssignmentState) + *p = x + return p +} + +func (x SegmentAssignmentState) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SegmentAssignmentState) Descriptor() protoreflect.EnumDescriptor { + return file_streaming_proto_enumTypes[3].Descriptor() +} + +func (SegmentAssignmentState) Type() protoreflect.EnumType { + return &file_streaming_proto_enumTypes[3] +} + +func (x SegmentAssignmentState) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use SegmentAssignmentState.Descriptor instead. +func (SegmentAssignmentState) EnumDescriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{3} +} + +// PChannelInfo is the information of a pchannel info, should only keep the +// basic info of a pchannel. It's used in many rpc and meta, so keep it simple. +type PChannelInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` // channel name + Term int64 `protobuf:"varint,2,opt,name=term,proto3" json:"term,omitempty"` // A monotonic increasing term, every time the channel is +} + +func (x *PChannelInfo) Reset() { + *x = PChannelInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PChannelInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PChannelInfo) ProtoMessage() {} + +func (x *PChannelInfo) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[0] + 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 PChannelInfo.ProtoReflect.Descriptor instead. +func (*PChannelInfo) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{0} +} + +func (x *PChannelInfo) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *PChannelInfo) GetTerm() int64 { + if x != nil { + return x.Term + } + return 0 +} + +// PChannelAssignmentLog is the log of meta information of a pchannel, should +// only keep the data that is necessary to persistent. +type PChannelAssignmentLog struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Term int64 `protobuf:"varint,1,opt,name=term,proto3" json:"term,omitempty"` // term when server assigned. + Node *StreamingNodeInfo `protobuf:"bytes,2,opt,name=node,proto3" json:"node,omitempty"` // streaming node that the channel is assigned to. +} + +func (x *PChannelAssignmentLog) Reset() { + *x = PChannelAssignmentLog{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PChannelAssignmentLog) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PChannelAssignmentLog) ProtoMessage() {} + +func (x *PChannelAssignmentLog) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[1] + 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 PChannelAssignmentLog.ProtoReflect.Descriptor instead. +func (*PChannelAssignmentLog) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{1} +} + +func (x *PChannelAssignmentLog) GetTerm() int64 { + if x != nil { + return x.Term + } + return 0 +} + +func (x *PChannelAssignmentLog) GetNode() *StreamingNodeInfo { + if x != nil { + return x.Node + } + return nil +} + +// PChannelMeta is the meta information of a pchannel, should only keep the data +// that is necessary to persistent. It's only used in meta, so do not use it in +// rpc. +type PChannelMeta struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Channel *PChannelInfo `protobuf:"bytes,1,opt,name=channel,proto3" json:"channel,omitempty"` // keep the meta info that current assigned to. + Node *StreamingNodeInfo `protobuf:"bytes,2,opt,name=node,proto3" json:"node,omitempty"` // nil if channel is not uninitialized. + State PChannelMetaState `protobuf:"varint,3,opt,name=state,proto3,enum=milvus.proto.streaming.PChannelMetaState" json:"state,omitempty"` // state of the channel. + Histories []*PChannelAssignmentLog `protobuf:"bytes,4,rep,name=histories,proto3" json:"histories,omitempty"` // keep the meta info assignment log that used to be assigned to. +} + +func (x *PChannelMeta) Reset() { + *x = PChannelMeta{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PChannelMeta) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PChannelMeta) ProtoMessage() {} + +func (x *PChannelMeta) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[2] + 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 PChannelMeta.ProtoReflect.Descriptor instead. +func (*PChannelMeta) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{2} +} + +func (x *PChannelMeta) GetChannel() *PChannelInfo { + if x != nil { + return x.Channel + } + return nil +} + +func (x *PChannelMeta) GetNode() *StreamingNodeInfo { + if x != nil { + return x.Node + } + return nil +} + +func (x *PChannelMeta) GetState() PChannelMetaState { + if x != nil { + return x.State + } + return PChannelMetaState_PCHANNEL_META_STATE_UNKNOWN +} + +func (x *PChannelMeta) GetHistories() []*PChannelAssignmentLog { + if x != nil { + return x.Histories + } + return nil +} + +// VersionPair is the version pair of global and local. +type VersionPair struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Global int64 `protobuf:"varint,1,opt,name=global,proto3" json:"global,omitempty"` + Local int64 `protobuf:"varint,2,opt,name=local,proto3" json:"local,omitempty"` +} + +func (x *VersionPair) Reset() { + *x = VersionPair{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *VersionPair) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*VersionPair) ProtoMessage() {} + +func (x *VersionPair) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[3] + 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 VersionPair.ProtoReflect.Descriptor instead. +func (*VersionPair) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{3} +} + +func (x *VersionPair) GetGlobal() int64 { + if x != nil { + return x.Global + } + return 0 +} + +func (x *VersionPair) GetLocal() int64 { + if x != nil { + return x.Local + } + return 0 +} + +// BroadcastTask is the task to broadcast the message. +type BroadcastTask struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + TaskId int64 `protobuf:"varint,1,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"` // task id. + Message *messagespb.Message `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"` // message to be broadcast. + State BroadcastTaskState `protobuf:"varint,3,opt,name=state,proto3,enum=milvus.proto.streaming.BroadcastTaskState" json:"state,omitempty"` // state of the task. +} + +func (x *BroadcastTask) Reset() { + *x = BroadcastTask{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BroadcastTask) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BroadcastTask) ProtoMessage() {} + +func (x *BroadcastTask) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[4] + 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 BroadcastTask.ProtoReflect.Descriptor instead. +func (*BroadcastTask) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{4} +} + +func (x *BroadcastTask) GetTaskId() int64 { + if x != nil { + return x.TaskId + } + return 0 +} + +func (x *BroadcastTask) GetMessage() *messagespb.Message { + if x != nil { + return x.Message + } + return nil +} + +func (x *BroadcastTask) GetState() BroadcastTaskState { + if x != nil { + return x.State + } + return BroadcastTaskState_BROADCAST_TASK_STATE_UNKNOWN +} + +// BroadcastRequest is the request of the Broadcast RPC. +type BroadcastRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Message *messagespb.Message `protobuf:"bytes,1,opt,name=message,proto3" json:"message,omitempty"` // message to be broadcast. +} + +func (x *BroadcastRequest) Reset() { + *x = BroadcastRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BroadcastRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BroadcastRequest) ProtoMessage() {} + +func (x *BroadcastRequest) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[5] + 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 BroadcastRequest.ProtoReflect.Descriptor instead. +func (*BroadcastRequest) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{5} +} + +func (x *BroadcastRequest) GetMessage() *messagespb.Message { + if x != nil { + return x.Message + } + return nil +} + +// BroadcastResponse is the response of the Broadcast RPC. +type BroadcastResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Results map[string]*ProduceMessageResponseResult `protobuf:"bytes,1,rep,name=results,proto3" json:"results,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *BroadcastResponse) Reset() { + *x = BroadcastResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BroadcastResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BroadcastResponse) ProtoMessage() {} + +func (x *BroadcastResponse) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[6] + 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 BroadcastResponse.ProtoReflect.Descriptor instead. +func (*BroadcastResponse) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{6} +} + +func (x *BroadcastResponse) GetResults() map[string]*ProduceMessageResponseResult { + if x != nil { + return x.Results + } + return nil +} + +// AssignmentDiscoverRequest is the request of Discovery +type AssignmentDiscoverRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Command: + // + // *AssignmentDiscoverRequest_ReportError + // *AssignmentDiscoverRequest_Close + Command isAssignmentDiscoverRequest_Command `protobuf_oneof:"command"` +} + +func (x *AssignmentDiscoverRequest) Reset() { + *x = AssignmentDiscoverRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AssignmentDiscoverRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AssignmentDiscoverRequest) ProtoMessage() {} + +func (x *AssignmentDiscoverRequest) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[7] + 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 AssignmentDiscoverRequest.ProtoReflect.Descriptor instead. +func (*AssignmentDiscoverRequest) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{7} +} + +func (m *AssignmentDiscoverRequest) GetCommand() isAssignmentDiscoverRequest_Command { + if m != nil { + return m.Command + } + return nil +} + +func (x *AssignmentDiscoverRequest) GetReportError() *ReportAssignmentErrorRequest { + if x, ok := x.GetCommand().(*AssignmentDiscoverRequest_ReportError); ok { + return x.ReportError + } + return nil +} + +func (x *AssignmentDiscoverRequest) GetClose() *CloseAssignmentDiscoverRequest { + if x, ok := x.GetCommand().(*AssignmentDiscoverRequest_Close); ok { + return x.Close + } + return nil +} + +type isAssignmentDiscoverRequest_Command interface { + isAssignmentDiscoverRequest_Command() +} + +type AssignmentDiscoverRequest_ReportError struct { + ReportError *ReportAssignmentErrorRequest `protobuf:"bytes,1,opt,name=report_error,json=reportError,proto3,oneof"` // report streaming error, trigger reassign right now. +} + +type AssignmentDiscoverRequest_Close struct { + Close *CloseAssignmentDiscoverRequest `protobuf:"bytes,2,opt,name=close,proto3,oneof"` // close the stream. +} + +func (*AssignmentDiscoverRequest_ReportError) isAssignmentDiscoverRequest_Command() {} + +func (*AssignmentDiscoverRequest_Close) isAssignmentDiscoverRequest_Command() {} + +// ReportAssignmentErrorRequest is the request to report assignment error +// happens. +type ReportAssignmentErrorRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Pchannel *PChannelInfo `protobuf:"bytes,1,opt,name=pchannel,proto3" json:"pchannel,omitempty"` // channel + Err *StreamingError `protobuf:"bytes,2,opt,name=err,proto3" json:"err,omitempty"` // error happend on log node +} + +func (x *ReportAssignmentErrorRequest) Reset() { + *x = ReportAssignmentErrorRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ReportAssignmentErrorRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ReportAssignmentErrorRequest) ProtoMessage() {} + +func (x *ReportAssignmentErrorRequest) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[8] + 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 ReportAssignmentErrorRequest.ProtoReflect.Descriptor instead. +func (*ReportAssignmentErrorRequest) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{8} +} + +func (x *ReportAssignmentErrorRequest) GetPchannel() *PChannelInfo { + if x != nil { + return x.Pchannel + } + return nil +} + +func (x *ReportAssignmentErrorRequest) GetErr() *StreamingError { + if x != nil { + return x.Err + } + return nil +} + +// CloseAssignmentDiscoverRequest is the request to close the stream. +type CloseAssignmentDiscoverRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *CloseAssignmentDiscoverRequest) Reset() { + *x = CloseAssignmentDiscoverRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CloseAssignmentDiscoverRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CloseAssignmentDiscoverRequest) ProtoMessage() {} + +func (x *CloseAssignmentDiscoverRequest) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[9] + 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 CloseAssignmentDiscoverRequest.ProtoReflect.Descriptor instead. +func (*CloseAssignmentDiscoverRequest) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{9} +} + +// AssignmentDiscoverResponse is the response of Discovery +type AssignmentDiscoverResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Response: + // + // *AssignmentDiscoverResponse_FullAssignment + // *AssignmentDiscoverResponse_Close + Response isAssignmentDiscoverResponse_Response `protobuf_oneof:"response"` +} + +func (x *AssignmentDiscoverResponse) Reset() { + *x = AssignmentDiscoverResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AssignmentDiscoverResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AssignmentDiscoverResponse) ProtoMessage() {} + +func (x *AssignmentDiscoverResponse) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[10] + 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 AssignmentDiscoverResponse.ProtoReflect.Descriptor instead. +func (*AssignmentDiscoverResponse) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{10} +} + +func (m *AssignmentDiscoverResponse) GetResponse() isAssignmentDiscoverResponse_Response { + if m != nil { + return m.Response + } + return nil +} + +func (x *AssignmentDiscoverResponse) GetFullAssignment() *FullStreamingNodeAssignmentWithVersion { + if x, ok := x.GetResponse().(*AssignmentDiscoverResponse_FullAssignment); ok { + return x.FullAssignment + } + return nil +} + +func (x *AssignmentDiscoverResponse) GetClose() *CloseAssignmentDiscoverResponse { + if x, ok := x.GetResponse().(*AssignmentDiscoverResponse_Close); ok { + return x.Close + } + return nil +} + +type isAssignmentDiscoverResponse_Response interface { + isAssignmentDiscoverResponse_Response() +} + +type AssignmentDiscoverResponse_FullAssignment struct { + FullAssignment *FullStreamingNodeAssignmentWithVersion `protobuf:"bytes,1,opt,name=full_assignment,json=fullAssignment,proto3,oneof"` // all assignment info. +} + +type AssignmentDiscoverResponse_Close struct { + // TODO: may be support partial assignment info in future. + Close *CloseAssignmentDiscoverResponse `protobuf:"bytes,2,opt,name=close,proto3,oneof"` +} + +func (*AssignmentDiscoverResponse_FullAssignment) isAssignmentDiscoverResponse_Response() {} + +func (*AssignmentDiscoverResponse_Close) isAssignmentDiscoverResponse_Response() {} + +// FullStreamingNodeAssignmentWithVersion is the full assignment info of a log +// node with version. +type FullStreamingNodeAssignmentWithVersion struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Version *VersionPair `protobuf:"bytes,1,opt,name=version,proto3" json:"version,omitempty"` + Assignments []*StreamingNodeAssignment `protobuf:"bytes,2,rep,name=assignments,proto3" json:"assignments,omitempty"` +} + +func (x *FullStreamingNodeAssignmentWithVersion) Reset() { + *x = FullStreamingNodeAssignmentWithVersion{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *FullStreamingNodeAssignmentWithVersion) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FullStreamingNodeAssignmentWithVersion) ProtoMessage() {} + +func (x *FullStreamingNodeAssignmentWithVersion) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[11] + 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 FullStreamingNodeAssignmentWithVersion.ProtoReflect.Descriptor instead. +func (*FullStreamingNodeAssignmentWithVersion) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{11} +} + +func (x *FullStreamingNodeAssignmentWithVersion) GetVersion() *VersionPair { + if x != nil { + return x.Version + } + return nil +} + +func (x *FullStreamingNodeAssignmentWithVersion) GetAssignments() []*StreamingNodeAssignment { + if x != nil { + return x.Assignments + } + return nil +} + +type CloseAssignmentDiscoverResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *CloseAssignmentDiscoverResponse) Reset() { + *x = CloseAssignmentDiscoverResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CloseAssignmentDiscoverResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CloseAssignmentDiscoverResponse) ProtoMessage() {} + +func (x *CloseAssignmentDiscoverResponse) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[12] + 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 CloseAssignmentDiscoverResponse.ProtoReflect.Descriptor instead. +func (*CloseAssignmentDiscoverResponse) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{12} +} + +// StreamingNodeInfo is the information of a streaming node. +type StreamingNodeInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ServerId int64 `protobuf:"varint,1,opt,name=server_id,json=serverId,proto3" json:"server_id,omitempty"` + Address string `protobuf:"bytes,2,opt,name=address,proto3" json:"address,omitempty"` +} + +func (x *StreamingNodeInfo) Reset() { + *x = StreamingNodeInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StreamingNodeInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StreamingNodeInfo) ProtoMessage() {} + +func (x *StreamingNodeInfo) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[13] + 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 StreamingNodeInfo.ProtoReflect.Descriptor instead. +func (*StreamingNodeInfo) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{13} +} + +func (x *StreamingNodeInfo) GetServerId() int64 { + if x != nil { + return x.ServerId + } + return 0 +} + +func (x *StreamingNodeInfo) GetAddress() string { + if x != nil { + return x.Address + } + return "" +} + +// StreamingNodeAssignment is the assignment info of a streaming node. +type StreamingNodeAssignment struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Node *StreamingNodeInfo `protobuf:"bytes,1,opt,name=node,proto3" json:"node,omitempty"` + Channels []*PChannelInfo `protobuf:"bytes,2,rep,name=channels,proto3" json:"channels,omitempty"` +} + +func (x *StreamingNodeAssignment) Reset() { + *x = StreamingNodeAssignment{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StreamingNodeAssignment) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StreamingNodeAssignment) ProtoMessage() {} + +func (x *StreamingNodeAssignment) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[14] + 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 StreamingNodeAssignment.ProtoReflect.Descriptor instead. +func (*StreamingNodeAssignment) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{14} +} + +func (x *StreamingNodeAssignment) GetNode() *StreamingNodeInfo { + if x != nil { + return x.Node + } + return nil +} + +func (x *StreamingNodeAssignment) GetChannels() []*PChannelInfo { + if x != nil { + return x.Channels + } + return nil +} + +// DeliverPolicy is the policy to deliver message. +type DeliverPolicy struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Policy: + // + // *DeliverPolicy_All + // *DeliverPolicy_Latest + // *DeliverPolicy_StartFrom + // *DeliverPolicy_StartAfter + Policy isDeliverPolicy_Policy `protobuf_oneof:"policy"` +} + +func (x *DeliverPolicy) Reset() { + *x = DeliverPolicy{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeliverPolicy) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeliverPolicy) ProtoMessage() {} + +func (x *DeliverPolicy) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[15] + 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 DeliverPolicy.ProtoReflect.Descriptor instead. +func (*DeliverPolicy) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{15} +} + +func (m *DeliverPolicy) GetPolicy() isDeliverPolicy_Policy { + if m != nil { + return m.Policy + } + return nil +} + +func (x *DeliverPolicy) GetAll() *emptypb.Empty { + if x, ok := x.GetPolicy().(*DeliverPolicy_All); ok { + return x.All + } + return nil +} + +func (x *DeliverPolicy) GetLatest() *emptypb.Empty { + if x, ok := x.GetPolicy().(*DeliverPolicy_Latest); ok { + return x.Latest + } + return nil +} + +func (x *DeliverPolicy) GetStartFrom() *messagespb.MessageID { + if x, ok := x.GetPolicy().(*DeliverPolicy_StartFrom); ok { + return x.StartFrom + } + return nil +} + +func (x *DeliverPolicy) GetStartAfter() *messagespb.MessageID { + if x, ok := x.GetPolicy().(*DeliverPolicy_StartAfter); ok { + return x.StartAfter + } + return nil +} + +type isDeliverPolicy_Policy interface { + isDeliverPolicy_Policy() +} + +type DeliverPolicy_All struct { + All *emptypb.Empty `protobuf:"bytes,1,opt,name=all,proto3,oneof"` // deliver all messages. +} + +type DeliverPolicy_Latest struct { + Latest *emptypb.Empty `protobuf:"bytes,2,opt,name=latest,proto3,oneof"` // deliver the latest message. +} + +type DeliverPolicy_StartFrom struct { + StartFrom *messagespb.MessageID `protobuf:"bytes,3,opt,name=start_from,json=startFrom,proto3,oneof"` // deliver message from this message id. [startFrom, ...] +} + +type DeliverPolicy_StartAfter struct { + StartAfter *messagespb.MessageID `protobuf:"bytes,4,opt,name=start_after,json=startAfter,proto3,oneof"` // deliver message after this message id. (startAfter, ...] +} + +func (*DeliverPolicy_All) isDeliverPolicy_Policy() {} + +func (*DeliverPolicy_Latest) isDeliverPolicy_Policy() {} + +func (*DeliverPolicy_StartFrom) isDeliverPolicy_Policy() {} + +func (*DeliverPolicy_StartAfter) isDeliverPolicy_Policy() {} + +// DeliverFilter is the filter to deliver message. +type DeliverFilter struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Filter: + // + // *DeliverFilter_TimeTickGt + // *DeliverFilter_TimeTickGte + // *DeliverFilter_MessageType + Filter isDeliverFilter_Filter `protobuf_oneof:"filter"` +} + +func (x *DeliverFilter) Reset() { + *x = DeliverFilter{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeliverFilter) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeliverFilter) ProtoMessage() {} + +func (x *DeliverFilter) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[16] + 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 DeliverFilter.ProtoReflect.Descriptor instead. +func (*DeliverFilter) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{16} +} + +func (m *DeliverFilter) GetFilter() isDeliverFilter_Filter { + if m != nil { + return m.Filter + } + return nil +} + +func (x *DeliverFilter) GetTimeTickGt() *DeliverFilterTimeTickGT { + if x, ok := x.GetFilter().(*DeliverFilter_TimeTickGt); ok { + return x.TimeTickGt + } + return nil +} + +func (x *DeliverFilter) GetTimeTickGte() *DeliverFilterTimeTickGTE { + if x, ok := x.GetFilter().(*DeliverFilter_TimeTickGte); ok { + return x.TimeTickGte + } + return nil +} + +func (x *DeliverFilter) GetMessageType() *DeliverFilterMessageType { + if x, ok := x.GetFilter().(*DeliverFilter_MessageType); ok { + return x.MessageType + } + return nil +} + +type isDeliverFilter_Filter interface { + isDeliverFilter_Filter() +} + +type DeliverFilter_TimeTickGt struct { + TimeTickGt *DeliverFilterTimeTickGT `protobuf:"bytes,1,opt,name=time_tick_gt,json=timeTickGt,proto3,oneof"` +} + +type DeliverFilter_TimeTickGte struct { + TimeTickGte *DeliverFilterTimeTickGTE `protobuf:"bytes,2,opt,name=time_tick_gte,json=timeTickGte,proto3,oneof"` +} + +type DeliverFilter_MessageType struct { + MessageType *DeliverFilterMessageType `protobuf:"bytes,3,opt,name=message_type,json=messageType,proto3,oneof"` +} + +func (*DeliverFilter_TimeTickGt) isDeliverFilter_Filter() {} + +func (*DeliverFilter_TimeTickGte) isDeliverFilter_Filter() {} + +func (*DeliverFilter_MessageType) isDeliverFilter_Filter() {} + +// DeliverFilterTimeTickGT is the filter to deliver message with time tick +// greater than this value. +type DeliverFilterTimeTickGT struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + TimeTick uint64 `protobuf:"varint,1,opt,name=time_tick,json=timeTick,proto3" json:"time_tick,omitempty"` // deliver message with time tick greater than this value. +} + +func (x *DeliverFilterTimeTickGT) Reset() { + *x = DeliverFilterTimeTickGT{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeliverFilterTimeTickGT) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeliverFilterTimeTickGT) ProtoMessage() {} + +func (x *DeliverFilterTimeTickGT) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[17] + 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 DeliverFilterTimeTickGT.ProtoReflect.Descriptor instead. +func (*DeliverFilterTimeTickGT) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{17} +} + +func (x *DeliverFilterTimeTickGT) GetTimeTick() uint64 { + if x != nil { + return x.TimeTick + } + return 0 +} + +// DeliverFilterTimeTickGTE is the filter to deliver message with time tick +// greater than or equal to this value. +type DeliverFilterTimeTickGTE struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + TimeTick uint64 `protobuf:"varint,1,opt,name=time_tick,json=timeTick,proto3" json:"time_tick,omitempty"` // deliver message with time tick greater than or +} + +func (x *DeliverFilterTimeTickGTE) Reset() { + *x = DeliverFilterTimeTickGTE{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeliverFilterTimeTickGTE) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeliverFilterTimeTickGTE) ProtoMessage() {} + +func (x *DeliverFilterTimeTickGTE) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[18] + 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 DeliverFilterTimeTickGTE.ProtoReflect.Descriptor instead. +func (*DeliverFilterTimeTickGTE) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{18} +} + +func (x *DeliverFilterTimeTickGTE) GetTimeTick() uint64 { + if x != nil { + return x.TimeTick + } + return 0 +} + +type DeliverFilterMessageType struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // deliver message with message type. + MessageTypes []messagespb.MessageType `protobuf:"varint,1,rep,packed,name=message_types,json=messageTypes,proto3,enum=milvus.proto.messages.MessageType" json:"message_types,omitempty"` +} + +func (x *DeliverFilterMessageType) Reset() { + *x = DeliverFilterMessageType{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[19] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeliverFilterMessageType) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeliverFilterMessageType) ProtoMessage() {} + +func (x *DeliverFilterMessageType) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[19] + 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 DeliverFilterMessageType.ProtoReflect.Descriptor instead. +func (*DeliverFilterMessageType) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{19} +} + +func (x *DeliverFilterMessageType) GetMessageTypes() []messagespb.MessageType { + if x != nil { + return x.MessageTypes + } + return nil +} + +// StreamingError is the error type for log internal component. +type StreamingError struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Code StreamingCode `protobuf:"varint,1,opt,name=code,proto3,enum=milvus.proto.streaming.StreamingCode" json:"code,omitempty"` + Cause string `protobuf:"bytes,2,opt,name=cause,proto3" json:"cause,omitempty"` +} + +func (x *StreamingError) Reset() { + *x = StreamingError{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[20] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StreamingError) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StreamingError) ProtoMessage() {} + +func (x *StreamingError) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[20] + 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 StreamingError.ProtoReflect.Descriptor instead. +func (*StreamingError) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{20} +} + +func (x *StreamingError) GetCode() StreamingCode { + if x != nil { + return x.Code + } + return StreamingCode_STREAMING_CODE_OK +} + +func (x *StreamingError) GetCause() string { + if x != nil { + return x.Cause + } + return "" +} + +// ProduceRequest is the request of the Produce RPC. +// Channel name will be passthrough in the header of stream bu not in the +// request body. +type ProduceRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Request: + // + // *ProduceRequest_Produce + // *ProduceRequest_Close + Request isProduceRequest_Request `protobuf_oneof:"request"` +} + +func (x *ProduceRequest) Reset() { + *x = ProduceRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[21] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ProduceRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProduceRequest) ProtoMessage() {} + +func (x *ProduceRequest) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[21] + 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 ProduceRequest.ProtoReflect.Descriptor instead. +func (*ProduceRequest) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{21} +} + +func (m *ProduceRequest) GetRequest() isProduceRequest_Request { + if m != nil { + return m.Request + } + return nil +} + +func (x *ProduceRequest) GetProduce() *ProduceMessageRequest { + if x, ok := x.GetRequest().(*ProduceRequest_Produce); ok { + return x.Produce + } + return nil +} + +func (x *ProduceRequest) GetClose() *CloseProducerRequest { + if x, ok := x.GetRequest().(*ProduceRequest_Close); ok { + return x.Close + } + return nil +} + +type isProduceRequest_Request interface { + isProduceRequest_Request() +} + +type ProduceRequest_Produce struct { + Produce *ProduceMessageRequest `protobuf:"bytes,1,opt,name=produce,proto3,oneof"` +} + +type ProduceRequest_Close struct { + Close *CloseProducerRequest `protobuf:"bytes,2,opt,name=close,proto3,oneof"` +} + +func (*ProduceRequest_Produce) isProduceRequest_Request() {} + +func (*ProduceRequest_Close) isProduceRequest_Request() {} + +// CreateProducerRequest is the request of the CreateProducer RPC. +// CreateProducerRequest is passed in the header of stream. +type CreateProducerRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Pchannel *PChannelInfo `protobuf:"bytes,1,opt,name=pchannel,proto3" json:"pchannel,omitempty"` +} + +func (x *CreateProducerRequest) Reset() { + *x = CreateProducerRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[22] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateProducerRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateProducerRequest) ProtoMessage() {} + +func (x *CreateProducerRequest) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[22] + 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 CreateProducerRequest.ProtoReflect.Descriptor instead. +func (*CreateProducerRequest) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{22} +} + +func (x *CreateProducerRequest) GetPchannel() *PChannelInfo { + if x != nil { + return x.Pchannel + } + return nil +} + +// ProduceMessageRequest is the request of the Produce RPC. +type ProduceMessageRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + RequestId int64 `protobuf:"varint,1,opt,name=request_id,json=requestId,proto3" json:"request_id,omitempty"` // request id for reply. + Message *messagespb.Message `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"` // message to be sent. +} + +func (x *ProduceMessageRequest) Reset() { + *x = ProduceMessageRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[23] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ProduceMessageRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProduceMessageRequest) ProtoMessage() {} + +func (x *ProduceMessageRequest) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[23] + 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 ProduceMessageRequest.ProtoReflect.Descriptor instead. +func (*ProduceMessageRequest) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{23} +} + +func (x *ProduceMessageRequest) GetRequestId() int64 { + if x != nil { + return x.RequestId + } + return 0 +} + +func (x *ProduceMessageRequest) GetMessage() *messagespb.Message { + if x != nil { + return x.Message + } + return nil +} + +// CloseProducerRequest is the request of the CloseProducer RPC. +// After CloseProducerRequest is requested, no more ProduceRequest can be sent. +type CloseProducerRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *CloseProducerRequest) Reset() { + *x = CloseProducerRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[24] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CloseProducerRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CloseProducerRequest) ProtoMessage() {} + +func (x *CloseProducerRequest) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[24] + 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 CloseProducerRequest.ProtoReflect.Descriptor instead. +func (*CloseProducerRequest) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{24} +} + +// ProduceResponse is the response of the Produce RPC. +type ProduceResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Response: + // + // *ProduceResponse_Create + // *ProduceResponse_Produce + // *ProduceResponse_Close + Response isProduceResponse_Response `protobuf_oneof:"response"` +} + +func (x *ProduceResponse) Reset() { + *x = ProduceResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[25] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ProduceResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProduceResponse) ProtoMessage() {} + +func (x *ProduceResponse) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[25] + 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 ProduceResponse.ProtoReflect.Descriptor instead. +func (*ProduceResponse) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{25} +} + +func (m *ProduceResponse) GetResponse() isProduceResponse_Response { + if m != nil { + return m.Response + } + return nil +} + +func (x *ProduceResponse) GetCreate() *CreateProducerResponse { + if x, ok := x.GetResponse().(*ProduceResponse_Create); ok { + return x.Create + } + return nil +} + +func (x *ProduceResponse) GetProduce() *ProduceMessageResponse { + if x, ok := x.GetResponse().(*ProduceResponse_Produce); ok { + return x.Produce + } + return nil +} + +func (x *ProduceResponse) GetClose() *CloseProducerResponse { + if x, ok := x.GetResponse().(*ProduceResponse_Close); ok { + return x.Close + } + return nil +} + +type isProduceResponse_Response interface { + isProduceResponse_Response() +} + +type ProduceResponse_Create struct { + Create *CreateProducerResponse `protobuf:"bytes,1,opt,name=create,proto3,oneof"` +} + +type ProduceResponse_Produce struct { + Produce *ProduceMessageResponse `protobuf:"bytes,2,opt,name=produce,proto3,oneof"` +} + +type ProduceResponse_Close struct { + Close *CloseProducerResponse `protobuf:"bytes,3,opt,name=close,proto3,oneof"` +} + +func (*ProduceResponse_Create) isProduceResponse_Response() {} + +func (*ProduceResponse_Produce) isProduceResponse_Response() {} + +func (*ProduceResponse_Close) isProduceResponse_Response() {} + +// CreateProducerResponse is the result of the CreateProducer RPC. +type CreateProducerResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + WalName string `protobuf:"bytes,1,opt,name=wal_name,json=walName,proto3" json:"wal_name,omitempty"` // wal name at server side. + ProducerServerId int64 `protobuf:"varint,2,opt,name=producer_server_id,json=producerServerId,proto3" json:"producer_server_id,omitempty"` // A unique producer server id on streamingnode +} + +func (x *CreateProducerResponse) Reset() { + *x = CreateProducerResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[26] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateProducerResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateProducerResponse) ProtoMessage() {} + +func (x *CreateProducerResponse) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[26] + 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 CreateProducerResponse.ProtoReflect.Descriptor instead. +func (*CreateProducerResponse) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{26} +} + +func (x *CreateProducerResponse) GetWalName() string { + if x != nil { + return x.WalName + } + return "" +} + +func (x *CreateProducerResponse) GetProducerServerId() int64 { + if x != nil { + return x.ProducerServerId + } + return 0 +} + +// ProduceMessageResponse is the response of the ProduceMessage RPC. +type ProduceMessageResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + RequestId int64 `protobuf:"varint,1,opt,name=request_id,json=requestId,proto3" json:"request_id,omitempty"` + // Types that are assignable to Response: + // + // *ProduceMessageResponse_Result + // *ProduceMessageResponse_Error + Response isProduceMessageResponse_Response `protobuf_oneof:"response"` +} + +func (x *ProduceMessageResponse) Reset() { + *x = ProduceMessageResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[27] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ProduceMessageResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProduceMessageResponse) ProtoMessage() {} + +func (x *ProduceMessageResponse) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[27] + 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 ProduceMessageResponse.ProtoReflect.Descriptor instead. +func (*ProduceMessageResponse) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{27} +} + +func (x *ProduceMessageResponse) GetRequestId() int64 { + if x != nil { + return x.RequestId + } + return 0 +} + +func (m *ProduceMessageResponse) GetResponse() isProduceMessageResponse_Response { + if m != nil { + return m.Response + } + return nil +} + +func (x *ProduceMessageResponse) GetResult() *ProduceMessageResponseResult { + if x, ok := x.GetResponse().(*ProduceMessageResponse_Result); ok { + return x.Result + } + return nil +} + +func (x *ProduceMessageResponse) GetError() *StreamingError { + if x, ok := x.GetResponse().(*ProduceMessageResponse_Error); ok { + return x.Error + } + return nil +} + +type isProduceMessageResponse_Response interface { + isProduceMessageResponse_Response() +} + +type ProduceMessageResponse_Result struct { + Result *ProduceMessageResponseResult `protobuf:"bytes,2,opt,name=result,proto3,oneof"` +} + +type ProduceMessageResponse_Error struct { + Error *StreamingError `protobuf:"bytes,3,opt,name=error,proto3,oneof"` +} + +func (*ProduceMessageResponse_Result) isProduceMessageResponse_Response() {} + +func (*ProduceMessageResponse_Error) isProduceMessageResponse_Response() {} + +// ProduceMessageResponseResult is the result of the produce message streaming +// RPC. +type ProduceMessageResponseResult struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Id *messagespb.MessageID `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` // the offset of the message in the channel. + Timetick uint64 `protobuf:"varint,2,opt,name=timetick,proto3" json:"timetick,omitempty"` // the timetick of that message sent. + TxnContext *messagespb.TxnContext `protobuf:"bytes,3,opt,name=txnContext,proto3" json:"txnContext,omitempty"` // the txn context of the message. + Extra *anypb.Any `protobuf:"bytes,4,opt,name=extra,proto3" json:"extra,omitempty"` // the extra message. +} + +func (x *ProduceMessageResponseResult) Reset() { + *x = ProduceMessageResponseResult{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[28] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ProduceMessageResponseResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProduceMessageResponseResult) ProtoMessage() {} + +func (x *ProduceMessageResponseResult) ProtoReflect() protoreflect.Message { + mi := &file_streaming_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 ProduceMessageResponseResult.ProtoReflect.Descriptor instead. +func (*ProduceMessageResponseResult) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{28} +} + +func (x *ProduceMessageResponseResult) GetId() *messagespb.MessageID { + if x != nil { + return x.Id + } + return nil +} + +func (x *ProduceMessageResponseResult) GetTimetick() uint64 { + if x != nil { + return x.Timetick + } + return 0 +} + +func (x *ProduceMessageResponseResult) GetTxnContext() *messagespb.TxnContext { + if x != nil { + return x.TxnContext + } + return nil +} + +func (x *ProduceMessageResponseResult) GetExtra() *anypb.Any { + if x != nil { + return x.Extra + } + return nil +} + +// CloseProducerResponse is the result of the CloseProducer RPC. +type CloseProducerResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *CloseProducerResponse) Reset() { + *x = CloseProducerResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[29] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CloseProducerResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CloseProducerResponse) ProtoMessage() {} + +func (x *CloseProducerResponse) ProtoReflect() protoreflect.Message { + mi := &file_streaming_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 CloseProducerResponse.ProtoReflect.Descriptor instead. +func (*CloseProducerResponse) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{29} +} + +// ConsumeRequest is the request of the Consume RPC. +// Add more control block in future. +type ConsumeRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Request: + // + // *ConsumeRequest_CreateVchannelConsumer + // *ConsumeRequest_CreateVchannelConsumers + // *ConsumeRequest_CloseVchannel + // *ConsumeRequest_Close + Request isConsumeRequest_Request `protobuf_oneof:"request"` +} + +func (x *ConsumeRequest) Reset() { + *x = ConsumeRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[30] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ConsumeRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ConsumeRequest) ProtoMessage() {} + +func (x *ConsumeRequest) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[30] + 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 ConsumeRequest.ProtoReflect.Descriptor instead. +func (*ConsumeRequest) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{30} +} + +func (m *ConsumeRequest) GetRequest() isConsumeRequest_Request { + if m != nil { + return m.Request + } + return nil +} + +func (x *ConsumeRequest) GetCreateVchannelConsumer() *CreateVChannelConsumerRequest { + if x, ok := x.GetRequest().(*ConsumeRequest_CreateVchannelConsumer); ok { + return x.CreateVchannelConsumer + } + return nil +} + +func (x *ConsumeRequest) GetCreateVchannelConsumers() *CreateVChannelConsumersRequest { + if x, ok := x.GetRequest().(*ConsumeRequest_CreateVchannelConsumers); ok { + return x.CreateVchannelConsumers + } + return nil +} + +func (x *ConsumeRequest) GetCloseVchannel() *CloseVChannelConsumerRequest { + if x, ok := x.GetRequest().(*ConsumeRequest_CloseVchannel); ok { + return x.CloseVchannel + } + return nil +} + +func (x *ConsumeRequest) GetClose() *CloseConsumerRequest { + if x, ok := x.GetRequest().(*ConsumeRequest_Close); ok { + return x.Close + } + return nil +} + +type isConsumeRequest_Request interface { + isConsumeRequest_Request() +} + +type ConsumeRequest_CreateVchannelConsumer struct { + CreateVchannelConsumer *CreateVChannelConsumerRequest `protobuf:"bytes,1,opt,name=create_vchannel_consumer,json=createVchannelConsumer,proto3,oneof"` +} + +type ConsumeRequest_CreateVchannelConsumers struct { + CreateVchannelConsumers *CreateVChannelConsumersRequest `protobuf:"bytes,2,opt,name=create_vchannel_consumers,json=createVchannelConsumers,proto3,oneof"` // Create multiple vchannel consumers, used for recovery in future. +} + +type ConsumeRequest_CloseVchannel struct { + CloseVchannel *CloseVChannelConsumerRequest `protobuf:"bytes,3,opt,name=close_vchannel,json=closeVchannel,proto3,oneof"` +} + +type ConsumeRequest_Close struct { + Close *CloseConsumerRequest `protobuf:"bytes,4,opt,name=close,proto3,oneof"` +} + +func (*ConsumeRequest_CreateVchannelConsumer) isConsumeRequest_Request() {} + +func (*ConsumeRequest_CreateVchannelConsumers) isConsumeRequest_Request() {} + +func (*ConsumeRequest_CloseVchannel) isConsumeRequest_Request() {} + +func (*ConsumeRequest_Close) isConsumeRequest_Request() {} + +// CloseConsumerRequest is the request of the CloseConsumer RPC. +// After CloseConsumerRequest is requested, no more ConsumeRequest can be sent. +type CloseConsumerRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *CloseConsumerRequest) Reset() { + *x = CloseConsumerRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[31] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CloseConsumerRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CloseConsumerRequest) ProtoMessage() {} + +func (x *CloseConsumerRequest) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[31] + 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 CloseConsumerRequest.ProtoReflect.Descriptor instead. +func (*CloseConsumerRequest) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{31} +} + +// CreateConsumerRequest is the request of the CreateConsumer RPC. +// CreateConsumerRequest is passed in the header of stream. +type CreateConsumerRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Pchannel *PChannelInfo `protobuf:"bytes,1,opt,name=pchannel,proto3" json:"pchannel,omitempty"` +} + +func (x *CreateConsumerRequest) Reset() { + *x = CreateConsumerRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[32] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateConsumerRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateConsumerRequest) ProtoMessage() {} + +func (x *CreateConsumerRequest) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[32] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CreateConsumerRequest.ProtoReflect.Descriptor instead. +func (*CreateConsumerRequest) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{32} +} + +func (x *CreateConsumerRequest) GetPchannel() *PChannelInfo { + if x != nil { + return x.Pchannel + } + return nil +} + +type CreateVChannelConsumersRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CreateVchannels []*CreateVChannelConsumerRequest `protobuf:"bytes,1,rep,name=create_vchannels,json=createVchannels,proto3" json:"create_vchannels,omitempty"` +} + +func (x *CreateVChannelConsumersRequest) Reset() { + *x = CreateVChannelConsumersRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[33] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateVChannelConsumersRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateVChannelConsumersRequest) ProtoMessage() {} + +func (x *CreateVChannelConsumersRequest) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[33] + 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 CreateVChannelConsumersRequest.ProtoReflect.Descriptor instead. +func (*CreateVChannelConsumersRequest) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{33} +} + +func (x *CreateVChannelConsumersRequest) GetCreateVchannels() []*CreateVChannelConsumerRequest { + if x != nil { + return x.CreateVchannels + } + return nil +} + +// CreateVChannelConsumerRequest is the request of the CreateVChannelConsumer +// RPC. It's used to create a new vchannel consumer at server side. +type CreateVChannelConsumerRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Vchannel string `protobuf:"bytes,1,opt,name=vchannel,proto3" json:"vchannel,omitempty"` + DeliverPolicy *DeliverPolicy `protobuf:"bytes,2,opt,name=deliver_policy,json=deliverPolicy,proto3" json:"deliver_policy,omitempty"` // deliver policy. + DeliverFilters []*DeliverFilter `protobuf:"bytes,3,rep,name=deliver_filters,json=deliverFilters,proto3" json:"deliver_filters,omitempty"` // deliver filter. +} + +func (x *CreateVChannelConsumerRequest) Reset() { + *x = CreateVChannelConsumerRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[34] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateVChannelConsumerRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateVChannelConsumerRequest) ProtoMessage() {} + +func (x *CreateVChannelConsumerRequest) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[34] + 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 CreateVChannelConsumerRequest.ProtoReflect.Descriptor instead. +func (*CreateVChannelConsumerRequest) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{34} +} + +func (x *CreateVChannelConsumerRequest) GetVchannel() string { + if x != nil { + return x.Vchannel + } + return "" +} + +func (x *CreateVChannelConsumerRequest) GetDeliverPolicy() *DeliverPolicy { + if x != nil { + return x.DeliverPolicy + } + return nil +} + +func (x *CreateVChannelConsumerRequest) GetDeliverFilters() []*DeliverFilter { + if x != nil { + return x.DeliverFilters + } + return nil +} + +// ConsumeMessageRequest is the request of the Consume RPC. +type CreateVChannelConsumersResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CreateVchannels []*CreateVChannelConsumerResponse `protobuf:"bytes,1,rep,name=create_vchannels,json=createVchannels,proto3" json:"create_vchannels,omitempty"` +} + +func (x *CreateVChannelConsumersResponse) Reset() { + *x = CreateVChannelConsumersResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[35] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateVChannelConsumersResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateVChannelConsumersResponse) ProtoMessage() {} + +func (x *CreateVChannelConsumersResponse) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[35] + 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 CreateVChannelConsumersResponse.ProtoReflect.Descriptor instead. +func (*CreateVChannelConsumersResponse) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{35} +} + +func (x *CreateVChannelConsumersResponse) GetCreateVchannels() []*CreateVChannelConsumerResponse { + if x != nil { + return x.CreateVchannels + } + return nil +} + +// CreateVChannelConsumerResponse is the response of the CreateVChannelConsumer +// RPC. +type CreateVChannelConsumerResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Response: + // + // *CreateVChannelConsumerResponse_ConsumerId + // *CreateVChannelConsumerResponse_Error + Response isCreateVChannelConsumerResponse_Response `protobuf_oneof:"response"` +} + +func (x *CreateVChannelConsumerResponse) Reset() { + *x = CreateVChannelConsumerResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[36] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateVChannelConsumerResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateVChannelConsumerResponse) ProtoMessage() {} + +func (x *CreateVChannelConsumerResponse) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[36] + 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 CreateVChannelConsumerResponse.ProtoReflect.Descriptor instead. +func (*CreateVChannelConsumerResponse) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{36} +} + +func (m *CreateVChannelConsumerResponse) GetResponse() isCreateVChannelConsumerResponse_Response { + if m != nil { + return m.Response + } + return nil +} + +func (x *CreateVChannelConsumerResponse) GetConsumerId() int64 { + if x, ok := x.GetResponse().(*CreateVChannelConsumerResponse_ConsumerId); ok { + return x.ConsumerId + } + return 0 +} + +func (x *CreateVChannelConsumerResponse) GetError() *StreamingError { + if x, ok := x.GetResponse().(*CreateVChannelConsumerResponse_Error); ok { + return x.Error + } + return nil +} + +type isCreateVChannelConsumerResponse_Response interface { + isCreateVChannelConsumerResponse_Response() +} + +type CreateVChannelConsumerResponse_ConsumerId struct { + ConsumerId int64 `protobuf:"varint,1,opt,name=consumer_id,json=consumerId,proto3,oneof"` +} + +type CreateVChannelConsumerResponse_Error struct { + Error *StreamingError `protobuf:"bytes,2,opt,name=error,proto3,oneof"` +} + +func (*CreateVChannelConsumerResponse_ConsumerId) isCreateVChannelConsumerResponse_Response() {} + +func (*CreateVChannelConsumerResponse_Error) isCreateVChannelConsumerResponse_Response() {} + +// CloseVChannelConsumerRequest is the request of the CloseVChannelConsumer RPC. +type CloseVChannelConsumerRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ConsumerId int64 `protobuf:"varint,1,opt,name=consumer_id,json=consumerId,proto3" json:"consumer_id,omitempty"` +} + +func (x *CloseVChannelConsumerRequest) Reset() { + *x = CloseVChannelConsumerRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[37] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CloseVChannelConsumerRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CloseVChannelConsumerRequest) ProtoMessage() {} + +func (x *CloseVChannelConsumerRequest) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[37] + 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 CloseVChannelConsumerRequest.ProtoReflect.Descriptor instead. +func (*CloseVChannelConsumerRequest) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{37} +} + +func (x *CloseVChannelConsumerRequest) GetConsumerId() int64 { + if x != nil { + return x.ConsumerId + } + return 0 +} + +// CloseVChannelConsumerResponse is the response of the CloseVChannelConsumer +// RPC. +type CloseVChannelConsumerResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ConsumerId int64 `protobuf:"varint,1,opt,name=consumer_id,json=consumerId,proto3" json:"consumer_id,omitempty"` +} + +func (x *CloseVChannelConsumerResponse) Reset() { + *x = CloseVChannelConsumerResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[38] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CloseVChannelConsumerResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CloseVChannelConsumerResponse) ProtoMessage() {} + +func (x *CloseVChannelConsumerResponse) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[38] + 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 CloseVChannelConsumerResponse.ProtoReflect.Descriptor instead. +func (*CloseVChannelConsumerResponse) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{38} +} + +func (x *CloseVChannelConsumerResponse) GetConsumerId() int64 { + if x != nil { + return x.ConsumerId + } + return 0 +} + +// ConsumeResponse is the reponse of the Consume RPC. +type ConsumeResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Response: + // + // *ConsumeResponse_Create + // *ConsumeResponse_Consume + // *ConsumeResponse_CreateVchannel + // *ConsumeResponse_CreateVchannels + // *ConsumeResponse_CloseVchannel + // *ConsumeResponse_Close + Response isConsumeResponse_Response `protobuf_oneof:"response"` +} + +func (x *ConsumeResponse) Reset() { + *x = ConsumeResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[39] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ConsumeResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ConsumeResponse) ProtoMessage() {} + +func (x *ConsumeResponse) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[39] + 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 ConsumeResponse.ProtoReflect.Descriptor instead. +func (*ConsumeResponse) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{39} +} + +func (m *ConsumeResponse) GetResponse() isConsumeResponse_Response { + if m != nil { + return m.Response + } + return nil +} + +func (x *ConsumeResponse) GetCreate() *CreateConsumerResponse { + if x, ok := x.GetResponse().(*ConsumeResponse_Create); ok { + return x.Create + } + return nil +} + +func (x *ConsumeResponse) GetConsume() *ConsumeMessageReponse { + if x, ok := x.GetResponse().(*ConsumeResponse_Consume); ok { + return x.Consume + } + return nil +} + +func (x *ConsumeResponse) GetCreateVchannel() *CreateVChannelConsumerResponse { + if x, ok := x.GetResponse().(*ConsumeResponse_CreateVchannel); ok { + return x.CreateVchannel + } + return nil +} + +func (x *ConsumeResponse) GetCreateVchannels() *CreateVChannelConsumersResponse { + if x, ok := x.GetResponse().(*ConsumeResponse_CreateVchannels); ok { + return x.CreateVchannels + } + return nil +} + +func (x *ConsumeResponse) GetCloseVchannel() *CloseVChannelConsumerResponse { + if x, ok := x.GetResponse().(*ConsumeResponse_CloseVchannel); ok { + return x.CloseVchannel + } + return nil +} + +func (x *ConsumeResponse) GetClose() *CloseConsumerResponse { + if x, ok := x.GetResponse().(*ConsumeResponse_Close); ok { + return x.Close + } + return nil +} + +type isConsumeResponse_Response interface { + isConsumeResponse_Response() +} + +type ConsumeResponse_Create struct { + Create *CreateConsumerResponse `protobuf:"bytes,1,opt,name=create,proto3,oneof"` +} + +type ConsumeResponse_Consume struct { + Consume *ConsumeMessageReponse `protobuf:"bytes,2,opt,name=consume,proto3,oneof"` +} + +type ConsumeResponse_CreateVchannel struct { + CreateVchannel *CreateVChannelConsumerResponse `protobuf:"bytes,3,opt,name=create_vchannel,json=createVchannel,proto3,oneof"` +} + +type ConsumeResponse_CreateVchannels struct { + CreateVchannels *CreateVChannelConsumersResponse `protobuf:"bytes,4,opt,name=create_vchannels,json=createVchannels,proto3,oneof"` +} + +type ConsumeResponse_CloseVchannel struct { + CloseVchannel *CloseVChannelConsumerResponse `protobuf:"bytes,5,opt,name=close_vchannel,json=closeVchannel,proto3,oneof"` +} + +type ConsumeResponse_Close struct { + Close *CloseConsumerResponse `protobuf:"bytes,6,opt,name=close,proto3,oneof"` +} + +func (*ConsumeResponse_Create) isConsumeResponse_Response() {} + +func (*ConsumeResponse_Consume) isConsumeResponse_Response() {} + +func (*ConsumeResponse_CreateVchannel) isConsumeResponse_Response() {} + +func (*ConsumeResponse_CreateVchannels) isConsumeResponse_Response() {} + +func (*ConsumeResponse_CloseVchannel) isConsumeResponse_Response() {} + +func (*ConsumeResponse_Close) isConsumeResponse_Response() {} + +type CreateConsumerResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + WalName string `protobuf:"bytes,1,opt,name=wal_name,json=walName,proto3" json:"wal_name,omitempty"` // wal name at server side. + // A unique consumer id on streamingnode for this + // consumer in streamingnode lifetime. + ConsumerServerId int64 `protobuf:"varint,2,opt,name=consumer_server_id,json=consumerServerId,proto3" json:"consumer_server_id,omitempty"` +} + +func (x *CreateConsumerResponse) Reset() { + *x = CreateConsumerResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[40] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateConsumerResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateConsumerResponse) ProtoMessage() {} + +func (x *CreateConsumerResponse) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[40] + 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 CreateConsumerResponse.ProtoReflect.Descriptor instead. +func (*CreateConsumerResponse) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{40} +} + +func (x *CreateConsumerResponse) GetWalName() string { + if x != nil { + return x.WalName + } + return "" +} + +func (x *CreateConsumerResponse) GetConsumerServerId() int64 { + if x != nil { + return x.ConsumerServerId + } + return 0 +} + +type ConsumeMessageReponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ConsumerId int64 `protobuf:"varint,1,opt,name=consumer_id,json=consumerId,proto3" json:"consumer_id,omitempty"` + Message *messagespb.ImmutableMessage `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"` +} + +func (x *ConsumeMessageReponse) Reset() { + *x = ConsumeMessageReponse{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[41] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ConsumeMessageReponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ConsumeMessageReponse) ProtoMessage() {} + +func (x *ConsumeMessageReponse) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[41] + 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 ConsumeMessageReponse.ProtoReflect.Descriptor instead. +func (*ConsumeMessageReponse) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{41} +} + +func (x *ConsumeMessageReponse) GetConsumerId() int64 { + if x != nil { + return x.ConsumerId + } + return 0 +} + +func (x *ConsumeMessageReponse) GetMessage() *messagespb.ImmutableMessage { + if x != nil { + return x.Message + } + return nil +} + +type CloseConsumerResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *CloseConsumerResponse) Reset() { + *x = CloseConsumerResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[42] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CloseConsumerResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CloseConsumerResponse) ProtoMessage() {} + +func (x *CloseConsumerResponse) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[42] + 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 CloseConsumerResponse.ProtoReflect.Descriptor instead. +func (*CloseConsumerResponse) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{42} +} + +// StreamingManagerAssignRequest is the request message of Assign RPC. +type StreamingNodeManagerAssignRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Pchannel *PChannelInfo `protobuf:"bytes,1,opt,name=pchannel,proto3" json:"pchannel,omitempty"` +} + +func (x *StreamingNodeManagerAssignRequest) Reset() { + *x = StreamingNodeManagerAssignRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[43] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StreamingNodeManagerAssignRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StreamingNodeManagerAssignRequest) ProtoMessage() {} + +func (x *StreamingNodeManagerAssignRequest) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[43] + 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 StreamingNodeManagerAssignRequest.ProtoReflect.Descriptor instead. +func (*StreamingNodeManagerAssignRequest) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{43} +} + +func (x *StreamingNodeManagerAssignRequest) GetPchannel() *PChannelInfo { + if x != nil { + return x.Pchannel + } + return nil +} + +type StreamingNodeManagerAssignResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *StreamingNodeManagerAssignResponse) Reset() { + *x = StreamingNodeManagerAssignResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[44] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StreamingNodeManagerAssignResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StreamingNodeManagerAssignResponse) ProtoMessage() {} + +func (x *StreamingNodeManagerAssignResponse) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[44] + 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 StreamingNodeManagerAssignResponse.ProtoReflect.Descriptor instead. +func (*StreamingNodeManagerAssignResponse) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{44} +} + +type StreamingNodeManagerRemoveRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Pchannel *PChannelInfo `protobuf:"bytes,1,opt,name=pchannel,proto3" json:"pchannel,omitempty"` +} + +func (x *StreamingNodeManagerRemoveRequest) Reset() { + *x = StreamingNodeManagerRemoveRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[45] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StreamingNodeManagerRemoveRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StreamingNodeManagerRemoveRequest) ProtoMessage() {} + +func (x *StreamingNodeManagerRemoveRequest) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[45] + 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 StreamingNodeManagerRemoveRequest.ProtoReflect.Descriptor instead. +func (*StreamingNodeManagerRemoveRequest) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{45} +} + +func (x *StreamingNodeManagerRemoveRequest) GetPchannel() *PChannelInfo { + if x != nil { + return x.Pchannel + } + return nil +} + +type StreamingNodeManagerRemoveResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *StreamingNodeManagerRemoveResponse) Reset() { + *x = StreamingNodeManagerRemoveResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[46] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StreamingNodeManagerRemoveResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StreamingNodeManagerRemoveResponse) ProtoMessage() {} + +func (x *StreamingNodeManagerRemoveResponse) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[46] + 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 StreamingNodeManagerRemoveResponse.ProtoReflect.Descriptor instead. +func (*StreamingNodeManagerRemoveResponse) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{46} +} + +type StreamingNodeManagerCollectStatusRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *StreamingNodeManagerCollectStatusRequest) Reset() { + *x = StreamingNodeManagerCollectStatusRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[47] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StreamingNodeManagerCollectStatusRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StreamingNodeManagerCollectStatusRequest) ProtoMessage() {} + +func (x *StreamingNodeManagerCollectStatusRequest) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[47] + 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 StreamingNodeManagerCollectStatusRequest.ProtoReflect.Descriptor instead. +func (*StreamingNodeManagerCollectStatusRequest) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{47} +} + +type StreamingNodeBalanceAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *StreamingNodeBalanceAttributes) Reset() { + *x = StreamingNodeBalanceAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[48] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StreamingNodeBalanceAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StreamingNodeBalanceAttributes) ProtoMessage() {} + +func (x *StreamingNodeBalanceAttributes) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[48] + 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 StreamingNodeBalanceAttributes.ProtoReflect.Descriptor instead. +func (*StreamingNodeBalanceAttributes) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{48} +} + +type StreamingNodeManagerCollectStatusResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + BalanceAttributes *StreamingNodeBalanceAttributes `protobuf:"bytes,1,opt,name=balance_attributes,json=balanceAttributes,proto3" json:"balance_attributes,omitempty"` +} + +func (x *StreamingNodeManagerCollectStatusResponse) Reset() { + *x = StreamingNodeManagerCollectStatusResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[49] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StreamingNodeManagerCollectStatusResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StreamingNodeManagerCollectStatusResponse) ProtoMessage() {} + +func (x *StreamingNodeManagerCollectStatusResponse) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[49] + 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 StreamingNodeManagerCollectStatusResponse.ProtoReflect.Descriptor instead. +func (*StreamingNodeManagerCollectStatusResponse) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{49} +} + +func (x *StreamingNodeManagerCollectStatusResponse) GetBalanceAttributes() *StreamingNodeBalanceAttributes { + if x != nil { + return x.BalanceAttributes + } + return nil +} + +// / +// / SegmentAssignment +// / +// SegmentAssignmentMeta is the stat of segment assignment. +// These meta is only used to recover status at streaming node segment +// assignment, don't use it outside. +// Used to storage the segment assignment stat +// at meta-store. The WAL use it to determine when to make the segment sealed. +type SegmentAssignmentMeta 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"` + PartitionId int64 `protobuf:"varint,2,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"` + SegmentId int64 `protobuf:"varint,3,opt,name=segment_id,json=segmentId,proto3" json:"segment_id,omitempty"` + Vchannel string `protobuf:"bytes,4,opt,name=vchannel,proto3" json:"vchannel,omitempty"` + State SegmentAssignmentState `protobuf:"varint,5,opt,name=state,proto3,enum=milvus.proto.streaming.SegmentAssignmentState" json:"state,omitempty"` + Stat *SegmentAssignmentStat `protobuf:"bytes,6,opt,name=stat,proto3" json:"stat,omitempty"` +} + +func (x *SegmentAssignmentMeta) Reset() { + *x = SegmentAssignmentMeta{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[50] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentAssignmentMeta) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentAssignmentMeta) ProtoMessage() {} + +func (x *SegmentAssignmentMeta) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[50] + 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 SegmentAssignmentMeta.ProtoReflect.Descriptor instead. +func (*SegmentAssignmentMeta) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{50} +} + +func (x *SegmentAssignmentMeta) GetCollectionId() int64 { + if x != nil { + return x.CollectionId + } + return 0 +} + +func (x *SegmentAssignmentMeta) GetPartitionId() int64 { + if x != nil { + return x.PartitionId + } + return 0 +} + +func (x *SegmentAssignmentMeta) GetSegmentId() int64 { + if x != nil { + return x.SegmentId + } + return 0 +} + +func (x *SegmentAssignmentMeta) GetVchannel() string { + if x != nil { + return x.Vchannel + } + return "" +} + +func (x *SegmentAssignmentMeta) GetState() SegmentAssignmentState { + if x != nil { + return x.State + } + return SegmentAssignmentState_SEGMENT_ASSIGNMENT_STATE_UNKNOWN +} + +func (x *SegmentAssignmentMeta) GetStat() *SegmentAssignmentStat { + if x != nil { + return x.Stat + } + return nil +} + +// SegmentAssignmentStat is the stat of segment assignment. +type SegmentAssignmentStat struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + MaxBinarySize uint64 `protobuf:"varint,1,opt,name=max_binary_size,json=maxBinarySize,proto3" json:"max_binary_size,omitempty"` + InsertedRows uint64 `protobuf:"varint,2,opt,name=inserted_rows,json=insertedRows,proto3" json:"inserted_rows,omitempty"` + InsertedBinarySize uint64 `protobuf:"varint,3,opt,name=inserted_binary_size,json=insertedBinarySize,proto3" json:"inserted_binary_size,omitempty"` + CreateTimestampNanoseconds int64 `protobuf:"varint,4,opt,name=create_timestamp_nanoseconds,json=createTimestampNanoseconds,proto3" json:"create_timestamp_nanoseconds,omitempty"` + LastModifiedTimestampNanoseconds int64 `protobuf:"varint,5,opt,name=last_modified_timestamp_nanoseconds,json=lastModifiedTimestampNanoseconds,proto3" json:"last_modified_timestamp_nanoseconds,omitempty"` + BinlogCounter uint64 `protobuf:"varint,6,opt,name=binlog_counter,json=binlogCounter,proto3" json:"binlog_counter,omitempty"` + CreateSegmentTimeTick uint64 `protobuf:"varint,7,opt,name=create_segment_time_tick,json=createSegmentTimeTick,proto3" json:"create_segment_time_tick,omitempty"` // The timetick of create segment message in wal. +} + +func (x *SegmentAssignmentStat) Reset() { + *x = SegmentAssignmentStat{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[51] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SegmentAssignmentStat) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SegmentAssignmentStat) ProtoMessage() {} + +func (x *SegmentAssignmentStat) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[51] + 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 SegmentAssignmentStat.ProtoReflect.Descriptor instead. +func (*SegmentAssignmentStat) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{51} +} + +func (x *SegmentAssignmentStat) GetMaxBinarySize() uint64 { + if x != nil { + return x.MaxBinarySize + } + return 0 +} + +func (x *SegmentAssignmentStat) GetInsertedRows() uint64 { + if x != nil { + return x.InsertedRows + } + return 0 +} + +func (x *SegmentAssignmentStat) GetInsertedBinarySize() uint64 { + if x != nil { + return x.InsertedBinarySize + } + return 0 +} + +func (x *SegmentAssignmentStat) GetCreateTimestampNanoseconds() int64 { + if x != nil { + return x.CreateTimestampNanoseconds + } + return 0 +} + +func (x *SegmentAssignmentStat) GetLastModifiedTimestampNanoseconds() int64 { + if x != nil { + return x.LastModifiedTimestampNanoseconds + } + return 0 +} + +func (x *SegmentAssignmentStat) GetBinlogCounter() uint64 { + if x != nil { + return x.BinlogCounter + } + return 0 +} + +func (x *SegmentAssignmentStat) GetCreateSegmentTimeTick() uint64 { + if x != nil { + return x.CreateSegmentTimeTick + } + return 0 +} + +var File_streaming_proto protoreflect.FileDescriptor + +var file_streaming_proto_rawDesc = []byte{ + 0x0a, 0x0f, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x12, 0x16, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x1a, 0x0e, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x0c, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1b, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x19, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x61, 0x6e, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, + 0x36, 0x0a, 0x0c, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x12, + 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, + 0x61, 0x6d, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x65, 0x72, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x04, 0x74, 0x65, 0x72, 0x6d, 0x22, 0x6a, 0x0a, 0x15, 0x50, 0x43, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x4c, 0x6f, 0x67, + 0x12, 0x12, 0x0a, 0x04, 0x74, 0x65, 0x72, 0x6d, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, + 0x74, 0x65, 0x72, 0x6d, 0x12, 0x3d, 0x0a, 0x04, 0x6e, 0x6f, 0x64, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x04, 0x6e, + 0x6f, 0x64, 0x65, 0x22, 0x9b, 0x02, 0x0a, 0x0c, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, + 0x4d, 0x65, 0x74, 0x61, 0x12, 0x3e, 0x0a, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, + 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x07, 0x63, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x3d, 0x0a, 0x04, 0x6e, 0x6f, 0x64, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x04, 0x6e, + 0x6f, 0x64, 0x65, 0x12, 0x3f, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0e, 0x32, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x43, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, + 0x74, 0x61, 0x74, 0x65, 0x12, 0x4b, 0x0a, 0x09, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x69, 0x65, + 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, + 0x2e, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, + 0x65, 0x6e, 0x74, 0x4c, 0x6f, 0x67, 0x52, 0x09, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x69, 0x65, + 0x73, 0x22, 0x3b, 0x0a, 0x0b, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x69, 0x72, + 0x12, 0x16, 0x0a, 0x06, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x06, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x12, 0x14, 0x0a, 0x05, 0x6c, 0x6f, 0x63, 0x61, + 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x22, 0xa4, + 0x01, 0x0a, 0x0d, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x54, 0x61, 0x73, 0x6b, + 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x38, 0x0a, 0x07, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 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, 0x07, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x12, 0x40, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0e, 0x32, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x42, 0x72, 0x6f, 0x61, + 0x64, 0x63, 0x61, 0x73, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, + 0x73, 0x74, 0x61, 0x74, 0x65, 0x22, 0x4c, 0x0a, 0x10, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, + 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x07, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 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, 0x07, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x22, 0xd7, 0x01, 0x0a, 0x11, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, + 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x50, 0x0a, 0x07, 0x72, 0x65, 0x73, + 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x69, 0x6e, 0x67, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x1a, 0x70, 0x0a, 0x0c, 0x52, + 0x65, 0x73, 0x75, 0x6c, 0x74, 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, 0x4a, 0x0a, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x65, 0x73, 0x75, + 0x6c, 0x74, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xd1, 0x01, + 0x0a, 0x19, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x63, + 0x6f, 0x76, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x59, 0x0a, 0x0c, 0x72, + 0x65, 0x70, 0x6f, 0x72, 0x74, 0x5f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x34, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x72, + 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x45, 0x72, 0x72, 0x6f, 0x72, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0b, 0x72, 0x65, 0x70, 0x6f, 0x72, + 0x74, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x4e, 0x0a, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, + 0x6c, 0x6f, 0x73, 0x65, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, + 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, + 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x42, 0x09, 0x0a, 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, + 0x64, 0x22, 0x9a, 0x01, 0x0a, 0x1c, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x41, 0x73, 0x73, 0x69, + 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x40, 0x0a, 0x08, 0x70, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x43, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x70, 0x63, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x38, 0x0a, 0x03, 0x65, 0x72, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, + 0x6d, 0x69, 0x6e, 0x67, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x03, 0x65, 0x72, 0x72, 0x22, 0x20, + 0x0a, 0x1e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, + 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x22, 0xe4, 0x01, 0x0a, 0x1a, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x44, + 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x69, 0x0a, 0x0f, 0x66, 0x75, 0x6c, 0x6c, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, + 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, + 0x67, 0x2e, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, + 0x6f, 0x64, 0x65, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x57, 0x69, 0x74, + 0x68, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x0e, 0x66, 0x75, 0x6c, 0x6c, + 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x4f, 0x0a, 0x05, 0x63, 0x6c, + 0x6f, 0x73, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, + 0x6e, 0x67, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, + 0x6e, 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x48, 0x00, 0x52, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x42, 0x0a, 0x0a, 0x08, 0x72, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xba, 0x01, 0x0a, 0x26, 0x46, 0x75, 0x6c, 0x6c, + 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x41, 0x73, 0x73, + 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x57, 0x69, 0x74, 0x68, 0x56, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x12, 0x3d, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x56, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x69, 0x72, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x12, 0x51, 0x0a, 0x0b, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, + 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x41, 0x73, 0x73, + 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x0b, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, + 0x65, 0x6e, 0x74, 0x73, 0x22, 0x21, 0x0a, 0x1f, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x41, 0x73, 0x73, + 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x4a, 0x0a, 0x11, 0x53, 0x74, 0x72, 0x65, 0x61, + 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1b, 0x0a, 0x09, + 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x08, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x49, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x61, 0x64, 0x64, + 0x72, 0x65, 0x73, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, + 0x65, 0x73, 0x73, 0x22, 0x9a, 0x01, 0x0a, 0x17, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, + 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, + 0x3d, 0x0a, 0x04, 0x6e, 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, + 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, + 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x04, 0x6e, 0x6f, 0x64, 0x65, 0x12, 0x40, + 0x0a, 0x08, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, + 0x22, 0xff, 0x01, 0x0a, 0x0d, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x50, 0x6f, 0x6c, 0x69, + 0x63, 0x79, 0x12, 0x2a, 0x0a, 0x03, 0x61, 0x6c, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, + 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x48, 0x00, 0x52, 0x03, 0x61, 0x6c, 0x6c, 0x12, 0x30, + 0x0a, 0x06, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, + 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x48, 0x00, 0x52, 0x06, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, + 0x12, 0x41, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x66, 0x72, 0x6f, 0x6d, 0x18, 0x03, + 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, 0x48, 0x00, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x46, + 0x72, 0x6f, 0x6d, 0x12, 0x43, 0x0a, 0x0b, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x61, 0x66, 0x74, + 0x65, 0x72, 0x18, 0x04, 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, 0x48, 0x00, 0x52, 0x0a, 0x73, 0x74, + 0x61, 0x72, 0x74, 0x41, 0x66, 0x74, 0x65, 0x72, 0x42, 0x08, 0x0a, 0x06, 0x70, 0x6f, 0x6c, 0x69, + 0x63, 0x79, 0x22, 0x9d, 0x02, 0x0a, 0x0d, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x46, 0x69, + 0x6c, 0x74, 0x65, 0x72, 0x12, 0x53, 0x0a, 0x0c, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x74, 0x69, 0x63, + 0x6b, 0x5f, 0x67, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x69, 0x6e, 0x67, 0x2e, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x74, 0x65, + 0x72, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x47, 0x54, 0x48, 0x00, 0x52, 0x0a, 0x74, + 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x47, 0x74, 0x12, 0x56, 0x0a, 0x0d, 0x74, 0x69, 0x6d, + 0x65, 0x5f, 0x74, 0x69, 0x63, 0x6b, 0x5f, 0x67, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, + 0x72, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x47, + 0x54, 0x45, 0x48, 0x00, 0x52, 0x0b, 0x74, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x47, 0x74, + 0x65, 0x12, 0x55, 0x0a, 0x0c, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x79, 0x70, + 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, + 0x2e, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x48, 0x00, 0x52, 0x0b, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x42, 0x08, 0x0a, 0x06, 0x66, 0x69, 0x6c, 0x74, + 0x65, 0x72, 0x22, 0x36, 0x0a, 0x17, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x46, 0x69, 0x6c, + 0x74, 0x65, 0x72, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x47, 0x54, 0x12, 0x1b, 0x0a, + 0x09, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x74, 0x69, 0x63, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, + 0x52, 0x08, 0x74, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x22, 0x37, 0x0a, 0x18, 0x44, 0x65, + 0x6c, 0x69, 0x76, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x54, 0x69, 0x6d, 0x65, 0x54, + 0x69, 0x63, 0x6b, 0x47, 0x54, 0x45, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x74, + 0x69, 0x63, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x74, 0x69, 0x6d, 0x65, 0x54, + 0x69, 0x63, 0x6b, 0x22, 0x63, 0x0a, 0x18, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x46, 0x69, + 0x6c, 0x74, 0x65, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, + 0x47, 0x0a, 0x0d, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, + 0x18, 0x01, 0x20, 0x03, 0x28, 0x0e, 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, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x73, 0x22, 0x61, 0x0a, 0x0e, 0x53, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x39, 0x0a, 0x04, 0x63, 0x6f, + 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, + 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x64, 0x65, 0x52, + 0x04, 0x63, 0x6f, 0x64, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x61, 0x75, 0x73, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x63, 0x61, 0x75, 0x73, 0x65, 0x22, 0xac, 0x01, 0x0a, 0x0e, + 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x49, + 0x0a, 0x07, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, + 0x52, 0x07, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x12, 0x44, 0x0a, 0x05, 0x63, 0x6c, 0x6f, + 0x73, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, + 0x67, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x42, + 0x09, 0x0a, 0x07, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x59, 0x0a, 0x15, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x40, 0x0a, 0x08, 0x70, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, + 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x70, 0x63, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x22, 0x70, 0x0a, 0x15, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, + 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x38, 0x0a, + 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 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, 0x07, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x16, 0x0a, 0x14, 0x43, 0x6c, 0x6f, 0x73, 0x65, + 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, + 0xfa, 0x01, 0x0a, 0x0f, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x48, 0x0a, 0x06, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x06, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x12, 0x4a, 0x0a, + 0x07, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, + 0x52, 0x07, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x12, 0x45, 0x0a, 0x05, 0x63, 0x6c, 0x6f, + 0x73, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, + 0x67, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, + 0x42, 0x0a, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x61, 0x0a, 0x16, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x77, 0x61, 0x6c, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x77, 0x61, 0x6c, 0x4e, 0x61, 0x6d, + 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x5f, 0x73, 0x65, + 0x72, 0x76, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x70, + 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x49, 0x64, 0x22, + 0xd3, 0x01, 0x0a, 0x16, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, + 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x4e, 0x0a, 0x06, 0x72, 0x65, 0x73, + 0x75, 0x6c, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, + 0x6e, 0x67, 0x2e, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x48, + 0x00, 0x52, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x3e, 0x0a, 0x05, 0x65, 0x72, 0x72, + 0x6f, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, + 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x45, 0x72, 0x72, 0x6f, 0x72, + 0x48, 0x00, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x42, 0x0a, 0x0a, 0x08, 0x72, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xdb, 0x01, 0x0a, 0x1c, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, + 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 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, 0x1a, 0x0a, 0x08, 0x74, 0x69, 0x6d, 0x65, + 0x74, 0x69, 0x63, 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x74, 0x69, 0x6d, 0x65, + 0x74, 0x69, 0x63, 0x6b, 0x12, 0x41, 0x0a, 0x0a, 0x74, 0x78, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, + 0x78, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, + 0x2e, 0x54, 0x78, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x52, 0x0a, 0x74, 0x78, 0x6e, + 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x2a, 0x0a, 0x05, 0x65, 0x78, 0x74, 0x72, 0x61, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x41, 0x6e, 0x79, 0x52, 0x05, 0x65, 0x78, + 0x74, 0x72, 0x61, 0x22, 0x17, 0x0a, 0x15, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, + 0x75, 0x63, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa9, 0x03, 0x0a, + 0x0e, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x71, 0x0a, 0x18, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x35, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, + 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x16, 0x63, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x56, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, + 0x65, 0x72, 0x12, 0x74, 0x0a, 0x19, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x63, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x73, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, + 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, + 0x73, 0x75, 0x6d, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, + 0x17, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, + 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x73, 0x12, 0x5d, 0x0a, 0x0e, 0x63, 0x6c, 0x6f, 0x73, + 0x65, 0x5f, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x34, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x56, + 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0d, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x56, + 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x44, 0x0a, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, + 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x42, 0x09, 0x0a, + 0x07, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x16, 0x0a, 0x14, 0x43, 0x6c, 0x6f, 0x73, + 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x22, 0x59, 0x0a, 0x15, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, + 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x40, 0x0a, 0x08, 0x70, 0x63, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, + 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, + 0x6f, 0x52, 0x08, 0x70, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x22, 0x82, 0x01, 0x0a, 0x1e, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, + 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x60, + 0x0a, 0x10, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, + 0x67, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, + 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, + 0x0f, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, + 0x22, 0xd9, 0x01, 0x0a, 0x1d, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x4c, + 0x0a, 0x0e, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, + 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x0d, 0x64, + 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x4e, 0x0a, 0x0f, + 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x5f, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x18, + 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x44, + 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x52, 0x0e, 0x64, 0x65, + 0x6c, 0x69, 0x76, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x22, 0x84, 0x01, 0x0a, + 0x1f, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, + 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x61, 0x0a, 0x10, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x63, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x52, 0x0f, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x63, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x73, 0x22, 0x8f, 0x01, 0x0a, 0x1e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x43, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, + 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, 0x0a, 0x63, + 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, 0x3e, 0x0a, 0x05, 0x65, 0x72, 0x72, + 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, + 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x45, 0x72, 0x72, 0x6f, 0x72, + 0x48, 0x00, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x42, 0x0a, 0x0a, 0x08, 0x72, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x3f, 0x0a, 0x1c, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x56, 0x43, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, + 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, + 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x22, 0x40, 0x0a, 0x1d, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x56, + 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, + 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x6f, + 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x22, 0xa2, 0x04, 0x0a, 0x0f, 0x43, 0x6f, 0x6e, + 0x73, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x48, 0x0a, 0x06, + 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x73, + 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x06, + 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x12, 0x49, 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, + 0x2e, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, + 0x65, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x07, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, + 0x65, 0x12, 0x61, 0x0a, 0x0f, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x63, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x48, 0x00, 0x52, 0x0e, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x63, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x64, 0x0a, 0x10, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x76, + 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x43, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x0f, 0x63, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x56, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x5e, 0x0a, 0x0e, 0x63, 0x6c, + 0x6f, 0x73, 0x65, 0x5f, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6c, 0x6f, 0x73, + 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, + 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x0d, 0x63, 0x6c, 0x6f, + 0x73, 0x65, 0x56, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x45, 0x0a, 0x05, 0x63, 0x6c, + 0x6f, 0x73, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, + 0x6e, 0x67, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x05, 0x63, 0x6c, 0x6f, 0x73, + 0x65, 0x42, 0x0a, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x61, 0x0a, + 0x16, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x77, 0x61, 0x6c, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x77, 0x61, 0x6c, 0x4e, 0x61, + 0x6d, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x73, + 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, + 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x49, 0x64, + 0x22, 0x7b, 0x0a, 0x15, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, + 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, + 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, 0x41, 0x0a, 0x07, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 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, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x17, 0x0a, + 0x15, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x65, 0x0a, 0x21, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x41, 0x73, + 0x73, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x40, 0x0a, 0x08, 0x70, + 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, + 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, + 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x70, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x22, 0x24, 0x0a, + 0x22, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, + 0x6e, 0x61, 0x67, 0x65, 0x72, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x65, 0x0a, 0x21, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, + 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x52, 0x65, 0x6d, 0x6f, 0x76, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x40, 0x0a, 0x08, 0x70, 0x63, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, 0x6f, + 0x52, 0x08, 0x70, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x22, 0x24, 0x0a, 0x22, 0x53, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, + 0x65, 0x72, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x2a, 0x0a, 0x28, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, + 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x20, 0x0a, 0x1e, + 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x42, 0x61, 0x6c, + 0x61, 0x6e, 0x63, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x22, 0x92, + 0x01, 0x0a, 0x29, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, + 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x53, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x65, 0x0a, 0x12, + 0x62, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, + 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x42, + 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, + 0x52, 0x11, 0x62, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x22, 0xa3, 0x02, 0x0a, 0x15, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, + 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 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, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, + 0x12, 0x44, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, + 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x41, 0x0a, 0x04, 0x73, 0x74, 0x61, 0x74, 0x18, 0x06, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x53, + 0x74, 0x61, 0x74, 0x52, 0x04, 0x73, 0x74, 0x61, 0x74, 0x22, 0x87, 0x03, 0x0a, 0x15, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x53, + 0x74, 0x61, 0x74, 0x12, 0x26, 0x0a, 0x0f, 0x6d, 0x61, 0x78, 0x5f, 0x62, 0x69, 0x6e, 0x61, 0x72, + 0x79, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x6d, 0x61, + 0x78, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x69, + 0x6e, 0x73, 0x65, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x04, 0x52, 0x0c, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x65, 0x64, 0x52, 0x6f, 0x77, 0x73, + 0x12, 0x30, 0x0a, 0x14, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x62, 0x69, 0x6e, + 0x61, 0x72, 0x79, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x12, + 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x65, 0x64, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x53, 0x69, + 0x7a, 0x65, 0x12, 0x40, 0x0a, 0x1c, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x6e, 0x61, 0x6e, 0x6f, 0x73, 0x65, 0x63, 0x6f, 0x6e, + 0x64, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x1a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x4e, 0x61, 0x6e, 0x6f, 0x73, 0x65, 0x63, + 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x4d, 0x0a, 0x23, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x6d, 0x6f, 0x64, + 0x69, 0x66, 0x69, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, + 0x6e, 0x61, 0x6e, 0x6f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x20, 0x6c, 0x61, 0x73, 0x74, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x54, + 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x4e, 0x61, 0x6e, 0x6f, 0x73, 0x65, 0x63, 0x6f, + 0x6e, 0x64, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x5f, 0x63, 0x6f, + 0x75, 0x6e, 0x74, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x62, 0x69, 0x6e, + 0x6c, 0x6f, 0x67, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x12, 0x37, 0x0a, 0x18, 0x63, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x69, 0x6d, + 0x65, 0x5f, 0x74, 0x69, 0x63, 0x6b, 0x18, 0x07, 0x20, 0x01, 0x28, 0x04, 0x52, 0x15, 0x63, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x54, + 0x69, 0x63, 0x6b, 0x2a, 0xc5, 0x01, 0x0a, 0x11, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, + 0x4d, 0x65, 0x74, 0x61, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x1b, 0x50, 0x43, 0x48, + 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, + 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x25, 0x0a, 0x21, 0x50, 0x43, + 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, + 0x45, 0x5f, 0x55, 0x4e, 0x49, 0x4e, 0x49, 0x54, 0x49, 0x41, 0x4c, 0x49, 0x5a, 0x45, 0x44, 0x10, + 0x01, 0x12, 0x21, 0x0a, 0x1d, 0x50, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x4d, 0x45, + 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x49, + 0x4e, 0x47, 0x10, 0x02, 0x12, 0x20, 0x0a, 0x1c, 0x50, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, + 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x41, 0x53, 0x53, 0x49, + 0x47, 0x4e, 0x45, 0x44, 0x10, 0x03, 0x12, 0x23, 0x0a, 0x1f, 0x50, 0x43, 0x48, 0x41, 0x4e, 0x4e, + 0x45, 0x4c, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, + 0x41, 0x56, 0x41, 0x49, 0x4c, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x04, 0x2a, 0x77, 0x0a, 0x12, 0x42, + 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x12, 0x20, 0x0a, 0x1c, 0x42, 0x52, 0x4f, 0x41, 0x44, 0x43, 0x41, 0x53, 0x54, 0x5f, 0x54, + 0x41, 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, + 0x4e, 0x10, 0x00, 0x12, 0x20, 0x0a, 0x1c, 0x42, 0x52, 0x4f, 0x41, 0x44, 0x43, 0x41, 0x53, 0x54, + 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x50, 0x45, 0x4e, 0x44, + 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x1d, 0x0a, 0x19, 0x42, 0x52, 0x4f, 0x41, 0x44, 0x43, 0x41, + 0x53, 0x54, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x44, 0x4f, + 0x4e, 0x45, 0x10, 0x02, 0x2a, 0xdc, 0x03, 0x0a, 0x0d, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, + 0x6e, 0x67, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x15, 0x0a, 0x11, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, + 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x4f, 0x4b, 0x10, 0x00, 0x12, 0x24, 0x0a, + 0x20, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, + 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x45, 0x58, 0x49, 0x53, + 0x54, 0x10, 0x01, 0x12, 0x21, 0x0a, 0x1d, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, + 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x46, 0x45, + 0x4e, 0x43, 0x45, 0x44, 0x10, 0x02, 0x12, 0x1e, 0x0a, 0x1a, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, + 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x4f, 0x4e, 0x5f, 0x53, 0x48, 0x55, 0x54, + 0x44, 0x4f, 0x57, 0x4e, 0x10, 0x03, 0x12, 0x26, 0x0a, 0x22, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, + 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44, + 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x5f, 0x53, 0x45, 0x51, 0x10, 0x04, 0x12, 0x29, + 0x0a, 0x25, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, + 0x5f, 0x55, 0x4e, 0x4d, 0x41, 0x54, 0x43, 0x48, 0x45, 0x44, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x4e, + 0x45, 0x4c, 0x5f, 0x54, 0x45, 0x52, 0x4d, 0x10, 0x05, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x54, 0x52, + 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x47, 0x4e, 0x4f, + 0x52, 0x45, 0x44, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x06, 0x12, + 0x18, 0x0a, 0x14, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, + 0x45, 0x5f, 0x49, 0x4e, 0x4e, 0x45, 0x52, 0x10, 0x07, 0x12, 0x23, 0x0a, 0x1f, 0x53, 0x54, 0x52, + 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x56, 0x41, + 0x49, 0x4c, 0x44, 0x5f, 0x41, 0x52, 0x47, 0x55, 0x4d, 0x45, 0x4e, 0x54, 0x10, 0x08, 0x12, 0x26, + 0x0a, 0x22, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, + 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x45, 0x58, 0x50, + 0x49, 0x52, 0x45, 0x44, 0x10, 0x09, 0x12, 0x2c, 0x0a, 0x28, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, + 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44, + 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, 0x41, + 0x54, 0x45, 0x10, 0x0a, 0x12, 0x20, 0x0a, 0x1c, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, + 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x55, 0x4e, 0x52, 0x45, 0x43, 0x4f, 0x56, 0x45, 0x52, + 0x41, 0x42, 0x4c, 0x45, 0x10, 0x0b, 0x12, 0x1b, 0x0a, 0x16, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, + 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, + 0x10, 0xe7, 0x07, 0x2a, 0xd5, 0x01, 0x0a, 0x16, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, + 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x24, + 0x0a, 0x20, 0x53, 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, + 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, + 0x57, 0x4e, 0x10, 0x00, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f, + 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, + 0x5f, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x45, + 0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x4d, 0x45, 0x4e, 0x54, + 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x47, 0x52, 0x4f, 0x57, 0x49, 0x4e, 0x47, 0x10, 0x02, + 0x12, 0x23, 0x0a, 0x1f, 0x53, 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x41, 0x53, 0x53, 0x49, + 0x47, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x45, 0x41, + 0x4c, 0x45, 0x44, 0x10, 0x03, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54, + 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, + 0x45, 0x5f, 0x46, 0x4c, 0x55, 0x53, 0x48, 0x45, 0x44, 0x10, 0x04, 0x32, 0x89, 0x01, 0x0a, 0x19, + 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 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, 0x32, 0x84, 0x01, 0x0a, 0x1e, 0x53, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, + 0x61, 0x73, 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x62, 0x0a, 0x09, 0x42, 0x72, + 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, + 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, + 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x32, 0xa5, + 0x01, 0x0a, 0x1f, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6f, 0x72, + 0x64, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x12, 0x81, 0x01, 0x0a, 0x12, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, + 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x12, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, + 0x6e, 0x67, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, + 0x63, 0x6f, 0x76, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x32, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, + 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x32, 0xe1, 0x01, 0x0a, 0x1b, 0x53, 0x74, 0x72, 0x65, 0x61, + 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x48, 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x72, 0x53, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x60, 0x0a, 0x07, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, + 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x72, 0x6f, 0x64, 0x75, + 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, + 0x6e, 0x67, 0x2e, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x60, 0x0a, 0x07, 0x43, 0x6f, 0x6e, 0x73, + 0x75, 0x6d, 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6f, 0x6e, + 0x73, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, + 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x32, 0xbe, 0x03, 0x0a, 0x1b, 0x53, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, + 0x67, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x81, 0x01, 0x0a, 0x06, 0x41, + 0x73, 0x73, 0x69, 0x67, 0x6e, 0x12, 0x39, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, + 0x67, 0x65, 0x72, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x3a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x41, 0x73, + 0x73, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x81, + 0x01, 0x0a, 0x06, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x12, 0x39, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, + 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, + 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, + 0x65, 0x72, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x96, 0x01, 0x0a, 0x0d, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x53, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x12, 0x40, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, + 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x41, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, + 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, + 0x61, 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x33, 0x5a, 0x31, 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, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x70, 0x62, + 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_streaming_proto_rawDescOnce sync.Once + file_streaming_proto_rawDescData = file_streaming_proto_rawDesc +) + +func file_streaming_proto_rawDescGZIP() []byte { + file_streaming_proto_rawDescOnce.Do(func() { + file_streaming_proto_rawDescData = protoimpl.X.CompressGZIP(file_streaming_proto_rawDescData) + }) + return file_streaming_proto_rawDescData +} + +var file_streaming_proto_enumTypes = make([]protoimpl.EnumInfo, 4) +var file_streaming_proto_msgTypes = make([]protoimpl.MessageInfo, 53) +var file_streaming_proto_goTypes = []interface{}{ + (PChannelMetaState)(0), // 0: milvus.proto.streaming.PChannelMetaState + (BroadcastTaskState)(0), // 1: milvus.proto.streaming.BroadcastTaskState + (StreamingCode)(0), // 2: milvus.proto.streaming.StreamingCode + (SegmentAssignmentState)(0), // 3: milvus.proto.streaming.SegmentAssignmentState + (*PChannelInfo)(nil), // 4: milvus.proto.streaming.PChannelInfo + (*PChannelAssignmentLog)(nil), // 5: milvus.proto.streaming.PChannelAssignmentLog + (*PChannelMeta)(nil), // 6: milvus.proto.streaming.PChannelMeta + (*VersionPair)(nil), // 7: milvus.proto.streaming.VersionPair + (*BroadcastTask)(nil), // 8: milvus.proto.streaming.BroadcastTask + (*BroadcastRequest)(nil), // 9: milvus.proto.streaming.BroadcastRequest + (*BroadcastResponse)(nil), // 10: milvus.proto.streaming.BroadcastResponse + (*AssignmentDiscoverRequest)(nil), // 11: milvus.proto.streaming.AssignmentDiscoverRequest + (*ReportAssignmentErrorRequest)(nil), // 12: milvus.proto.streaming.ReportAssignmentErrorRequest + (*CloseAssignmentDiscoverRequest)(nil), // 13: milvus.proto.streaming.CloseAssignmentDiscoverRequest + (*AssignmentDiscoverResponse)(nil), // 14: milvus.proto.streaming.AssignmentDiscoverResponse + (*FullStreamingNodeAssignmentWithVersion)(nil), // 15: milvus.proto.streaming.FullStreamingNodeAssignmentWithVersion + (*CloseAssignmentDiscoverResponse)(nil), // 16: milvus.proto.streaming.CloseAssignmentDiscoverResponse + (*StreamingNodeInfo)(nil), // 17: milvus.proto.streaming.StreamingNodeInfo + (*StreamingNodeAssignment)(nil), // 18: milvus.proto.streaming.StreamingNodeAssignment + (*DeliverPolicy)(nil), // 19: milvus.proto.streaming.DeliverPolicy + (*DeliverFilter)(nil), // 20: milvus.proto.streaming.DeliverFilter + (*DeliverFilterTimeTickGT)(nil), // 21: milvus.proto.streaming.DeliverFilterTimeTickGT + (*DeliverFilterTimeTickGTE)(nil), // 22: milvus.proto.streaming.DeliverFilterTimeTickGTE + (*DeliverFilterMessageType)(nil), // 23: milvus.proto.streaming.DeliverFilterMessageType + (*StreamingError)(nil), // 24: milvus.proto.streaming.StreamingError + (*ProduceRequest)(nil), // 25: milvus.proto.streaming.ProduceRequest + (*CreateProducerRequest)(nil), // 26: milvus.proto.streaming.CreateProducerRequest + (*ProduceMessageRequest)(nil), // 27: milvus.proto.streaming.ProduceMessageRequest + (*CloseProducerRequest)(nil), // 28: milvus.proto.streaming.CloseProducerRequest + (*ProduceResponse)(nil), // 29: milvus.proto.streaming.ProduceResponse + (*CreateProducerResponse)(nil), // 30: milvus.proto.streaming.CreateProducerResponse + (*ProduceMessageResponse)(nil), // 31: milvus.proto.streaming.ProduceMessageResponse + (*ProduceMessageResponseResult)(nil), // 32: milvus.proto.streaming.ProduceMessageResponseResult + (*CloseProducerResponse)(nil), // 33: milvus.proto.streaming.CloseProducerResponse + (*ConsumeRequest)(nil), // 34: milvus.proto.streaming.ConsumeRequest + (*CloseConsumerRequest)(nil), // 35: milvus.proto.streaming.CloseConsumerRequest + (*CreateConsumerRequest)(nil), // 36: milvus.proto.streaming.CreateConsumerRequest + (*CreateVChannelConsumersRequest)(nil), // 37: milvus.proto.streaming.CreateVChannelConsumersRequest + (*CreateVChannelConsumerRequest)(nil), // 38: milvus.proto.streaming.CreateVChannelConsumerRequest + (*CreateVChannelConsumersResponse)(nil), // 39: milvus.proto.streaming.CreateVChannelConsumersResponse + (*CreateVChannelConsumerResponse)(nil), // 40: milvus.proto.streaming.CreateVChannelConsumerResponse + (*CloseVChannelConsumerRequest)(nil), // 41: milvus.proto.streaming.CloseVChannelConsumerRequest + (*CloseVChannelConsumerResponse)(nil), // 42: milvus.proto.streaming.CloseVChannelConsumerResponse + (*ConsumeResponse)(nil), // 43: milvus.proto.streaming.ConsumeResponse + (*CreateConsumerResponse)(nil), // 44: milvus.proto.streaming.CreateConsumerResponse + (*ConsumeMessageReponse)(nil), // 45: milvus.proto.streaming.ConsumeMessageReponse + (*CloseConsumerResponse)(nil), // 46: milvus.proto.streaming.CloseConsumerResponse + (*StreamingNodeManagerAssignRequest)(nil), // 47: milvus.proto.streaming.StreamingNodeManagerAssignRequest + (*StreamingNodeManagerAssignResponse)(nil), // 48: milvus.proto.streaming.StreamingNodeManagerAssignResponse + (*StreamingNodeManagerRemoveRequest)(nil), // 49: milvus.proto.streaming.StreamingNodeManagerRemoveRequest + (*StreamingNodeManagerRemoveResponse)(nil), // 50: milvus.proto.streaming.StreamingNodeManagerRemoveResponse + (*StreamingNodeManagerCollectStatusRequest)(nil), // 51: milvus.proto.streaming.StreamingNodeManagerCollectStatusRequest + (*StreamingNodeBalanceAttributes)(nil), // 52: milvus.proto.streaming.StreamingNodeBalanceAttributes + (*StreamingNodeManagerCollectStatusResponse)(nil), // 53: milvus.proto.streaming.StreamingNodeManagerCollectStatusResponse + (*SegmentAssignmentMeta)(nil), // 54: milvus.proto.streaming.SegmentAssignmentMeta + (*SegmentAssignmentStat)(nil), // 55: milvus.proto.streaming.SegmentAssignmentStat + nil, // 56: milvus.proto.streaming.BroadcastResponse.ResultsEntry + (*messagespb.Message)(nil), // 57: milvus.proto.messages.Message + (*emptypb.Empty)(nil), // 58: google.protobuf.Empty + (*messagespb.MessageID)(nil), // 59: milvus.proto.messages.MessageID + (messagespb.MessageType)(0), // 60: milvus.proto.messages.MessageType + (*messagespb.TxnContext)(nil), // 61: milvus.proto.messages.TxnContext + (*anypb.Any)(nil), // 62: google.protobuf.Any + (*messagespb.ImmutableMessage)(nil), // 63: milvus.proto.messages.ImmutableMessage + (*milvuspb.GetComponentStatesRequest)(nil), // 64: milvus.proto.milvus.GetComponentStatesRequest + (*milvuspb.ComponentStates)(nil), // 65: milvus.proto.milvus.ComponentStates +} +var file_streaming_proto_depIdxs = []int32{ + 17, // 0: milvus.proto.streaming.PChannelAssignmentLog.node:type_name -> milvus.proto.streaming.StreamingNodeInfo + 4, // 1: milvus.proto.streaming.PChannelMeta.channel:type_name -> milvus.proto.streaming.PChannelInfo + 17, // 2: milvus.proto.streaming.PChannelMeta.node:type_name -> milvus.proto.streaming.StreamingNodeInfo + 0, // 3: milvus.proto.streaming.PChannelMeta.state:type_name -> milvus.proto.streaming.PChannelMetaState + 5, // 4: milvus.proto.streaming.PChannelMeta.histories:type_name -> milvus.proto.streaming.PChannelAssignmentLog + 57, // 5: milvus.proto.streaming.BroadcastTask.message:type_name -> milvus.proto.messages.Message + 1, // 6: milvus.proto.streaming.BroadcastTask.state:type_name -> milvus.proto.streaming.BroadcastTaskState + 57, // 7: milvus.proto.streaming.BroadcastRequest.message:type_name -> milvus.proto.messages.Message + 56, // 8: milvus.proto.streaming.BroadcastResponse.results:type_name -> milvus.proto.streaming.BroadcastResponse.ResultsEntry + 12, // 9: milvus.proto.streaming.AssignmentDiscoverRequest.report_error:type_name -> milvus.proto.streaming.ReportAssignmentErrorRequest + 13, // 10: milvus.proto.streaming.AssignmentDiscoverRequest.close:type_name -> milvus.proto.streaming.CloseAssignmentDiscoverRequest + 4, // 11: milvus.proto.streaming.ReportAssignmentErrorRequest.pchannel:type_name -> milvus.proto.streaming.PChannelInfo + 24, // 12: milvus.proto.streaming.ReportAssignmentErrorRequest.err:type_name -> milvus.proto.streaming.StreamingError + 15, // 13: milvus.proto.streaming.AssignmentDiscoverResponse.full_assignment:type_name -> milvus.proto.streaming.FullStreamingNodeAssignmentWithVersion + 16, // 14: milvus.proto.streaming.AssignmentDiscoverResponse.close:type_name -> milvus.proto.streaming.CloseAssignmentDiscoverResponse + 7, // 15: milvus.proto.streaming.FullStreamingNodeAssignmentWithVersion.version:type_name -> milvus.proto.streaming.VersionPair + 18, // 16: milvus.proto.streaming.FullStreamingNodeAssignmentWithVersion.assignments:type_name -> milvus.proto.streaming.StreamingNodeAssignment + 17, // 17: milvus.proto.streaming.StreamingNodeAssignment.node:type_name -> milvus.proto.streaming.StreamingNodeInfo + 4, // 18: milvus.proto.streaming.StreamingNodeAssignment.channels:type_name -> milvus.proto.streaming.PChannelInfo + 58, // 19: milvus.proto.streaming.DeliverPolicy.all:type_name -> google.protobuf.Empty + 58, // 20: milvus.proto.streaming.DeliverPolicy.latest:type_name -> google.protobuf.Empty + 59, // 21: milvus.proto.streaming.DeliverPolicy.start_from:type_name -> milvus.proto.messages.MessageID + 59, // 22: milvus.proto.streaming.DeliverPolicy.start_after:type_name -> milvus.proto.messages.MessageID + 21, // 23: milvus.proto.streaming.DeliverFilter.time_tick_gt:type_name -> milvus.proto.streaming.DeliverFilterTimeTickGT + 22, // 24: milvus.proto.streaming.DeliverFilter.time_tick_gte:type_name -> milvus.proto.streaming.DeliverFilterTimeTickGTE + 23, // 25: milvus.proto.streaming.DeliverFilter.message_type:type_name -> milvus.proto.streaming.DeliverFilterMessageType + 60, // 26: milvus.proto.streaming.DeliverFilterMessageType.message_types:type_name -> milvus.proto.messages.MessageType + 2, // 27: milvus.proto.streaming.StreamingError.code:type_name -> milvus.proto.streaming.StreamingCode + 27, // 28: milvus.proto.streaming.ProduceRequest.produce:type_name -> milvus.proto.streaming.ProduceMessageRequest + 28, // 29: milvus.proto.streaming.ProduceRequest.close:type_name -> milvus.proto.streaming.CloseProducerRequest + 4, // 30: milvus.proto.streaming.CreateProducerRequest.pchannel:type_name -> milvus.proto.streaming.PChannelInfo + 57, // 31: milvus.proto.streaming.ProduceMessageRequest.message:type_name -> milvus.proto.messages.Message + 30, // 32: milvus.proto.streaming.ProduceResponse.create:type_name -> milvus.proto.streaming.CreateProducerResponse + 31, // 33: milvus.proto.streaming.ProduceResponse.produce:type_name -> milvus.proto.streaming.ProduceMessageResponse + 33, // 34: milvus.proto.streaming.ProduceResponse.close:type_name -> milvus.proto.streaming.CloseProducerResponse + 32, // 35: milvus.proto.streaming.ProduceMessageResponse.result:type_name -> milvus.proto.streaming.ProduceMessageResponseResult + 24, // 36: milvus.proto.streaming.ProduceMessageResponse.error:type_name -> milvus.proto.streaming.StreamingError + 59, // 37: milvus.proto.streaming.ProduceMessageResponseResult.id:type_name -> milvus.proto.messages.MessageID + 61, // 38: milvus.proto.streaming.ProduceMessageResponseResult.txnContext:type_name -> milvus.proto.messages.TxnContext + 62, // 39: milvus.proto.streaming.ProduceMessageResponseResult.extra:type_name -> google.protobuf.Any + 38, // 40: milvus.proto.streaming.ConsumeRequest.create_vchannel_consumer:type_name -> milvus.proto.streaming.CreateVChannelConsumerRequest + 37, // 41: milvus.proto.streaming.ConsumeRequest.create_vchannel_consumers:type_name -> milvus.proto.streaming.CreateVChannelConsumersRequest + 41, // 42: milvus.proto.streaming.ConsumeRequest.close_vchannel:type_name -> milvus.proto.streaming.CloseVChannelConsumerRequest + 35, // 43: milvus.proto.streaming.ConsumeRequest.close:type_name -> milvus.proto.streaming.CloseConsumerRequest + 4, // 44: milvus.proto.streaming.CreateConsumerRequest.pchannel:type_name -> milvus.proto.streaming.PChannelInfo + 38, // 45: milvus.proto.streaming.CreateVChannelConsumersRequest.create_vchannels:type_name -> milvus.proto.streaming.CreateVChannelConsumerRequest + 19, // 46: milvus.proto.streaming.CreateVChannelConsumerRequest.deliver_policy:type_name -> milvus.proto.streaming.DeliverPolicy + 20, // 47: milvus.proto.streaming.CreateVChannelConsumerRequest.deliver_filters:type_name -> milvus.proto.streaming.DeliverFilter + 40, // 48: milvus.proto.streaming.CreateVChannelConsumersResponse.create_vchannels:type_name -> milvus.proto.streaming.CreateVChannelConsumerResponse + 24, // 49: milvus.proto.streaming.CreateVChannelConsumerResponse.error:type_name -> milvus.proto.streaming.StreamingError + 44, // 50: milvus.proto.streaming.ConsumeResponse.create:type_name -> milvus.proto.streaming.CreateConsumerResponse + 45, // 51: milvus.proto.streaming.ConsumeResponse.consume:type_name -> milvus.proto.streaming.ConsumeMessageReponse + 40, // 52: milvus.proto.streaming.ConsumeResponse.create_vchannel:type_name -> milvus.proto.streaming.CreateVChannelConsumerResponse + 39, // 53: milvus.proto.streaming.ConsumeResponse.create_vchannels:type_name -> milvus.proto.streaming.CreateVChannelConsumersResponse + 42, // 54: milvus.proto.streaming.ConsumeResponse.close_vchannel:type_name -> milvus.proto.streaming.CloseVChannelConsumerResponse + 46, // 55: milvus.proto.streaming.ConsumeResponse.close:type_name -> milvus.proto.streaming.CloseConsumerResponse + 63, // 56: milvus.proto.streaming.ConsumeMessageReponse.message:type_name -> milvus.proto.messages.ImmutableMessage + 4, // 57: milvus.proto.streaming.StreamingNodeManagerAssignRequest.pchannel:type_name -> milvus.proto.streaming.PChannelInfo + 4, // 58: milvus.proto.streaming.StreamingNodeManagerRemoveRequest.pchannel:type_name -> milvus.proto.streaming.PChannelInfo + 52, // 59: milvus.proto.streaming.StreamingNodeManagerCollectStatusResponse.balance_attributes:type_name -> milvus.proto.streaming.StreamingNodeBalanceAttributes + 3, // 60: milvus.proto.streaming.SegmentAssignmentMeta.state:type_name -> milvus.proto.streaming.SegmentAssignmentState + 55, // 61: milvus.proto.streaming.SegmentAssignmentMeta.stat:type_name -> milvus.proto.streaming.SegmentAssignmentStat + 32, // 62: milvus.proto.streaming.BroadcastResponse.ResultsEntry.value:type_name -> milvus.proto.streaming.ProduceMessageResponseResult + 64, // 63: milvus.proto.streaming.StreamingNodeStateService.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest + 9, // 64: milvus.proto.streaming.StreamingCoordBroadcastService.Broadcast:input_type -> milvus.proto.streaming.BroadcastRequest + 11, // 65: milvus.proto.streaming.StreamingCoordAssignmentService.AssignmentDiscover:input_type -> milvus.proto.streaming.AssignmentDiscoverRequest + 25, // 66: milvus.proto.streaming.StreamingNodeHandlerService.Produce:input_type -> milvus.proto.streaming.ProduceRequest + 34, // 67: milvus.proto.streaming.StreamingNodeHandlerService.Consume:input_type -> milvus.proto.streaming.ConsumeRequest + 47, // 68: milvus.proto.streaming.StreamingNodeManagerService.Assign:input_type -> milvus.proto.streaming.StreamingNodeManagerAssignRequest + 49, // 69: milvus.proto.streaming.StreamingNodeManagerService.Remove:input_type -> milvus.proto.streaming.StreamingNodeManagerRemoveRequest + 51, // 70: milvus.proto.streaming.StreamingNodeManagerService.CollectStatus:input_type -> milvus.proto.streaming.StreamingNodeManagerCollectStatusRequest + 65, // 71: milvus.proto.streaming.StreamingNodeStateService.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates + 10, // 72: milvus.proto.streaming.StreamingCoordBroadcastService.Broadcast:output_type -> milvus.proto.streaming.BroadcastResponse + 14, // 73: milvus.proto.streaming.StreamingCoordAssignmentService.AssignmentDiscover:output_type -> milvus.proto.streaming.AssignmentDiscoverResponse + 29, // 74: milvus.proto.streaming.StreamingNodeHandlerService.Produce:output_type -> milvus.proto.streaming.ProduceResponse + 43, // 75: milvus.proto.streaming.StreamingNodeHandlerService.Consume:output_type -> milvus.proto.streaming.ConsumeResponse + 48, // 76: milvus.proto.streaming.StreamingNodeManagerService.Assign:output_type -> milvus.proto.streaming.StreamingNodeManagerAssignResponse + 50, // 77: milvus.proto.streaming.StreamingNodeManagerService.Remove:output_type -> milvus.proto.streaming.StreamingNodeManagerRemoveResponse + 53, // 78: milvus.proto.streaming.StreamingNodeManagerService.CollectStatus:output_type -> milvus.proto.streaming.StreamingNodeManagerCollectStatusResponse + 71, // [71:79] is the sub-list for method output_type + 63, // [63:71] is the sub-list for method input_type + 63, // [63:63] is the sub-list for extension type_name + 63, // [63:63] is the sub-list for extension extendee + 0, // [0:63] is the sub-list for field type_name +} + +func init() { file_streaming_proto_init() } +func file_streaming_proto_init() { + if File_streaming_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_streaming_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PChannelInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PChannelAssignmentLog); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PChannelMeta); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*VersionPair); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BroadcastTask); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BroadcastRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BroadcastResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AssignmentDiscoverRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ReportAssignmentErrorRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CloseAssignmentDiscoverRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AssignmentDiscoverResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*FullStreamingNodeAssignmentWithVersion); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CloseAssignmentDiscoverResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*StreamingNodeInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*StreamingNodeAssignment); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DeliverPolicy); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DeliverFilter); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DeliverFilterTimeTickGT); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DeliverFilterTimeTickGTE); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DeliverFilterMessageType); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*StreamingError); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ProduceRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CreateProducerRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ProduceMessageRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CloseProducerRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ProduceResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CreateProducerResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ProduceMessageResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ProduceMessageResponseResult); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CloseProducerResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ConsumeRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CloseConsumerRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CreateConsumerRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CreateVChannelConsumersRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CreateVChannelConsumerRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CreateVChannelConsumersResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CreateVChannelConsumerResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CloseVChannelConsumerRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CloseVChannelConsumerResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ConsumeResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CreateConsumerResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ConsumeMessageReponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CloseConsumerResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*StreamingNodeManagerAssignRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*StreamingNodeManagerAssignResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*StreamingNodeManagerRemoveRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*StreamingNodeManagerRemoveResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*StreamingNodeManagerCollectStatusRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*StreamingNodeBalanceAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*StreamingNodeManagerCollectStatusResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[50].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentAssignmentMeta); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentAssignmentStat); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + file_streaming_proto_msgTypes[7].OneofWrappers = []interface{}{ + (*AssignmentDiscoverRequest_ReportError)(nil), + (*AssignmentDiscoverRequest_Close)(nil), + } + file_streaming_proto_msgTypes[10].OneofWrappers = []interface{}{ + (*AssignmentDiscoverResponse_FullAssignment)(nil), + (*AssignmentDiscoverResponse_Close)(nil), + } + file_streaming_proto_msgTypes[15].OneofWrappers = []interface{}{ + (*DeliverPolicy_All)(nil), + (*DeliverPolicy_Latest)(nil), + (*DeliverPolicy_StartFrom)(nil), + (*DeliverPolicy_StartAfter)(nil), + } + file_streaming_proto_msgTypes[16].OneofWrappers = []interface{}{ + (*DeliverFilter_TimeTickGt)(nil), + (*DeliverFilter_TimeTickGte)(nil), + (*DeliverFilter_MessageType)(nil), + } + file_streaming_proto_msgTypes[21].OneofWrappers = []interface{}{ + (*ProduceRequest_Produce)(nil), + (*ProduceRequest_Close)(nil), + } + file_streaming_proto_msgTypes[25].OneofWrappers = []interface{}{ + (*ProduceResponse_Create)(nil), + (*ProduceResponse_Produce)(nil), + (*ProduceResponse_Close)(nil), + } + file_streaming_proto_msgTypes[27].OneofWrappers = []interface{}{ + (*ProduceMessageResponse_Result)(nil), + (*ProduceMessageResponse_Error)(nil), + } + file_streaming_proto_msgTypes[30].OneofWrappers = []interface{}{ + (*ConsumeRequest_CreateVchannelConsumer)(nil), + (*ConsumeRequest_CreateVchannelConsumers)(nil), + (*ConsumeRequest_CloseVchannel)(nil), + (*ConsumeRequest_Close)(nil), + } + file_streaming_proto_msgTypes[36].OneofWrappers = []interface{}{ + (*CreateVChannelConsumerResponse_ConsumerId)(nil), + (*CreateVChannelConsumerResponse_Error)(nil), + } + file_streaming_proto_msgTypes[39].OneofWrappers = []interface{}{ + (*ConsumeResponse_Create)(nil), + (*ConsumeResponse_Consume)(nil), + (*ConsumeResponse_CreateVchannel)(nil), + (*ConsumeResponse_CreateVchannels)(nil), + (*ConsumeResponse_CloseVchannel)(nil), + (*ConsumeResponse_Close)(nil), + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_streaming_proto_rawDesc, + NumEnums: 4, + NumMessages: 53, + NumExtensions: 0, + NumServices: 5, + }, + GoTypes: file_streaming_proto_goTypes, + DependencyIndexes: file_streaming_proto_depIdxs, + EnumInfos: file_streaming_proto_enumTypes, + MessageInfos: file_streaming_proto_msgTypes, + }.Build() + File_streaming_proto = out.File + file_streaming_proto_rawDesc = nil + file_streaming_proto_goTypes = nil + file_streaming_proto_depIdxs = nil +} diff --git a/pkg/proto/streamingpb/streaming_grpc.pb.go b/pkg/proto/streamingpb/streaming_grpc.pb.go new file mode 100644 index 0000000000000..1d3d8beef70f4 --- /dev/null +++ b/pkg/proto/streamingpb/streaming_grpc.pb.go @@ -0,0 +1,731 @@ +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. +// versions: +// - protoc-gen-go-grpc v1.3.0 +// - protoc v3.21.4 +// source: streaming.proto + +package streamingpb + +import ( + context "context" + milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" +) + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +// Requires gRPC-Go v1.32.0 or later. +const _ = grpc.SupportPackageIsVersion7 + +const ( + StreamingNodeStateService_GetComponentStates_FullMethodName = "/milvus.proto.streaming.StreamingNodeStateService/GetComponentStates" +) + +// StreamingNodeStateServiceClient is the client API for StreamingNodeStateService service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +type StreamingNodeStateServiceClient interface { + GetComponentStates(ctx context.Context, in *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) +} + +type streamingNodeStateServiceClient struct { + cc grpc.ClientConnInterface +} + +func NewStreamingNodeStateServiceClient(cc grpc.ClientConnInterface) StreamingNodeStateServiceClient { + return &streamingNodeStateServiceClient{cc} +} + +func (c *streamingNodeStateServiceClient) GetComponentStates(ctx context.Context, in *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) { + out := new(milvuspb.ComponentStates) + err := c.cc.Invoke(ctx, StreamingNodeStateService_GetComponentStates_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// StreamingNodeStateServiceServer is the server API for StreamingNodeStateService service. +// All implementations should embed UnimplementedStreamingNodeStateServiceServer +// for forward compatibility +type StreamingNodeStateServiceServer interface { + GetComponentStates(context.Context, *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) +} + +// UnimplementedStreamingNodeStateServiceServer should be embedded to have forward compatible implementations. +type UnimplementedStreamingNodeStateServiceServer struct { +} + +func (UnimplementedStreamingNodeStateServiceServer) GetComponentStates(context.Context, *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetComponentStates not implemented") +} + +// UnsafeStreamingNodeStateServiceServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to StreamingNodeStateServiceServer will +// result in compilation errors. +type UnsafeStreamingNodeStateServiceServer interface { + mustEmbedUnimplementedStreamingNodeStateServiceServer() +} + +func RegisterStreamingNodeStateServiceServer(s grpc.ServiceRegistrar, srv StreamingNodeStateServiceServer) { + s.RegisterService(&StreamingNodeStateService_ServiceDesc, srv) +} + +func _StreamingNodeStateService_GetComponentStates_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.GetComponentStatesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(StreamingNodeStateServiceServer).GetComponentStates(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: StreamingNodeStateService_GetComponentStates_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(StreamingNodeStateServiceServer).GetComponentStates(ctx, req.(*milvuspb.GetComponentStatesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +// StreamingNodeStateService_ServiceDesc is the grpc.ServiceDesc for StreamingNodeStateService service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var StreamingNodeStateService_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "milvus.proto.streaming.StreamingNodeStateService", + HandlerType: (*StreamingNodeStateServiceServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "GetComponentStates", + Handler: _StreamingNodeStateService_GetComponentStates_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "streaming.proto", +} + +const ( + StreamingCoordBroadcastService_Broadcast_FullMethodName = "/milvus.proto.streaming.StreamingCoordBroadcastService/Broadcast" +) + +// StreamingCoordBroadcastServiceClient is the client API for StreamingCoordBroadcastService service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +type StreamingCoordBroadcastServiceClient interface { + // Broadcast receives broadcast messages from other component and make sure that the message is broadcast to all wal. + // It performs an atomic broadcast to all wal, achieve eventual consistency. + Broadcast(ctx context.Context, in *BroadcastRequest, opts ...grpc.CallOption) (*BroadcastResponse, error) +} + +type streamingCoordBroadcastServiceClient struct { + cc grpc.ClientConnInterface +} + +func NewStreamingCoordBroadcastServiceClient(cc grpc.ClientConnInterface) StreamingCoordBroadcastServiceClient { + return &streamingCoordBroadcastServiceClient{cc} +} + +func (c *streamingCoordBroadcastServiceClient) Broadcast(ctx context.Context, in *BroadcastRequest, opts ...grpc.CallOption) (*BroadcastResponse, error) { + out := new(BroadcastResponse) + err := c.cc.Invoke(ctx, StreamingCoordBroadcastService_Broadcast_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// StreamingCoordBroadcastServiceServer is the server API for StreamingCoordBroadcastService service. +// All implementations should embed UnimplementedStreamingCoordBroadcastServiceServer +// for forward compatibility +type StreamingCoordBroadcastServiceServer interface { + // Broadcast receives broadcast messages from other component and make sure that the message is broadcast to all wal. + // It performs an atomic broadcast to all wal, achieve eventual consistency. + Broadcast(context.Context, *BroadcastRequest) (*BroadcastResponse, error) +} + +// UnimplementedStreamingCoordBroadcastServiceServer should be embedded to have forward compatible implementations. +type UnimplementedStreamingCoordBroadcastServiceServer struct { +} + +func (UnimplementedStreamingCoordBroadcastServiceServer) Broadcast(context.Context, *BroadcastRequest) (*BroadcastResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method Broadcast not implemented") +} + +// UnsafeStreamingCoordBroadcastServiceServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to StreamingCoordBroadcastServiceServer will +// result in compilation errors. +type UnsafeStreamingCoordBroadcastServiceServer interface { + mustEmbedUnimplementedStreamingCoordBroadcastServiceServer() +} + +func RegisterStreamingCoordBroadcastServiceServer(s grpc.ServiceRegistrar, srv StreamingCoordBroadcastServiceServer) { + s.RegisterService(&StreamingCoordBroadcastService_ServiceDesc, srv) +} + +func _StreamingCoordBroadcastService_Broadcast_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(BroadcastRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(StreamingCoordBroadcastServiceServer).Broadcast(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: StreamingCoordBroadcastService_Broadcast_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(StreamingCoordBroadcastServiceServer).Broadcast(ctx, req.(*BroadcastRequest)) + } + return interceptor(ctx, in, info, handler) +} + +// StreamingCoordBroadcastService_ServiceDesc is the grpc.ServiceDesc for StreamingCoordBroadcastService service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var StreamingCoordBroadcastService_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "milvus.proto.streaming.StreamingCoordBroadcastService", + HandlerType: (*StreamingCoordBroadcastServiceServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "Broadcast", + Handler: _StreamingCoordBroadcastService_Broadcast_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "streaming.proto", +} + +const ( + StreamingCoordAssignmentService_AssignmentDiscover_FullMethodName = "/milvus.proto.streaming.StreamingCoordAssignmentService/AssignmentDiscover" +) + +// StreamingCoordAssignmentServiceClient is the client API for StreamingCoordAssignmentService service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +type StreamingCoordAssignmentServiceClient interface { + // AssignmentDiscover is used to discover all log nodes managed by the + // streamingcoord. Channel assignment information will be pushed to client + // by stream. + AssignmentDiscover(ctx context.Context, opts ...grpc.CallOption) (StreamingCoordAssignmentService_AssignmentDiscoverClient, error) +} + +type streamingCoordAssignmentServiceClient struct { + cc grpc.ClientConnInterface +} + +func NewStreamingCoordAssignmentServiceClient(cc grpc.ClientConnInterface) StreamingCoordAssignmentServiceClient { + return &streamingCoordAssignmentServiceClient{cc} +} + +func (c *streamingCoordAssignmentServiceClient) AssignmentDiscover(ctx context.Context, opts ...grpc.CallOption) (StreamingCoordAssignmentService_AssignmentDiscoverClient, error) { + stream, err := c.cc.NewStream(ctx, &StreamingCoordAssignmentService_ServiceDesc.Streams[0], StreamingCoordAssignmentService_AssignmentDiscover_FullMethodName, opts...) + if err != nil { + return nil, err + } + x := &streamingCoordAssignmentServiceAssignmentDiscoverClient{stream} + return x, nil +} + +type StreamingCoordAssignmentService_AssignmentDiscoverClient interface { + Send(*AssignmentDiscoverRequest) error + Recv() (*AssignmentDiscoverResponse, error) + grpc.ClientStream +} + +type streamingCoordAssignmentServiceAssignmentDiscoverClient struct { + grpc.ClientStream +} + +func (x *streamingCoordAssignmentServiceAssignmentDiscoverClient) Send(m *AssignmentDiscoverRequest) error { + return x.ClientStream.SendMsg(m) +} + +func (x *streamingCoordAssignmentServiceAssignmentDiscoverClient) Recv() (*AssignmentDiscoverResponse, error) { + m := new(AssignmentDiscoverResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +// StreamingCoordAssignmentServiceServer is the server API for StreamingCoordAssignmentService service. +// All implementations should embed UnimplementedStreamingCoordAssignmentServiceServer +// for forward compatibility +type StreamingCoordAssignmentServiceServer interface { + // AssignmentDiscover is used to discover all log nodes managed by the + // streamingcoord. Channel assignment information will be pushed to client + // by stream. + AssignmentDiscover(StreamingCoordAssignmentService_AssignmentDiscoverServer) error +} + +// UnimplementedStreamingCoordAssignmentServiceServer should be embedded to have forward compatible implementations. +type UnimplementedStreamingCoordAssignmentServiceServer struct { +} + +func (UnimplementedStreamingCoordAssignmentServiceServer) AssignmentDiscover(StreamingCoordAssignmentService_AssignmentDiscoverServer) error { + return status.Errorf(codes.Unimplemented, "method AssignmentDiscover not implemented") +} + +// UnsafeStreamingCoordAssignmentServiceServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to StreamingCoordAssignmentServiceServer will +// result in compilation errors. +type UnsafeStreamingCoordAssignmentServiceServer interface { + mustEmbedUnimplementedStreamingCoordAssignmentServiceServer() +} + +func RegisterStreamingCoordAssignmentServiceServer(s grpc.ServiceRegistrar, srv StreamingCoordAssignmentServiceServer) { + s.RegisterService(&StreamingCoordAssignmentService_ServiceDesc, srv) +} + +func _StreamingCoordAssignmentService_AssignmentDiscover_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(StreamingCoordAssignmentServiceServer).AssignmentDiscover(&streamingCoordAssignmentServiceAssignmentDiscoverServer{stream}) +} + +type StreamingCoordAssignmentService_AssignmentDiscoverServer interface { + Send(*AssignmentDiscoverResponse) error + Recv() (*AssignmentDiscoverRequest, error) + grpc.ServerStream +} + +type streamingCoordAssignmentServiceAssignmentDiscoverServer struct { + grpc.ServerStream +} + +func (x *streamingCoordAssignmentServiceAssignmentDiscoverServer) Send(m *AssignmentDiscoverResponse) error { + return x.ServerStream.SendMsg(m) +} + +func (x *streamingCoordAssignmentServiceAssignmentDiscoverServer) Recv() (*AssignmentDiscoverRequest, error) { + m := new(AssignmentDiscoverRequest) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +// StreamingCoordAssignmentService_ServiceDesc is the grpc.ServiceDesc for StreamingCoordAssignmentService service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var StreamingCoordAssignmentService_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "milvus.proto.streaming.StreamingCoordAssignmentService", + HandlerType: (*StreamingCoordAssignmentServiceServer)(nil), + Methods: []grpc.MethodDesc{}, + Streams: []grpc.StreamDesc{ + { + StreamName: "AssignmentDiscover", + Handler: _StreamingCoordAssignmentService_AssignmentDiscover_Handler, + ServerStreams: true, + ClientStreams: true, + }, + }, + Metadata: "streaming.proto", +} + +const ( + StreamingNodeHandlerService_Produce_FullMethodName = "/milvus.proto.streaming.StreamingNodeHandlerService/Produce" + StreamingNodeHandlerService_Consume_FullMethodName = "/milvus.proto.streaming.StreamingNodeHandlerService/Consume" +) + +// StreamingNodeHandlerServiceClient is the client API for StreamingNodeHandlerService service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +type StreamingNodeHandlerServiceClient interface { + // Produce is a bi-directional streaming RPC to send messages to a channel. + // All messages sent to a channel will be assigned a unique messageID. + // The messageID is used to identify the message in the channel. + // The messageID isn't promised to be monotonous increasing with the + // sequence of responsing. Error: If channel isn't assign to this log node, + // the RPC will return error CHANNEL_NOT_EXIST. If channel is moving away to + // other log node, the RPC will return error CHANNEL_FENCED. + Produce(ctx context.Context, opts ...grpc.CallOption) (StreamingNodeHandlerService_ProduceClient, error) + // Consume is a server streaming RPC to receive messages from a channel. + // All message after given startMessageID and excluding will be sent to the + // client by stream. If no more message in the channel, the stream will be + // blocked until new message coming. Error: If channel isn't assign to this + // log node, the RPC will return error CHANNEL_NOT_EXIST. If channel is + // moving away to other log node, the RPC will return error CHANNEL_FENCED. + Consume(ctx context.Context, opts ...grpc.CallOption) (StreamingNodeHandlerService_ConsumeClient, error) +} + +type streamingNodeHandlerServiceClient struct { + cc grpc.ClientConnInterface +} + +func NewStreamingNodeHandlerServiceClient(cc grpc.ClientConnInterface) StreamingNodeHandlerServiceClient { + return &streamingNodeHandlerServiceClient{cc} +} + +func (c *streamingNodeHandlerServiceClient) Produce(ctx context.Context, opts ...grpc.CallOption) (StreamingNodeHandlerService_ProduceClient, error) { + stream, err := c.cc.NewStream(ctx, &StreamingNodeHandlerService_ServiceDesc.Streams[0], StreamingNodeHandlerService_Produce_FullMethodName, opts...) + if err != nil { + return nil, err + } + x := &streamingNodeHandlerServiceProduceClient{stream} + return x, nil +} + +type StreamingNodeHandlerService_ProduceClient interface { + Send(*ProduceRequest) error + Recv() (*ProduceResponse, error) + grpc.ClientStream +} + +type streamingNodeHandlerServiceProduceClient struct { + grpc.ClientStream +} + +func (x *streamingNodeHandlerServiceProduceClient) Send(m *ProduceRequest) error { + return x.ClientStream.SendMsg(m) +} + +func (x *streamingNodeHandlerServiceProduceClient) Recv() (*ProduceResponse, error) { + m := new(ProduceResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *streamingNodeHandlerServiceClient) Consume(ctx context.Context, opts ...grpc.CallOption) (StreamingNodeHandlerService_ConsumeClient, error) { + stream, err := c.cc.NewStream(ctx, &StreamingNodeHandlerService_ServiceDesc.Streams[1], StreamingNodeHandlerService_Consume_FullMethodName, opts...) + if err != nil { + return nil, err + } + x := &streamingNodeHandlerServiceConsumeClient{stream} + return x, nil +} + +type StreamingNodeHandlerService_ConsumeClient interface { + Send(*ConsumeRequest) error + Recv() (*ConsumeResponse, error) + grpc.ClientStream +} + +type streamingNodeHandlerServiceConsumeClient struct { + grpc.ClientStream +} + +func (x *streamingNodeHandlerServiceConsumeClient) Send(m *ConsumeRequest) error { + return x.ClientStream.SendMsg(m) +} + +func (x *streamingNodeHandlerServiceConsumeClient) Recv() (*ConsumeResponse, error) { + m := new(ConsumeResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +// StreamingNodeHandlerServiceServer is the server API for StreamingNodeHandlerService service. +// All implementations should embed UnimplementedStreamingNodeHandlerServiceServer +// for forward compatibility +type StreamingNodeHandlerServiceServer interface { + // Produce is a bi-directional streaming RPC to send messages to a channel. + // All messages sent to a channel will be assigned a unique messageID. + // The messageID is used to identify the message in the channel. + // The messageID isn't promised to be monotonous increasing with the + // sequence of responsing. Error: If channel isn't assign to this log node, + // the RPC will return error CHANNEL_NOT_EXIST. If channel is moving away to + // other log node, the RPC will return error CHANNEL_FENCED. + Produce(StreamingNodeHandlerService_ProduceServer) error + // Consume is a server streaming RPC to receive messages from a channel. + // All message after given startMessageID and excluding will be sent to the + // client by stream. If no more message in the channel, the stream will be + // blocked until new message coming. Error: If channel isn't assign to this + // log node, the RPC will return error CHANNEL_NOT_EXIST. If channel is + // moving away to other log node, the RPC will return error CHANNEL_FENCED. + Consume(StreamingNodeHandlerService_ConsumeServer) error +} + +// UnimplementedStreamingNodeHandlerServiceServer should be embedded to have forward compatible implementations. +type UnimplementedStreamingNodeHandlerServiceServer struct { +} + +func (UnimplementedStreamingNodeHandlerServiceServer) Produce(StreamingNodeHandlerService_ProduceServer) error { + return status.Errorf(codes.Unimplemented, "method Produce not implemented") +} +func (UnimplementedStreamingNodeHandlerServiceServer) Consume(StreamingNodeHandlerService_ConsumeServer) error { + return status.Errorf(codes.Unimplemented, "method Consume not implemented") +} + +// UnsafeStreamingNodeHandlerServiceServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to StreamingNodeHandlerServiceServer will +// result in compilation errors. +type UnsafeStreamingNodeHandlerServiceServer interface { + mustEmbedUnimplementedStreamingNodeHandlerServiceServer() +} + +func RegisterStreamingNodeHandlerServiceServer(s grpc.ServiceRegistrar, srv StreamingNodeHandlerServiceServer) { + s.RegisterService(&StreamingNodeHandlerService_ServiceDesc, srv) +} + +func _StreamingNodeHandlerService_Produce_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(StreamingNodeHandlerServiceServer).Produce(&streamingNodeHandlerServiceProduceServer{stream}) +} + +type StreamingNodeHandlerService_ProduceServer interface { + Send(*ProduceResponse) error + Recv() (*ProduceRequest, error) + grpc.ServerStream +} + +type streamingNodeHandlerServiceProduceServer struct { + grpc.ServerStream +} + +func (x *streamingNodeHandlerServiceProduceServer) Send(m *ProduceResponse) error { + return x.ServerStream.SendMsg(m) +} + +func (x *streamingNodeHandlerServiceProduceServer) Recv() (*ProduceRequest, error) { + m := new(ProduceRequest) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func _StreamingNodeHandlerService_Consume_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(StreamingNodeHandlerServiceServer).Consume(&streamingNodeHandlerServiceConsumeServer{stream}) +} + +type StreamingNodeHandlerService_ConsumeServer interface { + Send(*ConsumeResponse) error + Recv() (*ConsumeRequest, error) + grpc.ServerStream +} + +type streamingNodeHandlerServiceConsumeServer struct { + grpc.ServerStream +} + +func (x *streamingNodeHandlerServiceConsumeServer) Send(m *ConsumeResponse) error { + return x.ServerStream.SendMsg(m) +} + +func (x *streamingNodeHandlerServiceConsumeServer) Recv() (*ConsumeRequest, error) { + m := new(ConsumeRequest) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +// StreamingNodeHandlerService_ServiceDesc is the grpc.ServiceDesc for StreamingNodeHandlerService service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var StreamingNodeHandlerService_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "milvus.proto.streaming.StreamingNodeHandlerService", + HandlerType: (*StreamingNodeHandlerServiceServer)(nil), + Methods: []grpc.MethodDesc{}, + Streams: []grpc.StreamDesc{ + { + StreamName: "Produce", + Handler: _StreamingNodeHandlerService_Produce_Handler, + ServerStreams: true, + ClientStreams: true, + }, + { + StreamName: "Consume", + Handler: _StreamingNodeHandlerService_Consume_Handler, + ServerStreams: true, + ClientStreams: true, + }, + }, + Metadata: "streaming.proto", +} + +const ( + StreamingNodeManagerService_Assign_FullMethodName = "/milvus.proto.streaming.StreamingNodeManagerService/Assign" + StreamingNodeManagerService_Remove_FullMethodName = "/milvus.proto.streaming.StreamingNodeManagerService/Remove" + StreamingNodeManagerService_CollectStatus_FullMethodName = "/milvus.proto.streaming.StreamingNodeManagerService/CollectStatus" +) + +// StreamingNodeManagerServiceClient is the client API for StreamingNodeManagerService service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +type StreamingNodeManagerServiceClient interface { + // Assign is a unary RPC to assign a channel on a log node. + // Block until the channel assignd is ready to read or write on the log + // node. Error: If the channel already exists, return error with code + // CHANNEL_EXIST. + Assign(ctx context.Context, in *StreamingNodeManagerAssignRequest, opts ...grpc.CallOption) (*StreamingNodeManagerAssignResponse, error) + // Remove is unary RPC to remove a channel on a log node. + // Data of the channel on flying would be sent or flused as much as + // possible. Block until the resource of channel is released on the log + // node. New incoming request of handler of this channel will be rejected + // with special error. Error: If the channel does not exist, return error + // with code CHANNEL_NOT_EXIST. + Remove(ctx context.Context, in *StreamingNodeManagerRemoveRequest, opts ...grpc.CallOption) (*StreamingNodeManagerRemoveResponse, error) + // rpc CollectStatus() ... + // CollectStatus is unary RPC to collect all avaliable channel info and load + // balance info on a log node. Used to recover channel info on log coord, + // collect balance info and health check. + CollectStatus(ctx context.Context, in *StreamingNodeManagerCollectStatusRequest, opts ...grpc.CallOption) (*StreamingNodeManagerCollectStatusResponse, error) +} + +type streamingNodeManagerServiceClient struct { + cc grpc.ClientConnInterface +} + +func NewStreamingNodeManagerServiceClient(cc grpc.ClientConnInterface) StreamingNodeManagerServiceClient { + return &streamingNodeManagerServiceClient{cc} +} + +func (c *streamingNodeManagerServiceClient) Assign(ctx context.Context, in *StreamingNodeManagerAssignRequest, opts ...grpc.CallOption) (*StreamingNodeManagerAssignResponse, error) { + out := new(StreamingNodeManagerAssignResponse) + err := c.cc.Invoke(ctx, StreamingNodeManagerService_Assign_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *streamingNodeManagerServiceClient) Remove(ctx context.Context, in *StreamingNodeManagerRemoveRequest, opts ...grpc.CallOption) (*StreamingNodeManagerRemoveResponse, error) { + out := new(StreamingNodeManagerRemoveResponse) + err := c.cc.Invoke(ctx, StreamingNodeManagerService_Remove_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *streamingNodeManagerServiceClient) CollectStatus(ctx context.Context, in *StreamingNodeManagerCollectStatusRequest, opts ...grpc.CallOption) (*StreamingNodeManagerCollectStatusResponse, error) { + out := new(StreamingNodeManagerCollectStatusResponse) + err := c.cc.Invoke(ctx, StreamingNodeManagerService_CollectStatus_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// StreamingNodeManagerServiceServer is the server API for StreamingNodeManagerService service. +// All implementations should embed UnimplementedStreamingNodeManagerServiceServer +// for forward compatibility +type StreamingNodeManagerServiceServer interface { + // Assign is a unary RPC to assign a channel on a log node. + // Block until the channel assignd is ready to read or write on the log + // node. Error: If the channel already exists, return error with code + // CHANNEL_EXIST. + Assign(context.Context, *StreamingNodeManagerAssignRequest) (*StreamingNodeManagerAssignResponse, error) + // Remove is unary RPC to remove a channel on a log node. + // Data of the channel on flying would be sent or flused as much as + // possible. Block until the resource of channel is released on the log + // node. New incoming request of handler of this channel will be rejected + // with special error. Error: If the channel does not exist, return error + // with code CHANNEL_NOT_EXIST. + Remove(context.Context, *StreamingNodeManagerRemoveRequest) (*StreamingNodeManagerRemoveResponse, error) + // rpc CollectStatus() ... + // CollectStatus is unary RPC to collect all avaliable channel info and load + // balance info on a log node. Used to recover channel info on log coord, + // collect balance info and health check. + CollectStatus(context.Context, *StreamingNodeManagerCollectStatusRequest) (*StreamingNodeManagerCollectStatusResponse, error) +} + +// UnimplementedStreamingNodeManagerServiceServer should be embedded to have forward compatible implementations. +type UnimplementedStreamingNodeManagerServiceServer struct { +} + +func (UnimplementedStreamingNodeManagerServiceServer) Assign(context.Context, *StreamingNodeManagerAssignRequest) (*StreamingNodeManagerAssignResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method Assign not implemented") +} +func (UnimplementedStreamingNodeManagerServiceServer) Remove(context.Context, *StreamingNodeManagerRemoveRequest) (*StreamingNodeManagerRemoveResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method Remove not implemented") +} +func (UnimplementedStreamingNodeManagerServiceServer) CollectStatus(context.Context, *StreamingNodeManagerCollectStatusRequest) (*StreamingNodeManagerCollectStatusResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method CollectStatus not implemented") +} + +// UnsafeStreamingNodeManagerServiceServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to StreamingNodeManagerServiceServer will +// result in compilation errors. +type UnsafeStreamingNodeManagerServiceServer interface { + mustEmbedUnimplementedStreamingNodeManagerServiceServer() +} + +func RegisterStreamingNodeManagerServiceServer(s grpc.ServiceRegistrar, srv StreamingNodeManagerServiceServer) { + s.RegisterService(&StreamingNodeManagerService_ServiceDesc, srv) +} + +func _StreamingNodeManagerService_Assign_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(StreamingNodeManagerAssignRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(StreamingNodeManagerServiceServer).Assign(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: StreamingNodeManagerService_Assign_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(StreamingNodeManagerServiceServer).Assign(ctx, req.(*StreamingNodeManagerAssignRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _StreamingNodeManagerService_Remove_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(StreamingNodeManagerRemoveRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(StreamingNodeManagerServiceServer).Remove(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: StreamingNodeManagerService_Remove_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(StreamingNodeManagerServiceServer).Remove(ctx, req.(*StreamingNodeManagerRemoveRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _StreamingNodeManagerService_CollectStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(StreamingNodeManagerCollectStatusRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(StreamingNodeManagerServiceServer).CollectStatus(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: StreamingNodeManagerService_CollectStatus_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(StreamingNodeManagerServiceServer).CollectStatus(ctx, req.(*StreamingNodeManagerCollectStatusRequest)) + } + return interceptor(ctx, in, info, handler) +} + +// StreamingNodeManagerService_ServiceDesc is the grpc.ServiceDesc for StreamingNodeManagerService service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var StreamingNodeManagerService_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "milvus.proto.streaming.StreamingNodeManagerService", + HandlerType: (*StreamingNodeManagerServiceServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "Assign", + Handler: _StreamingNodeManagerService_Assign_Handler, + }, + { + MethodName: "Remove", + Handler: _StreamingNodeManagerService_Remove_Handler, + }, + { + MethodName: "CollectStatus", + Handler: _StreamingNodeManagerService_CollectStatus_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "streaming.proto", +} diff --git a/internal/proto/worker.proto b/pkg/proto/worker.proto similarity index 98% rename from internal/proto/worker.proto rename to pkg/proto/worker.proto index 6f8f72a0a441f..08d4f9828e4da 100644 --- a/internal/proto/worker.proto +++ b/pkg/proto/worker.proto @@ -2,7 +2,7 @@ syntax = "proto3"; package milvus.proto.index; -option go_package = "github.com/milvus-io/milvus/internal/proto/workerpb"; +option go_package = "github.com/milvus-io/milvus/pkg/proto/workerpb"; import "common.proto"; import "internal.proto"; diff --git a/pkg/proto/workerpb/worker.pb.go b/pkg/proto/workerpb/worker.pb.go new file mode 100644 index 0000000000000..bb09ccc728514 --- /dev/null +++ b/pkg/proto/workerpb/worker.pb.go @@ -0,0 +1,2576 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.33.0 +// protoc v3.21.4 +// source: worker.proto + +package workerpb + +import ( + commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + schemapb "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" + datapb "github.com/milvus-io/milvus/pkg/proto/datapb" + indexpb "github.com/milvus-io/milvus/pkg/proto/indexpb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type CreateJobRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ClusterID string `protobuf:"bytes,1,opt,name=clusterID,proto3" json:"clusterID,omitempty"` + IndexFilePrefix string `protobuf:"bytes,2,opt,name=index_file_prefix,json=indexFilePrefix,proto3" json:"index_file_prefix,omitempty"` + BuildID int64 `protobuf:"varint,3,opt,name=buildID,proto3" json:"buildID,omitempty"` + DataPaths []string `protobuf:"bytes,4,rep,name=data_paths,json=dataPaths,proto3" json:"data_paths,omitempty"` + IndexVersion int64 `protobuf:"varint,5,opt,name=index_version,json=indexVersion,proto3" json:"index_version,omitempty"` + IndexID int64 `protobuf:"varint,6,opt,name=indexID,proto3" json:"indexID,omitempty"` + IndexName string `protobuf:"bytes,7,opt,name=index_name,json=indexName,proto3" json:"index_name,omitempty"` + StorageConfig *indexpb.StorageConfig `protobuf:"bytes,8,opt,name=storage_config,json=storageConfig,proto3" json:"storage_config,omitempty"` + IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,9,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"` + TypeParams []*commonpb.KeyValuePair `protobuf:"bytes,10,rep,name=type_params,json=typeParams,proto3" json:"type_params,omitempty"` + NumRows int64 `protobuf:"varint,11,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"` + CurrentIndexVersion int32 `protobuf:"varint,12,opt,name=current_index_version,json=currentIndexVersion,proto3" json:"current_index_version,omitempty"` + CollectionID int64 `protobuf:"varint,13,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionID int64 `protobuf:"varint,14,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + SegmentID int64 `protobuf:"varint,15,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + FieldID int64 `protobuf:"varint,16,opt,name=fieldID,proto3" json:"fieldID,omitempty"` + FieldName string `protobuf:"bytes,17,opt,name=field_name,json=fieldName,proto3" json:"field_name,omitempty"` + FieldType schemapb.DataType `protobuf:"varint,18,opt,name=field_type,json=fieldType,proto3,enum=milvus.proto.schema.DataType" json:"field_type,omitempty"` + StorePath string `protobuf:"bytes,19,opt,name=store_path,json=storePath,proto3" json:"store_path,omitempty"` + StoreVersion int64 `protobuf:"varint,20,opt,name=store_version,json=storeVersion,proto3" json:"store_version,omitempty"` + IndexStorePath string `protobuf:"bytes,21,opt,name=index_store_path,json=indexStorePath,proto3" json:"index_store_path,omitempty"` + Dim int64 `protobuf:"varint,22,opt,name=dim,proto3" json:"dim,omitempty"` + DataIds []int64 `protobuf:"varint,23,rep,packed,name=data_ids,json=dataIds,proto3" json:"data_ids,omitempty"` + OptionalScalarFields []*indexpb.OptionalFieldInfo `protobuf:"bytes,24,rep,name=optional_scalar_fields,json=optionalScalarFields,proto3" json:"optional_scalar_fields,omitempty"` + Field *schemapb.FieldSchema `protobuf:"bytes,25,opt,name=field,proto3" json:"field,omitempty"` + PartitionKeyIsolation bool `protobuf:"varint,26,opt,name=partition_key_isolation,json=partitionKeyIsolation,proto3" json:"partition_key_isolation,omitempty"` +} + +func (x *CreateJobRequest) Reset() { + *x = CreateJobRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_worker_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateJobRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateJobRequest) ProtoMessage() {} + +func (x *CreateJobRequest) ProtoReflect() protoreflect.Message { + mi := &file_worker_proto_msgTypes[0] + 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 CreateJobRequest.ProtoReflect.Descriptor instead. +func (*CreateJobRequest) Descriptor() ([]byte, []int) { + return file_worker_proto_rawDescGZIP(), []int{0} +} + +func (x *CreateJobRequest) GetClusterID() string { + if x != nil { + return x.ClusterID + } + return "" +} + +func (x *CreateJobRequest) GetIndexFilePrefix() string { + if x != nil { + return x.IndexFilePrefix + } + return "" +} + +func (x *CreateJobRequest) GetBuildID() int64 { + if x != nil { + return x.BuildID + } + return 0 +} + +func (x *CreateJobRequest) GetDataPaths() []string { + if x != nil { + return x.DataPaths + } + return nil +} + +func (x *CreateJobRequest) GetIndexVersion() int64 { + if x != nil { + return x.IndexVersion + } + return 0 +} + +func (x *CreateJobRequest) GetIndexID() int64 { + if x != nil { + return x.IndexID + } + return 0 +} + +func (x *CreateJobRequest) GetIndexName() string { + if x != nil { + return x.IndexName + } + return "" +} + +func (x *CreateJobRequest) GetStorageConfig() *indexpb.StorageConfig { + if x != nil { + return x.StorageConfig + } + return nil +} + +func (x *CreateJobRequest) GetIndexParams() []*commonpb.KeyValuePair { + if x != nil { + return x.IndexParams + } + return nil +} + +func (x *CreateJobRequest) GetTypeParams() []*commonpb.KeyValuePair { + if x != nil { + return x.TypeParams + } + return nil +} + +func (x *CreateJobRequest) GetNumRows() int64 { + if x != nil { + return x.NumRows + } + return 0 +} + +func (x *CreateJobRequest) GetCurrentIndexVersion() int32 { + if x != nil { + return x.CurrentIndexVersion + } + return 0 +} + +func (x *CreateJobRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *CreateJobRequest) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *CreateJobRequest) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *CreateJobRequest) GetFieldID() int64 { + if x != nil { + return x.FieldID + } + return 0 +} + +func (x *CreateJobRequest) GetFieldName() string { + if x != nil { + return x.FieldName + } + return "" +} + +func (x *CreateJobRequest) GetFieldType() schemapb.DataType { + if x != nil { + return x.FieldType + } + return schemapb.DataType(0) +} + +func (x *CreateJobRequest) GetStorePath() string { + if x != nil { + return x.StorePath + } + return "" +} + +func (x *CreateJobRequest) GetStoreVersion() int64 { + if x != nil { + return x.StoreVersion + } + return 0 +} + +func (x *CreateJobRequest) GetIndexStorePath() string { + if x != nil { + return x.IndexStorePath + } + return "" +} + +func (x *CreateJobRequest) GetDim() int64 { + if x != nil { + return x.Dim + } + return 0 +} + +func (x *CreateJobRequest) GetDataIds() []int64 { + if x != nil { + return x.DataIds + } + return nil +} + +func (x *CreateJobRequest) GetOptionalScalarFields() []*indexpb.OptionalFieldInfo { + if x != nil { + return x.OptionalScalarFields + } + return nil +} + +func (x *CreateJobRequest) GetField() *schemapb.FieldSchema { + if x != nil { + return x.Field + } + return nil +} + +func (x *CreateJobRequest) GetPartitionKeyIsolation() bool { + if x != nil { + return x.PartitionKeyIsolation + } + return false +} + +type QueryJobsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ClusterID string `protobuf:"bytes,1,opt,name=clusterID,proto3" json:"clusterID,omitempty"` + BuildIDs []int64 `protobuf:"varint,2,rep,packed,name=buildIDs,proto3" json:"buildIDs,omitempty"` +} + +func (x *QueryJobsRequest) Reset() { + *x = QueryJobsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_worker_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *QueryJobsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryJobsRequest) ProtoMessage() {} + +func (x *QueryJobsRequest) ProtoReflect() protoreflect.Message { + mi := &file_worker_proto_msgTypes[1] + 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 QueryJobsRequest.ProtoReflect.Descriptor instead. +func (*QueryJobsRequest) Descriptor() ([]byte, []int) { + return file_worker_proto_rawDescGZIP(), []int{1} +} + +func (x *QueryJobsRequest) GetClusterID() string { + if x != nil { + return x.ClusterID + } + return "" +} + +func (x *QueryJobsRequest) GetBuildIDs() []int64 { + if x != nil { + return x.BuildIDs + } + return nil +} + +type QueryJobsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + ClusterID string `protobuf:"bytes,2,opt,name=clusterID,proto3" json:"clusterID,omitempty"` + IndexInfos []*IndexTaskInfo `protobuf:"bytes,3,rep,name=index_infos,json=indexInfos,proto3" json:"index_infos,omitempty"` +} + +func (x *QueryJobsResponse) Reset() { + *x = QueryJobsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_worker_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *QueryJobsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryJobsResponse) ProtoMessage() {} + +func (x *QueryJobsResponse) ProtoReflect() protoreflect.Message { + mi := &file_worker_proto_msgTypes[2] + 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 QueryJobsResponse.ProtoReflect.Descriptor instead. +func (*QueryJobsResponse) Descriptor() ([]byte, []int) { + return file_worker_proto_rawDescGZIP(), []int{2} +} + +func (x *QueryJobsResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *QueryJobsResponse) GetClusterID() string { + if x != nil { + return x.ClusterID + } + return "" +} + +func (x *QueryJobsResponse) GetIndexInfos() []*IndexTaskInfo { + if x != nil { + return x.IndexInfos + } + return nil +} + +type DropJobsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ClusterID string `protobuf:"bytes,1,opt,name=clusterID,proto3" json:"clusterID,omitempty"` + BuildIDs []int64 `protobuf:"varint,2,rep,packed,name=buildIDs,proto3" json:"buildIDs,omitempty"` +} + +func (x *DropJobsRequest) Reset() { + *x = DropJobsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_worker_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DropJobsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DropJobsRequest) ProtoMessage() {} + +func (x *DropJobsRequest) ProtoReflect() protoreflect.Message { + mi := &file_worker_proto_msgTypes[3] + 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 DropJobsRequest.ProtoReflect.Descriptor instead. +func (*DropJobsRequest) Descriptor() ([]byte, []int) { + return file_worker_proto_rawDescGZIP(), []int{3} +} + +func (x *DropJobsRequest) GetClusterID() string { + if x != nil { + return x.ClusterID + } + return "" +} + +func (x *DropJobsRequest) GetBuildIDs() []int64 { + if x != nil { + return x.BuildIDs + } + return nil +} + +type GetJobStatsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *GetJobStatsRequest) Reset() { + *x = GetJobStatsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_worker_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetJobStatsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetJobStatsRequest) ProtoMessage() {} + +func (x *GetJobStatsRequest) ProtoReflect() protoreflect.Message { + mi := &file_worker_proto_msgTypes[4] + 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 GetJobStatsRequest.ProtoReflect.Descriptor instead. +func (*GetJobStatsRequest) Descriptor() ([]byte, []int) { + return file_worker_proto_rawDescGZIP(), []int{4} +} + +type GetJobStatsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + TotalJobNum int64 `protobuf:"varint,2,opt,name=total_job_num,json=totalJobNum,proto3" json:"total_job_num,omitempty"` + InProgressJobNum int64 `protobuf:"varint,3,opt,name=in_progress_job_num,json=inProgressJobNum,proto3" json:"in_progress_job_num,omitempty"` + EnqueueJobNum int64 `protobuf:"varint,4,opt,name=enqueue_job_num,json=enqueueJobNum,proto3" json:"enqueue_job_num,omitempty"` + TaskSlots int64 `protobuf:"varint,5,opt,name=task_slots,json=taskSlots,proto3" json:"task_slots,omitempty"` + JobInfos []*indexpb.JobInfo `protobuf:"bytes,6,rep,name=job_infos,json=jobInfos,proto3" json:"job_infos,omitempty"` + EnableDisk bool `protobuf:"varint,7,opt,name=enable_disk,json=enableDisk,proto3" json:"enable_disk,omitempty"` +} + +func (x *GetJobStatsResponse) Reset() { + *x = GetJobStatsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_worker_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetJobStatsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetJobStatsResponse) ProtoMessage() {} + +func (x *GetJobStatsResponse) ProtoReflect() protoreflect.Message { + mi := &file_worker_proto_msgTypes[5] + 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 GetJobStatsResponse.ProtoReflect.Descriptor instead. +func (*GetJobStatsResponse) Descriptor() ([]byte, []int) { + return file_worker_proto_rawDescGZIP(), []int{5} +} + +func (x *GetJobStatsResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *GetJobStatsResponse) GetTotalJobNum() int64 { + if x != nil { + return x.TotalJobNum + } + return 0 +} + +func (x *GetJobStatsResponse) GetInProgressJobNum() int64 { + if x != nil { + return x.InProgressJobNum + } + return 0 +} + +func (x *GetJobStatsResponse) GetEnqueueJobNum() int64 { + if x != nil { + return x.EnqueueJobNum + } + return 0 +} + +func (x *GetJobStatsResponse) GetTaskSlots() int64 { + if x != nil { + return x.TaskSlots + } + return 0 +} + +func (x *GetJobStatsResponse) GetJobInfos() []*indexpb.JobInfo { + if x != nil { + return x.JobInfos + } + return nil +} + +func (x *GetJobStatsResponse) GetEnableDisk() bool { + if x != nil { + return x.EnableDisk + } + return false +} + +type AnalyzeRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ClusterID string `protobuf:"bytes,1,opt,name=clusterID,proto3" json:"clusterID,omitempty"` + TaskID int64 `protobuf:"varint,2,opt,name=taskID,proto3" json:"taskID,omitempty"` + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionID int64 `protobuf:"varint,4,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + FieldID int64 `protobuf:"varint,5,opt,name=fieldID,proto3" json:"fieldID,omitempty"` + FieldName string `protobuf:"bytes,6,opt,name=fieldName,proto3" json:"fieldName,omitempty"` + FieldType schemapb.DataType `protobuf:"varint,7,opt,name=field_type,json=fieldType,proto3,enum=milvus.proto.schema.DataType" json:"field_type,omitempty"` + SegmentStats map[int64]*indexpb.SegmentStats `protobuf:"bytes,8,rep,name=segment_stats,json=segmentStats,proto3" json:"segment_stats,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Version int64 `protobuf:"varint,9,opt,name=version,proto3" json:"version,omitempty"` + StorageConfig *indexpb.StorageConfig `protobuf:"bytes,10,opt,name=storage_config,json=storageConfig,proto3" json:"storage_config,omitempty"` + Dim int64 `protobuf:"varint,11,opt,name=dim,proto3" json:"dim,omitempty"` + MaxTrainSizeRatio float64 `protobuf:"fixed64,12,opt,name=max_train_size_ratio,json=maxTrainSizeRatio,proto3" json:"max_train_size_ratio,omitempty"` + NumClusters int64 `protobuf:"varint,13,opt,name=num_clusters,json=numClusters,proto3" json:"num_clusters,omitempty"` + Field *schemapb.FieldSchema `protobuf:"bytes,14,opt,name=field,proto3" json:"field,omitempty"` + MinClusterSizeRatio float64 `protobuf:"fixed64,15,opt,name=min_cluster_size_ratio,json=minClusterSizeRatio,proto3" json:"min_cluster_size_ratio,omitempty"` + MaxClusterSizeRatio float64 `protobuf:"fixed64,16,opt,name=max_cluster_size_ratio,json=maxClusterSizeRatio,proto3" json:"max_cluster_size_ratio,omitempty"` + MaxClusterSize int64 `protobuf:"varint,17,opt,name=max_cluster_size,json=maxClusterSize,proto3" json:"max_cluster_size,omitempty"` +} + +func (x *AnalyzeRequest) Reset() { + *x = AnalyzeRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_worker_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AnalyzeRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AnalyzeRequest) ProtoMessage() {} + +func (x *AnalyzeRequest) ProtoReflect() protoreflect.Message { + mi := &file_worker_proto_msgTypes[6] + 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 AnalyzeRequest.ProtoReflect.Descriptor instead. +func (*AnalyzeRequest) Descriptor() ([]byte, []int) { + return file_worker_proto_rawDescGZIP(), []int{6} +} + +func (x *AnalyzeRequest) GetClusterID() string { + if x != nil { + return x.ClusterID + } + return "" +} + +func (x *AnalyzeRequest) GetTaskID() int64 { + if x != nil { + return x.TaskID + } + return 0 +} + +func (x *AnalyzeRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *AnalyzeRequest) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *AnalyzeRequest) GetFieldID() int64 { + if x != nil { + return x.FieldID + } + return 0 +} + +func (x *AnalyzeRequest) GetFieldName() string { + if x != nil { + return x.FieldName + } + return "" +} + +func (x *AnalyzeRequest) GetFieldType() schemapb.DataType { + if x != nil { + return x.FieldType + } + return schemapb.DataType(0) +} + +func (x *AnalyzeRequest) GetSegmentStats() map[int64]*indexpb.SegmentStats { + if x != nil { + return x.SegmentStats + } + return nil +} + +func (x *AnalyzeRequest) GetVersion() int64 { + if x != nil { + return x.Version + } + return 0 +} + +func (x *AnalyzeRequest) GetStorageConfig() *indexpb.StorageConfig { + if x != nil { + return x.StorageConfig + } + return nil +} + +func (x *AnalyzeRequest) GetDim() int64 { + if x != nil { + return x.Dim + } + return 0 +} + +func (x *AnalyzeRequest) GetMaxTrainSizeRatio() float64 { + if x != nil { + return x.MaxTrainSizeRatio + } + return 0 +} + +func (x *AnalyzeRequest) GetNumClusters() int64 { + if x != nil { + return x.NumClusters + } + return 0 +} + +func (x *AnalyzeRequest) GetField() *schemapb.FieldSchema { + if x != nil { + return x.Field + } + return nil +} + +func (x *AnalyzeRequest) GetMinClusterSizeRatio() float64 { + if x != nil { + return x.MinClusterSizeRatio + } + return 0 +} + +func (x *AnalyzeRequest) GetMaxClusterSizeRatio() float64 { + if x != nil { + return x.MaxClusterSizeRatio + } + return 0 +} + +func (x *AnalyzeRequest) GetMaxClusterSize() int64 { + if x != nil { + return x.MaxClusterSize + } + return 0 +} + +type CreateStatsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ClusterID string `protobuf:"bytes,1,opt,name=clusterID,proto3" json:"clusterID,omitempty"` + TaskID int64 `protobuf:"varint,2,opt,name=taskID,proto3" json:"taskID,omitempty"` + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionID int64 `protobuf:"varint,4,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + InsertChannel string `protobuf:"bytes,5,opt,name=insert_channel,json=insertChannel,proto3" json:"insert_channel,omitempty"` + SegmentID int64 `protobuf:"varint,6,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + InsertLogs []*datapb.FieldBinlog `protobuf:"bytes,7,rep,name=insert_logs,json=insertLogs,proto3" json:"insert_logs,omitempty"` + DeltaLogs []*datapb.FieldBinlog `protobuf:"bytes,8,rep,name=delta_logs,json=deltaLogs,proto3" json:"delta_logs,omitempty"` + StorageConfig *indexpb.StorageConfig `protobuf:"bytes,9,opt,name=storage_config,json=storageConfig,proto3" json:"storage_config,omitempty"` + Schema *schemapb.CollectionSchema `protobuf:"bytes,10,opt,name=schema,proto3" json:"schema,omitempty"` + SubJobType indexpb.StatsSubJob `protobuf:"varint,11,opt,name=subJobType,proto3,enum=milvus.proto.index.StatsSubJob" json:"subJobType,omitempty"` + TargetSegmentID int64 `protobuf:"varint,12,opt,name=targetSegmentID,proto3" json:"targetSegmentID,omitempty"` + StartLogID int64 `protobuf:"varint,13,opt,name=startLogID,proto3" json:"startLogID,omitempty"` + EndLogID int64 `protobuf:"varint,14,opt,name=endLogID,proto3" json:"endLogID,omitempty"` + NumRows int64 `protobuf:"varint,15,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"` + CollectionTtl int64 `protobuf:"varint,16,opt,name=collection_ttl,json=collectionTtl,proto3" json:"collection_ttl,omitempty"` + CurrentTs uint64 `protobuf:"varint,17,opt,name=current_ts,json=currentTs,proto3" json:"current_ts,omitempty"` + TaskVersion int64 `protobuf:"varint,18,opt,name=task_version,json=taskVersion,proto3" json:"task_version,omitempty"` + BinlogMaxSize uint64 `protobuf:"varint,19,opt,name=binlogMaxSize,proto3" json:"binlogMaxSize,omitempty"` +} + +func (x *CreateStatsRequest) Reset() { + *x = CreateStatsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_worker_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateStatsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateStatsRequest) ProtoMessage() {} + +func (x *CreateStatsRequest) ProtoReflect() protoreflect.Message { + mi := &file_worker_proto_msgTypes[7] + 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 CreateStatsRequest.ProtoReflect.Descriptor instead. +func (*CreateStatsRequest) Descriptor() ([]byte, []int) { + return file_worker_proto_rawDescGZIP(), []int{7} +} + +func (x *CreateStatsRequest) GetClusterID() string { + if x != nil { + return x.ClusterID + } + return "" +} + +func (x *CreateStatsRequest) GetTaskID() int64 { + if x != nil { + return x.TaskID + } + return 0 +} + +func (x *CreateStatsRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *CreateStatsRequest) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *CreateStatsRequest) GetInsertChannel() string { + if x != nil { + return x.InsertChannel + } + return "" +} + +func (x *CreateStatsRequest) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *CreateStatsRequest) GetInsertLogs() []*datapb.FieldBinlog { + if x != nil { + return x.InsertLogs + } + return nil +} + +func (x *CreateStatsRequest) GetDeltaLogs() []*datapb.FieldBinlog { + if x != nil { + return x.DeltaLogs + } + return nil +} + +func (x *CreateStatsRequest) GetStorageConfig() *indexpb.StorageConfig { + if x != nil { + return x.StorageConfig + } + return nil +} + +func (x *CreateStatsRequest) GetSchema() *schemapb.CollectionSchema { + if x != nil { + return x.Schema + } + return nil +} + +func (x *CreateStatsRequest) GetSubJobType() indexpb.StatsSubJob { + if x != nil { + return x.SubJobType + } + return indexpb.StatsSubJob(0) +} + +func (x *CreateStatsRequest) GetTargetSegmentID() int64 { + if x != nil { + return x.TargetSegmentID + } + return 0 +} + +func (x *CreateStatsRequest) GetStartLogID() int64 { + if x != nil { + return x.StartLogID + } + return 0 +} + +func (x *CreateStatsRequest) GetEndLogID() int64 { + if x != nil { + return x.EndLogID + } + return 0 +} + +func (x *CreateStatsRequest) GetNumRows() int64 { + if x != nil { + return x.NumRows + } + return 0 +} + +func (x *CreateStatsRequest) GetCollectionTtl() int64 { + if x != nil { + return x.CollectionTtl + } + return 0 +} + +func (x *CreateStatsRequest) GetCurrentTs() uint64 { + if x != nil { + return x.CurrentTs + } + return 0 +} + +func (x *CreateStatsRequest) GetTaskVersion() int64 { + if x != nil { + return x.TaskVersion + } + return 0 +} + +func (x *CreateStatsRequest) GetBinlogMaxSize() uint64 { + if x != nil { + return x.BinlogMaxSize + } + return 0 +} + +type CreateJobV2Request struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ClusterID string `protobuf:"bytes,1,opt,name=clusterID,proto3" json:"clusterID,omitempty"` + TaskID int64 `protobuf:"varint,2,opt,name=taskID,proto3" json:"taskID,omitempty"` + JobType indexpb.JobType `protobuf:"varint,3,opt,name=job_type,json=jobType,proto3,enum=milvus.proto.index.JobType" json:"job_type,omitempty"` + // Types that are assignable to Request: + // + // *CreateJobV2Request_AnalyzeRequest + // *CreateJobV2Request_IndexRequest + // *CreateJobV2Request_StatsRequest + Request isCreateJobV2Request_Request `protobuf_oneof:"request"` +} + +func (x *CreateJobV2Request) Reset() { + *x = CreateJobV2Request{} + if protoimpl.UnsafeEnabled { + mi := &file_worker_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateJobV2Request) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateJobV2Request) ProtoMessage() {} + +func (x *CreateJobV2Request) ProtoReflect() protoreflect.Message { + mi := &file_worker_proto_msgTypes[8] + 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 CreateJobV2Request.ProtoReflect.Descriptor instead. +func (*CreateJobV2Request) Descriptor() ([]byte, []int) { + return file_worker_proto_rawDescGZIP(), []int{8} +} + +func (x *CreateJobV2Request) GetClusterID() string { + if x != nil { + return x.ClusterID + } + return "" +} + +func (x *CreateJobV2Request) GetTaskID() int64 { + if x != nil { + return x.TaskID + } + return 0 +} + +func (x *CreateJobV2Request) GetJobType() indexpb.JobType { + if x != nil { + return x.JobType + } + return indexpb.JobType(0) +} + +func (m *CreateJobV2Request) GetRequest() isCreateJobV2Request_Request { + if m != nil { + return m.Request + } + return nil +} + +func (x *CreateJobV2Request) GetAnalyzeRequest() *AnalyzeRequest { + if x, ok := x.GetRequest().(*CreateJobV2Request_AnalyzeRequest); ok { + return x.AnalyzeRequest + } + return nil +} + +func (x *CreateJobV2Request) GetIndexRequest() *CreateJobRequest { + if x, ok := x.GetRequest().(*CreateJobV2Request_IndexRequest); ok { + return x.IndexRequest + } + return nil +} + +func (x *CreateJobV2Request) GetStatsRequest() *CreateStatsRequest { + if x, ok := x.GetRequest().(*CreateJobV2Request_StatsRequest); ok { + return x.StatsRequest + } + return nil +} + +type isCreateJobV2Request_Request interface { + isCreateJobV2Request_Request() +} + +type CreateJobV2Request_AnalyzeRequest struct { + AnalyzeRequest *AnalyzeRequest `protobuf:"bytes,4,opt,name=analyze_request,json=analyzeRequest,proto3,oneof"` +} + +type CreateJobV2Request_IndexRequest struct { + IndexRequest *CreateJobRequest `protobuf:"bytes,5,opt,name=index_request,json=indexRequest,proto3,oneof"` +} + +type CreateJobV2Request_StatsRequest struct { + StatsRequest *CreateStatsRequest `protobuf:"bytes,6,opt,name=stats_request,json=statsRequest,proto3,oneof"` +} + +func (*CreateJobV2Request_AnalyzeRequest) isCreateJobV2Request_Request() {} + +func (*CreateJobV2Request_IndexRequest) isCreateJobV2Request_Request() {} + +func (*CreateJobV2Request_StatsRequest) isCreateJobV2Request_Request() {} + +type QueryJobsV2Request struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ClusterID string `protobuf:"bytes,1,opt,name=clusterID,proto3" json:"clusterID,omitempty"` + TaskIDs []int64 `protobuf:"varint,2,rep,packed,name=taskIDs,proto3" json:"taskIDs,omitempty"` + JobType indexpb.JobType `protobuf:"varint,3,opt,name=job_type,json=jobType,proto3,enum=milvus.proto.index.JobType" json:"job_type,omitempty"` +} + +func (x *QueryJobsV2Request) Reset() { + *x = QueryJobsV2Request{} + if protoimpl.UnsafeEnabled { + mi := &file_worker_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *QueryJobsV2Request) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryJobsV2Request) ProtoMessage() {} + +func (x *QueryJobsV2Request) ProtoReflect() protoreflect.Message { + mi := &file_worker_proto_msgTypes[9] + 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 QueryJobsV2Request.ProtoReflect.Descriptor instead. +func (*QueryJobsV2Request) Descriptor() ([]byte, []int) { + return file_worker_proto_rawDescGZIP(), []int{9} +} + +func (x *QueryJobsV2Request) GetClusterID() string { + if x != nil { + return x.ClusterID + } + return "" +} + +func (x *QueryJobsV2Request) GetTaskIDs() []int64 { + if x != nil { + return x.TaskIDs + } + return nil +} + +func (x *QueryJobsV2Request) GetJobType() indexpb.JobType { + if x != nil { + return x.JobType + } + return indexpb.JobType(0) +} + +type IndexTaskInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + BuildID int64 `protobuf:"varint,1,opt,name=buildID,proto3" json:"buildID,omitempty"` + State commonpb.IndexState `protobuf:"varint,2,opt,name=state,proto3,enum=milvus.proto.common.IndexState" json:"state,omitempty"` + IndexFileKeys []string `protobuf:"bytes,3,rep,name=index_file_keys,json=indexFileKeys,proto3" json:"index_file_keys,omitempty"` + SerializedSize uint64 `protobuf:"varint,4,opt,name=serialized_size,json=serializedSize,proto3" json:"serialized_size,omitempty"` + FailReason string `protobuf:"bytes,5,opt,name=fail_reason,json=failReason,proto3" json:"fail_reason,omitempty"` + CurrentIndexVersion int32 `protobuf:"varint,6,opt,name=current_index_version,json=currentIndexVersion,proto3" json:"current_index_version,omitempty"` + IndexStoreVersion int64 `protobuf:"varint,7,opt,name=index_store_version,json=indexStoreVersion,proto3" json:"index_store_version,omitempty"` +} + +func (x *IndexTaskInfo) Reset() { + *x = IndexTaskInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_worker_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *IndexTaskInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*IndexTaskInfo) ProtoMessage() {} + +func (x *IndexTaskInfo) ProtoReflect() protoreflect.Message { + mi := &file_worker_proto_msgTypes[10] + 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 IndexTaskInfo.ProtoReflect.Descriptor instead. +func (*IndexTaskInfo) Descriptor() ([]byte, []int) { + return file_worker_proto_rawDescGZIP(), []int{10} +} + +func (x *IndexTaskInfo) GetBuildID() int64 { + if x != nil { + return x.BuildID + } + return 0 +} + +func (x *IndexTaskInfo) GetState() commonpb.IndexState { + if x != nil { + return x.State + } + return commonpb.IndexState(0) +} + +func (x *IndexTaskInfo) GetIndexFileKeys() []string { + if x != nil { + return x.IndexFileKeys + } + return nil +} + +func (x *IndexTaskInfo) GetSerializedSize() uint64 { + if x != nil { + return x.SerializedSize + } + return 0 +} + +func (x *IndexTaskInfo) GetFailReason() string { + if x != nil { + return x.FailReason + } + return "" +} + +func (x *IndexTaskInfo) GetCurrentIndexVersion() int32 { + if x != nil { + return x.CurrentIndexVersion + } + return 0 +} + +func (x *IndexTaskInfo) GetIndexStoreVersion() int64 { + if x != nil { + return x.IndexStoreVersion + } + return 0 +} + +type IndexJobResults struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Results []*IndexTaskInfo `protobuf:"bytes,1,rep,name=results,proto3" json:"results,omitempty"` +} + +func (x *IndexJobResults) Reset() { + *x = IndexJobResults{} + if protoimpl.UnsafeEnabled { + mi := &file_worker_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *IndexJobResults) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*IndexJobResults) ProtoMessage() {} + +func (x *IndexJobResults) ProtoReflect() protoreflect.Message { + mi := &file_worker_proto_msgTypes[11] + 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 IndexJobResults.ProtoReflect.Descriptor instead. +func (*IndexJobResults) Descriptor() ([]byte, []int) { + return file_worker_proto_rawDescGZIP(), []int{11} +} + +func (x *IndexJobResults) GetResults() []*IndexTaskInfo { + if x != nil { + return x.Results + } + return nil +} + +type AnalyzeResult struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + TaskID int64 `protobuf:"varint,1,opt,name=taskID,proto3" json:"taskID,omitempty"` + State indexpb.JobState `protobuf:"varint,2,opt,name=state,proto3,enum=milvus.proto.index.JobState" json:"state,omitempty"` + FailReason string `protobuf:"bytes,3,opt,name=fail_reason,json=failReason,proto3" json:"fail_reason,omitempty"` + CentroidsFile string `protobuf:"bytes,4,opt,name=centroids_file,json=centroidsFile,proto3" json:"centroids_file,omitempty"` +} + +func (x *AnalyzeResult) Reset() { + *x = AnalyzeResult{} + if protoimpl.UnsafeEnabled { + mi := &file_worker_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AnalyzeResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AnalyzeResult) ProtoMessage() {} + +func (x *AnalyzeResult) ProtoReflect() protoreflect.Message { + mi := &file_worker_proto_msgTypes[12] + 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 AnalyzeResult.ProtoReflect.Descriptor instead. +func (*AnalyzeResult) Descriptor() ([]byte, []int) { + return file_worker_proto_rawDescGZIP(), []int{12} +} + +func (x *AnalyzeResult) GetTaskID() int64 { + if x != nil { + return x.TaskID + } + return 0 +} + +func (x *AnalyzeResult) GetState() indexpb.JobState { + if x != nil { + return x.State + } + return indexpb.JobState(0) +} + +func (x *AnalyzeResult) GetFailReason() string { + if x != nil { + return x.FailReason + } + return "" +} + +func (x *AnalyzeResult) GetCentroidsFile() string { + if x != nil { + return x.CentroidsFile + } + return "" +} + +type AnalyzeResults struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Results []*AnalyzeResult `protobuf:"bytes,1,rep,name=results,proto3" json:"results,omitempty"` +} + +func (x *AnalyzeResults) Reset() { + *x = AnalyzeResults{} + if protoimpl.UnsafeEnabled { + mi := &file_worker_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AnalyzeResults) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AnalyzeResults) ProtoMessage() {} + +func (x *AnalyzeResults) ProtoReflect() protoreflect.Message { + mi := &file_worker_proto_msgTypes[13] + 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 AnalyzeResults.ProtoReflect.Descriptor instead. +func (*AnalyzeResults) Descriptor() ([]byte, []int) { + return file_worker_proto_rawDescGZIP(), []int{13} +} + +func (x *AnalyzeResults) GetResults() []*AnalyzeResult { + if x != nil { + return x.Results + } + return nil +} + +type StatsResult struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + TaskID int64 `protobuf:"varint,1,opt,name=taskID,proto3" json:"taskID,omitempty"` + State indexpb.JobState `protobuf:"varint,2,opt,name=state,proto3,enum=milvus.proto.index.JobState" json:"state,omitempty"` + FailReason string `protobuf:"bytes,3,opt,name=fail_reason,json=failReason,proto3" json:"fail_reason,omitempty"` + CollectionID int64 `protobuf:"varint,4,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionID int64 `protobuf:"varint,5,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + SegmentID int64 `protobuf:"varint,6,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + Channel string `protobuf:"bytes,7,opt,name=channel,proto3" json:"channel,omitempty"` + InsertLogs []*datapb.FieldBinlog `protobuf:"bytes,8,rep,name=insert_logs,json=insertLogs,proto3" json:"insert_logs,omitempty"` + StatsLogs []*datapb.FieldBinlog `protobuf:"bytes,9,rep,name=stats_logs,json=statsLogs,proto3" json:"stats_logs,omitempty"` + TextStatsLogs map[int64]*datapb.TextIndexStats `protobuf:"bytes,10,rep,name=text_stats_logs,json=textStatsLogs,proto3" json:"text_stats_logs,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + NumRows int64 `protobuf:"varint,11,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"` + Bm25Logs []*datapb.FieldBinlog `protobuf:"bytes,12,rep,name=bm25_logs,json=bm25Logs,proto3" json:"bm25_logs,omitempty"` +} + +func (x *StatsResult) Reset() { + *x = StatsResult{} + if protoimpl.UnsafeEnabled { + mi := &file_worker_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StatsResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StatsResult) ProtoMessage() {} + +func (x *StatsResult) ProtoReflect() protoreflect.Message { + mi := &file_worker_proto_msgTypes[14] + 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 StatsResult.ProtoReflect.Descriptor instead. +func (*StatsResult) Descriptor() ([]byte, []int) { + return file_worker_proto_rawDescGZIP(), []int{14} +} + +func (x *StatsResult) GetTaskID() int64 { + if x != nil { + return x.TaskID + } + return 0 +} + +func (x *StatsResult) GetState() indexpb.JobState { + if x != nil { + return x.State + } + return indexpb.JobState(0) +} + +func (x *StatsResult) GetFailReason() string { + if x != nil { + return x.FailReason + } + return "" +} + +func (x *StatsResult) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *StatsResult) GetPartitionID() int64 { + if x != nil { + return x.PartitionID + } + return 0 +} + +func (x *StatsResult) GetSegmentID() int64 { + if x != nil { + return x.SegmentID + } + return 0 +} + +func (x *StatsResult) GetChannel() string { + if x != nil { + return x.Channel + } + return "" +} + +func (x *StatsResult) GetInsertLogs() []*datapb.FieldBinlog { + if x != nil { + return x.InsertLogs + } + return nil +} + +func (x *StatsResult) GetStatsLogs() []*datapb.FieldBinlog { + if x != nil { + return x.StatsLogs + } + return nil +} + +func (x *StatsResult) GetTextStatsLogs() map[int64]*datapb.TextIndexStats { + if x != nil { + return x.TextStatsLogs + } + return nil +} + +func (x *StatsResult) GetNumRows() int64 { + if x != nil { + return x.NumRows + } + return 0 +} + +func (x *StatsResult) GetBm25Logs() []*datapb.FieldBinlog { + if x != nil { + return x.Bm25Logs + } + return nil +} + +type StatsResults struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Results []*StatsResult `protobuf:"bytes,1,rep,name=results,proto3" json:"results,omitempty"` +} + +func (x *StatsResults) Reset() { + *x = StatsResults{} + if protoimpl.UnsafeEnabled { + mi := &file_worker_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StatsResults) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StatsResults) ProtoMessage() {} + +func (x *StatsResults) ProtoReflect() protoreflect.Message { + mi := &file_worker_proto_msgTypes[15] + 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 StatsResults.ProtoReflect.Descriptor instead. +func (*StatsResults) Descriptor() ([]byte, []int) { + return file_worker_proto_rawDescGZIP(), []int{15} +} + +func (x *StatsResults) GetResults() []*StatsResult { + if x != nil { + return x.Results + } + return nil +} + +type QueryJobsV2Response struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + ClusterID string `protobuf:"bytes,2,opt,name=clusterID,proto3" json:"clusterID,omitempty"` + // Types that are assignable to Result: + // + // *QueryJobsV2Response_IndexJobResults + // *QueryJobsV2Response_AnalyzeJobResults + // *QueryJobsV2Response_StatsJobResults + Result isQueryJobsV2Response_Result `protobuf_oneof:"result"` +} + +func (x *QueryJobsV2Response) Reset() { + *x = QueryJobsV2Response{} + if protoimpl.UnsafeEnabled { + mi := &file_worker_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *QueryJobsV2Response) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryJobsV2Response) ProtoMessage() {} + +func (x *QueryJobsV2Response) ProtoReflect() protoreflect.Message { + mi := &file_worker_proto_msgTypes[16] + 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 QueryJobsV2Response.ProtoReflect.Descriptor instead. +func (*QueryJobsV2Response) Descriptor() ([]byte, []int) { + return file_worker_proto_rawDescGZIP(), []int{16} +} + +func (x *QueryJobsV2Response) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *QueryJobsV2Response) GetClusterID() string { + if x != nil { + return x.ClusterID + } + return "" +} + +func (m *QueryJobsV2Response) GetResult() isQueryJobsV2Response_Result { + if m != nil { + return m.Result + } + return nil +} + +func (x *QueryJobsV2Response) GetIndexJobResults() *IndexJobResults { + if x, ok := x.GetResult().(*QueryJobsV2Response_IndexJobResults); ok { + return x.IndexJobResults + } + return nil +} + +func (x *QueryJobsV2Response) GetAnalyzeJobResults() *AnalyzeResults { + if x, ok := x.GetResult().(*QueryJobsV2Response_AnalyzeJobResults); ok { + return x.AnalyzeJobResults + } + return nil +} + +func (x *QueryJobsV2Response) GetStatsJobResults() *StatsResults { + if x, ok := x.GetResult().(*QueryJobsV2Response_StatsJobResults); ok { + return x.StatsJobResults + } + return nil +} + +type isQueryJobsV2Response_Result interface { + isQueryJobsV2Response_Result() +} + +type QueryJobsV2Response_IndexJobResults struct { + IndexJobResults *IndexJobResults `protobuf:"bytes,3,opt,name=index_job_results,json=indexJobResults,proto3,oneof"` +} + +type QueryJobsV2Response_AnalyzeJobResults struct { + AnalyzeJobResults *AnalyzeResults `protobuf:"bytes,4,opt,name=analyze_job_results,json=analyzeJobResults,proto3,oneof"` +} + +type QueryJobsV2Response_StatsJobResults struct { + StatsJobResults *StatsResults `protobuf:"bytes,5,opt,name=stats_job_results,json=statsJobResults,proto3,oneof"` +} + +func (*QueryJobsV2Response_IndexJobResults) isQueryJobsV2Response_Result() {} + +func (*QueryJobsV2Response_AnalyzeJobResults) isQueryJobsV2Response_Result() {} + +func (*QueryJobsV2Response_StatsJobResults) isQueryJobsV2Response_Result() {} + +type DropJobsV2Request struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ClusterID string `protobuf:"bytes,1,opt,name=clusterID,proto3" json:"clusterID,omitempty"` + TaskIDs []int64 `protobuf:"varint,2,rep,packed,name=taskIDs,proto3" json:"taskIDs,omitempty"` + JobType indexpb.JobType `protobuf:"varint,3,opt,name=job_type,json=jobType,proto3,enum=milvus.proto.index.JobType" json:"job_type,omitempty"` +} + +func (x *DropJobsV2Request) Reset() { + *x = DropJobsV2Request{} + if protoimpl.UnsafeEnabled { + mi := &file_worker_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DropJobsV2Request) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DropJobsV2Request) ProtoMessage() {} + +func (x *DropJobsV2Request) ProtoReflect() protoreflect.Message { + mi := &file_worker_proto_msgTypes[17] + 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 DropJobsV2Request.ProtoReflect.Descriptor instead. +func (*DropJobsV2Request) Descriptor() ([]byte, []int) { + return file_worker_proto_rawDescGZIP(), []int{17} +} + +func (x *DropJobsV2Request) GetClusterID() string { + if x != nil { + return x.ClusterID + } + return "" +} + +func (x *DropJobsV2Request) GetTaskIDs() []int64 { + if x != nil { + return x.TaskIDs + } + return nil +} + +func (x *DropJobsV2Request) GetJobType() indexpb.JobType { + if x != nil { + return x.JobType + } + return indexpb.JobType(0) +} + +var File_worker_proto protoreflect.FileDescriptor + +var file_worker_proto_rawDesc = []byte{ + 0x0a, 0x0c, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x12, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x1a, 0x0c, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x1a, 0x0e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x1a, 0x0c, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x0c, + 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x10, 0x64, 0x61, + 0x74, 0x61, 0x5f, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x11, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x22, 0xd9, 0x08, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, + 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, + 0x65, 0x72, 0x49, 0x44, 0x12, 0x2a, 0x0a, 0x11, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x66, 0x69, + 0x6c, 0x65, 0x5f, 0x70, 0x72, 0x65, 0x66, 0x69, 0x78, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x46, 0x69, 0x6c, 0x65, 0x50, 0x72, 0x65, 0x66, 0x69, 0x78, + 0x12, 0x18, 0x0a, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x64, 0x61, + 0x74, 0x61, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, + 0x64, 0x61, 0x74, 0x61, 0x50, 0x61, 0x74, 0x68, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x0c, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x18, + 0x0a, 0x07, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x07, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x48, 0x0a, 0x0e, 0x73, 0x74, 0x6f, 0x72, 0x61, + 0x67, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x52, 0x0d, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x12, 0x44, 0x0a, 0x0c, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, + 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, + 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0b, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x42, 0x0a, 0x0b, 0x74, 0x79, 0x70, 0x65, 0x5f, + 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, + 0x0a, 0x74, 0x79, 0x70, 0x65, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x6e, + 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6e, + 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x32, 0x0a, 0x15, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, + 0x74, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, + 0x0c, 0x20, 0x01, 0x28, 0x05, 0x52, 0x13, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x49, 0x6e, + 0x64, 0x65, 0x78, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, + 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x0e, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, + 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x0f, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x18, + 0x0a, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x10, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x66, 0x69, 0x65, 0x6c, + 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x11, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x66, 0x69, + 0x65, 0x6c, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3c, 0x0a, 0x0a, 0x66, 0x69, 0x65, 0x6c, 0x64, + 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x12, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1d, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x66, 0x69, 0x65, 0x6c, + 0x64, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x5f, 0x70, + 0x61, 0x74, 0x68, 0x18, 0x13, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x74, 0x6f, 0x72, 0x65, + 0x50, 0x61, 0x74, 0x68, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x5f, 0x76, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x14, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x73, 0x74, 0x6f, + 0x72, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x28, 0x0a, 0x10, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x15, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x6f, 0x72, 0x65, 0x50, + 0x61, 0x74, 0x68, 0x12, 0x10, 0x0a, 0x03, 0x64, 0x69, 0x6d, 0x18, 0x16, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x03, 0x64, 0x69, 0x6d, 0x12, 0x19, 0x0a, 0x08, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x69, 0x64, + 0x73, 0x18, 0x17, 0x20, 0x03, 0x28, 0x03, 0x52, 0x07, 0x64, 0x61, 0x74, 0x61, 0x49, 0x64, 0x73, + 0x12, 0x5b, 0x0a, 0x16, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x5f, 0x73, 0x63, 0x61, + 0x6c, 0x61, 0x72, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x18, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x46, 0x69, + 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x14, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x61, + 0x6c, 0x53, 0x63, 0x61, 0x6c, 0x61, 0x72, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x36, 0x0a, + 0x05, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x18, 0x19, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, + 0x66, 0x69, 0x65, 0x6c, 0x64, 0x12, 0x36, 0x0a, 0x17, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x69, 0x73, 0x6f, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x1a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x15, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x4b, 0x65, 0x79, 0x49, 0x73, 0x6f, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x4c, 0x0a, + 0x10, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, + 0x1a, 0x0a, 0x08, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, + 0x03, 0x52, 0x08, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x73, 0x22, 0xaa, 0x01, 0x0a, 0x11, + 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, + 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, + 0x72, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, + 0x65, 0x72, 0x49, 0x44, 0x12, 0x42, 0x0a, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x69, 0x6e, + 0x66, 0x6f, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, + 0x6e, 0x64, 0x65, 0x78, 0x54, 0x61, 0x73, 0x6b, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x22, 0x4b, 0x0a, 0x0f, 0x44, 0x72, 0x6f, 0x70, + 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, + 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, + 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x1a, 0x0a, 0x08, 0x62, 0x75, 0x69, + 0x6c, 0x64, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x08, 0x62, 0x75, 0x69, + 0x6c, 0x64, 0x49, 0x44, 0x73, 0x22, 0x14, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x53, + 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0xbf, 0x02, 0x0a, 0x13, + 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x22, 0x0a, 0x0d, 0x74, 0x6f, 0x74, 0x61, + 0x6c, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x75, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0b, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x4a, 0x6f, 0x62, 0x4e, 0x75, 0x6d, 0x12, 0x2d, 0x0a, 0x13, + 0x69, 0x6e, 0x5f, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, + 0x6e, 0x75, 0x6d, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x69, 0x6e, 0x50, 0x72, 0x6f, + 0x67, 0x72, 0x65, 0x73, 0x73, 0x4a, 0x6f, 0x62, 0x4e, 0x75, 0x6d, 0x12, 0x26, 0x0a, 0x0f, 0x65, + 0x6e, 0x71, 0x75, 0x65, 0x75, 0x65, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x75, 0x6d, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x65, 0x6e, 0x71, 0x75, 0x65, 0x75, 0x65, 0x4a, 0x6f, 0x62, + 0x4e, 0x75, 0x6d, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x73, 0x6c, 0x6f, 0x74, + 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x53, 0x6c, 0x6f, + 0x74, 0x73, 0x12, 0x38, 0x0a, 0x09, 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, + 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x49, 0x6e, + 0x66, 0x6f, 0x52, 0x08, 0x6a, 0x6f, 0x62, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x1f, 0x0a, 0x0b, + 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x64, 0x69, 0x73, 0x6b, 0x18, 0x07, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x0a, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x44, 0x69, 0x73, 0x6b, 0x22, 0xd6, 0x06, + 0x0a, 0x0e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x16, + 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, + 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, + 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, + 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x4e, + 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x66, 0x69, 0x65, 0x6c, 0x64, + 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3c, 0x0a, 0x0a, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x74, 0x79, + 0x70, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x44, + 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x54, 0x79, + 0x70, 0x65, 0x12, 0x59, 0x0a, 0x0d, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x73, 0x74, + 0x61, 0x74, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x41, + 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, + 0x0c, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x18, 0x0a, + 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, + 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x48, 0x0a, 0x0e, 0x73, 0x74, 0x6f, 0x72, 0x61, + 0x67, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x52, 0x0d, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x12, 0x10, 0x0a, 0x03, 0x64, 0x69, 0x6d, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, + 0x64, 0x69, 0x6d, 0x12, 0x2f, 0x0a, 0x14, 0x6d, 0x61, 0x78, 0x5f, 0x74, 0x72, 0x61, 0x69, 0x6e, + 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x18, 0x0c, 0x20, 0x01, 0x28, + 0x01, 0x52, 0x11, 0x6d, 0x61, 0x78, 0x54, 0x72, 0x61, 0x69, 0x6e, 0x53, 0x69, 0x7a, 0x65, 0x52, + 0x61, 0x74, 0x69, 0x6f, 0x12, 0x21, 0x0a, 0x0c, 0x6e, 0x75, 0x6d, 0x5f, 0x63, 0x6c, 0x75, 0x73, + 0x74, 0x65, 0x72, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x6e, 0x75, 0x6d, 0x43, + 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x12, 0x36, 0x0a, 0x05, 0x66, 0x69, 0x65, 0x6c, 0x64, + 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x46, 0x69, 0x65, + 0x6c, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x12, + 0x33, 0x0a, 0x16, 0x6d, 0x69, 0x6e, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x73, + 0x69, 0x7a, 0x65, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x01, 0x52, + 0x13, 0x6d, 0x69, 0x6e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x53, 0x69, 0x7a, 0x65, 0x52, + 0x61, 0x74, 0x69, 0x6f, 0x12, 0x33, 0x0a, 0x16, 0x6d, 0x61, 0x78, 0x5f, 0x63, 0x6c, 0x75, 0x73, + 0x74, 0x65, 0x72, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x18, 0x10, + 0x20, 0x01, 0x28, 0x01, 0x52, 0x13, 0x6d, 0x61, 0x78, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, + 0x53, 0x69, 0x7a, 0x65, 0x52, 0x61, 0x74, 0x69, 0x6f, 0x12, 0x28, 0x0a, 0x10, 0x6d, 0x61, 0x78, + 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x11, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0e, 0x6d, 0x61, 0x78, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x53, + 0x69, 0x7a, 0x65, 0x1a, 0x61, 0x0a, 0x11, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, + 0x61, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x36, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xaf, 0x06, 0x0a, 0x12, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, + 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x16, 0x0a, 0x06, 0x74, + 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, + 0x6b, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x73, + 0x65, 0x72, 0x74, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0d, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, + 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x3f, + 0x0a, 0x0b, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x07, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, + 0x6c, 0x6f, 0x67, 0x52, 0x0a, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x4c, 0x6f, 0x67, 0x73, 0x12, + 0x3d, 0x0a, 0x0a, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x5f, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x08, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, + 0x6c, 0x6f, 0x67, 0x52, 0x09, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x48, + 0x0a, 0x0e, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, 0x6f, 0x72, + 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x0d, 0x73, 0x74, 0x6f, 0x72, 0x61, + 0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x18, 0x0a, 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, 0x3f, 0x0a, 0x0a, 0x73, 0x75, 0x62, 0x4a, 0x6f, + 0x62, 0x54, 0x79, 0x70, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x2e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x53, 0x75, 0x62, 0x4a, 0x6f, 0x62, 0x52, 0x0a, 0x73, 0x75, + 0x62, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x12, 0x28, 0x0a, 0x0f, 0x74, 0x61, 0x72, 0x67, + 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x0c, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x0f, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x49, 0x44, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x4c, 0x6f, 0x67, 0x49, 0x44, + 0x18, 0x0d, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x4c, 0x6f, 0x67, + 0x49, 0x44, 0x12, 0x1a, 0x0a, 0x08, 0x65, 0x6e, 0x64, 0x4c, 0x6f, 0x67, 0x49, 0x44, 0x18, 0x0e, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x65, 0x6e, 0x64, 0x4c, 0x6f, 0x67, 0x49, 0x44, 0x12, 0x19, + 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x74, 0x6c, 0x18, 0x10, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x74, 0x6c, + 0x12, 0x1d, 0x0a, 0x0a, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x73, 0x18, 0x11, + 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x54, 0x73, 0x12, + 0x21, 0x0a, 0x0c, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, + 0x12, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x74, 0x61, 0x73, 0x6b, 0x56, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x12, 0x24, 0x0a, 0x0d, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x4d, 0x61, 0x78, 0x53, + 0x69, 0x7a, 0x65, 0x18, 0x13, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x62, 0x69, 0x6e, 0x6c, 0x6f, + 0x67, 0x4d, 0x61, 0x78, 0x53, 0x69, 0x7a, 0x65, 0x22, 0xf8, 0x02, 0x0a, 0x12, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x56, 0x32, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x16, 0x0a, + 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, + 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x36, 0x0a, 0x08, 0x6a, 0x6f, 0x62, 0x5f, 0x74, 0x79, 0x70, + 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, + 0x54, 0x79, 0x70, 0x65, 0x52, 0x07, 0x6a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x12, 0x4d, 0x0a, + 0x0f, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x41, 0x6e, 0x61, 0x6c, + 0x79, 0x7a, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0e, 0x61, 0x6e, + 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x4b, 0x0a, 0x0d, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4a, + 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0c, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x4d, 0x0a, 0x0d, 0x73, 0x74, 0x61, + 0x74, 0x73, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x74, 0x61, 0x74, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0c, 0x73, 0x74, 0x61, 0x74, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x42, 0x09, 0x0a, 0x07, 0x72, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x22, 0x84, 0x01, 0x0a, 0x12, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, + 0x73, 0x56, 0x32, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, + 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, + 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, + 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x49, + 0x44, 0x73, 0x12, 0x36, 0x0a, 0x08, 0x6a, 0x6f, 0x62, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, + 0x65, 0x52, 0x07, 0x6a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x22, 0xb6, 0x02, 0x0a, 0x0d, 0x49, + 0x6e, 0x64, 0x65, 0x78, 0x54, 0x61, 0x73, 0x6b, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x18, 0x0a, 0x07, + 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x62, + 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x35, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x49, 0x6e, 0x64, 0x65, + 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x26, 0x0a, + 0x0f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, + 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x46, 0x69, 0x6c, + 0x65, 0x4b, 0x65, 0x79, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, + 0x7a, 0x65, 0x64, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0e, + 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x1f, + 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, + 0x32, 0x0a, 0x15, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x05, 0x52, 0x13, + 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x56, 0x65, 0x72, 0x73, + 0x69, 0x6f, 0x6e, 0x12, 0x2e, 0x0a, 0x13, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x73, 0x74, 0x6f, + 0x72, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x11, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x6f, 0x72, 0x65, 0x56, 0x65, 0x72, 0x73, + 0x69, 0x6f, 0x6e, 0x22, 0x4e, 0x0a, 0x0f, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62, 0x52, + 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x3b, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, + 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, 0x6e, 0x64, + 0x65, 0x78, 0x54, 0x61, 0x73, 0x6b, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, + 0x6c, 0x74, 0x73, 0x22, 0xa3, 0x01, 0x0a, 0x0d, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, + 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x32, 0x0a, + 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, + 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x61, 0x73, + 0x6f, 0x6e, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x65, 0x6e, 0x74, 0x72, 0x6f, 0x69, 0x64, 0x73, 0x5f, + 0x66, 0x69, 0x6c, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x65, 0x6e, 0x74, + 0x72, 0x6f, 0x69, 0x64, 0x73, 0x46, 0x69, 0x6c, 0x65, 0x22, 0x4d, 0x0a, 0x0e, 0x41, 0x6e, 0x61, + 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x3b, 0x0a, 0x07, 0x72, + 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, + 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x91, 0x05, 0x0a, 0x0b, 0x53, 0x74, 0x61, + 0x74, 0x73, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, + 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, + 0x12, 0x32, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, + 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72, 0x65, 0x61, + 0x73, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, 0x69, 0x6c, 0x52, + 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, + 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x73, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, + 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x12, 0x3f, 0x0a, 0x0b, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x6c, 0x6f, + 0x67, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, + 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x0a, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, + 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x3d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x6c, 0x6f, + 0x67, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, + 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x73, 0x4c, + 0x6f, 0x67, 0x73, 0x12, 0x5a, 0x0a, 0x0f, 0x74, 0x65, 0x78, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x74, + 0x73, 0x5f, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x2e, 0x54, 0x65, + 0x78, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x52, 0x0d, 0x74, 0x65, 0x78, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x12, + 0x19, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x3b, 0x0a, 0x09, 0x62, 0x6d, + 0x32, 0x35, 0x5f, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x0c, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x08, 0x62, + 0x6d, 0x32, 0x35, 0x4c, 0x6f, 0x67, 0x73, 0x1a, 0x63, 0x0a, 0x12, 0x54, 0x65, 0x78, 0x74, 0x53, + 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, + 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, + 0x37, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x54, 0x65, 0x78, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, + 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x49, 0x0a, 0x0c, + 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x39, 0x0a, 0x07, + 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x07, + 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0xeb, 0x02, 0x0a, 0x13, 0x51, 0x75, 0x65, 0x72, + 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, + 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, + 0x49, 0x44, 0x12, 0x51, 0x0a, 0x11, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, + 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x75, 0x6c, + 0x74, 0x73, 0x48, 0x00, 0x52, 0x0f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62, 0x52, 0x65, + 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x54, 0x0a, 0x13, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, + 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, + 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x48, 0x00, 0x52, 0x11, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, + 0x65, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x4e, 0x0a, 0x11, 0x73, + 0x74, 0x61, 0x74, 0x73, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, 0x61, 0x74, + 0x73, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x48, 0x00, 0x52, 0x0f, 0x73, 0x74, 0x61, 0x74, + 0x73, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x08, 0x0a, 0x06, 0x72, + 0x65, 0x73, 0x75, 0x6c, 0x74, 0x22, 0x83, 0x01, 0x0a, 0x11, 0x44, 0x72, 0x6f, 0x70, 0x4a, 0x6f, + 0x62, 0x73, 0x56, 0x32, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, + 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, + 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x61, 0x73, + 0x6b, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x07, 0x74, 0x61, 0x73, 0x6b, + 0x49, 0x44, 0x73, 0x12, 0x36, 0x0a, 0x08, 0x6a, 0x6f, 0x62, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x54, 0x79, + 0x70, 0x65, 0x52, 0x07, 0x6a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x32, 0xb6, 0x08, 0x0a, 0x09, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x6c, 0x0a, 0x12, 0x47, 0x65, 0x74, + 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, + 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, + 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x73, 0x22, 0x00, 0x12, 0x71, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x53, 0x74, + 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, + 0x32, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, + 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, + 0x73, 0x74, 0x69, 0x63, 0x73, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x50, 0x0a, 0x09, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x4a, 0x6f, 0x62, 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, 0x09, + 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x51, + 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4e, 0x0a, 0x08, 0x44, 0x72, 0x6f, 0x70, + 0x4a, 0x6f, 0x62, 0x73, 0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x4a, 0x6f, + 0x62, 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, 0x0b, 0x47, 0x65, 0x74, 0x4a, + 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, + 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7b, 0x0a, 0x12, 0x53, 0x68, + 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 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, 0x54, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x56, 0x32, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x56, 0x32, 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, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x12, 0x26, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, + 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x52, 0x0a, 0x0a, 0x44, 0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x12, 0x25, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 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, 0x30, 0x5a, 0x2e, 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, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x77, 0x6f, + 0x72, 0x6b, 0x65, 0x72, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_worker_proto_rawDescOnce sync.Once + file_worker_proto_rawDescData = file_worker_proto_rawDesc +) + +func file_worker_proto_rawDescGZIP() []byte { + file_worker_proto_rawDescOnce.Do(func() { + file_worker_proto_rawDescData = protoimpl.X.CompressGZIP(file_worker_proto_rawDescData) + }) + return file_worker_proto_rawDescData +} + +var file_worker_proto_msgTypes = make([]protoimpl.MessageInfo, 20) +var file_worker_proto_goTypes = []interface{}{ + (*CreateJobRequest)(nil), // 0: milvus.proto.index.CreateJobRequest + (*QueryJobsRequest)(nil), // 1: milvus.proto.index.QueryJobsRequest + (*QueryJobsResponse)(nil), // 2: milvus.proto.index.QueryJobsResponse + (*DropJobsRequest)(nil), // 3: milvus.proto.index.DropJobsRequest + (*GetJobStatsRequest)(nil), // 4: milvus.proto.index.GetJobStatsRequest + (*GetJobStatsResponse)(nil), // 5: milvus.proto.index.GetJobStatsResponse + (*AnalyzeRequest)(nil), // 6: milvus.proto.index.AnalyzeRequest + (*CreateStatsRequest)(nil), // 7: milvus.proto.index.CreateStatsRequest + (*CreateJobV2Request)(nil), // 8: milvus.proto.index.CreateJobV2Request + (*QueryJobsV2Request)(nil), // 9: milvus.proto.index.QueryJobsV2Request + (*IndexTaskInfo)(nil), // 10: milvus.proto.index.IndexTaskInfo + (*IndexJobResults)(nil), // 11: milvus.proto.index.IndexJobResults + (*AnalyzeResult)(nil), // 12: milvus.proto.index.AnalyzeResult + (*AnalyzeResults)(nil), // 13: milvus.proto.index.AnalyzeResults + (*StatsResult)(nil), // 14: milvus.proto.index.StatsResult + (*StatsResults)(nil), // 15: milvus.proto.index.StatsResults + (*QueryJobsV2Response)(nil), // 16: milvus.proto.index.QueryJobsV2Response + (*DropJobsV2Request)(nil), // 17: milvus.proto.index.DropJobsV2Request + nil, // 18: milvus.proto.index.AnalyzeRequest.SegmentStatsEntry + nil, // 19: milvus.proto.index.StatsResult.TextStatsLogsEntry + (*indexpb.StorageConfig)(nil), // 20: milvus.proto.index.StorageConfig + (*commonpb.KeyValuePair)(nil), // 21: milvus.proto.common.KeyValuePair + (schemapb.DataType)(0), // 22: milvus.proto.schema.DataType + (*indexpb.OptionalFieldInfo)(nil), // 23: milvus.proto.index.OptionalFieldInfo + (*schemapb.FieldSchema)(nil), // 24: milvus.proto.schema.FieldSchema + (*commonpb.Status)(nil), // 25: milvus.proto.common.Status + (*indexpb.JobInfo)(nil), // 26: milvus.proto.index.JobInfo + (*datapb.FieldBinlog)(nil), // 27: milvus.proto.data.FieldBinlog + (*schemapb.CollectionSchema)(nil), // 28: milvus.proto.schema.CollectionSchema + (indexpb.StatsSubJob)(0), // 29: milvus.proto.index.StatsSubJob + (indexpb.JobType)(0), // 30: milvus.proto.index.JobType + (commonpb.IndexState)(0), // 31: milvus.proto.common.IndexState + (indexpb.JobState)(0), // 32: milvus.proto.index.JobState + (*indexpb.SegmentStats)(nil), // 33: milvus.proto.index.SegmentStats + (*datapb.TextIndexStats)(nil), // 34: milvus.proto.data.TextIndexStats + (*milvuspb.GetComponentStatesRequest)(nil), // 35: milvus.proto.milvus.GetComponentStatesRequest + (*internalpb.GetStatisticsChannelRequest)(nil), // 36: milvus.proto.internal.GetStatisticsChannelRequest + (*internalpb.ShowConfigurationsRequest)(nil), // 37: milvus.proto.internal.ShowConfigurationsRequest + (*milvuspb.GetMetricsRequest)(nil), // 38: milvus.proto.milvus.GetMetricsRequest + (*milvuspb.ComponentStates)(nil), // 39: milvus.proto.milvus.ComponentStates + (*milvuspb.StringResponse)(nil), // 40: milvus.proto.milvus.StringResponse + (*internalpb.ShowConfigurationsResponse)(nil), // 41: milvus.proto.internal.ShowConfigurationsResponse + (*milvuspb.GetMetricsResponse)(nil), // 42: milvus.proto.milvus.GetMetricsResponse +} +var file_worker_proto_depIdxs = []int32{ + 20, // 0: milvus.proto.index.CreateJobRequest.storage_config:type_name -> milvus.proto.index.StorageConfig + 21, // 1: milvus.proto.index.CreateJobRequest.index_params:type_name -> milvus.proto.common.KeyValuePair + 21, // 2: milvus.proto.index.CreateJobRequest.type_params:type_name -> milvus.proto.common.KeyValuePair + 22, // 3: milvus.proto.index.CreateJobRequest.field_type:type_name -> milvus.proto.schema.DataType + 23, // 4: milvus.proto.index.CreateJobRequest.optional_scalar_fields:type_name -> milvus.proto.index.OptionalFieldInfo + 24, // 5: milvus.proto.index.CreateJobRequest.field:type_name -> milvus.proto.schema.FieldSchema + 25, // 6: milvus.proto.index.QueryJobsResponse.status:type_name -> milvus.proto.common.Status + 10, // 7: milvus.proto.index.QueryJobsResponse.index_infos:type_name -> milvus.proto.index.IndexTaskInfo + 25, // 8: milvus.proto.index.GetJobStatsResponse.status:type_name -> milvus.proto.common.Status + 26, // 9: milvus.proto.index.GetJobStatsResponse.job_infos:type_name -> milvus.proto.index.JobInfo + 22, // 10: milvus.proto.index.AnalyzeRequest.field_type:type_name -> milvus.proto.schema.DataType + 18, // 11: milvus.proto.index.AnalyzeRequest.segment_stats:type_name -> milvus.proto.index.AnalyzeRequest.SegmentStatsEntry + 20, // 12: milvus.proto.index.AnalyzeRequest.storage_config:type_name -> milvus.proto.index.StorageConfig + 24, // 13: milvus.proto.index.AnalyzeRequest.field:type_name -> milvus.proto.schema.FieldSchema + 27, // 14: milvus.proto.index.CreateStatsRequest.insert_logs:type_name -> milvus.proto.data.FieldBinlog + 27, // 15: milvus.proto.index.CreateStatsRequest.delta_logs:type_name -> milvus.proto.data.FieldBinlog + 20, // 16: milvus.proto.index.CreateStatsRequest.storage_config:type_name -> milvus.proto.index.StorageConfig + 28, // 17: milvus.proto.index.CreateStatsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 29, // 18: milvus.proto.index.CreateStatsRequest.subJobType:type_name -> milvus.proto.index.StatsSubJob + 30, // 19: milvus.proto.index.CreateJobV2Request.job_type:type_name -> milvus.proto.index.JobType + 6, // 20: milvus.proto.index.CreateJobV2Request.analyze_request:type_name -> milvus.proto.index.AnalyzeRequest + 0, // 21: milvus.proto.index.CreateJobV2Request.index_request:type_name -> milvus.proto.index.CreateJobRequest + 7, // 22: milvus.proto.index.CreateJobV2Request.stats_request:type_name -> milvus.proto.index.CreateStatsRequest + 30, // 23: milvus.proto.index.QueryJobsV2Request.job_type:type_name -> milvus.proto.index.JobType + 31, // 24: milvus.proto.index.IndexTaskInfo.state:type_name -> milvus.proto.common.IndexState + 10, // 25: milvus.proto.index.IndexJobResults.results:type_name -> milvus.proto.index.IndexTaskInfo + 32, // 26: milvus.proto.index.AnalyzeResult.state:type_name -> milvus.proto.index.JobState + 12, // 27: milvus.proto.index.AnalyzeResults.results:type_name -> milvus.proto.index.AnalyzeResult + 32, // 28: milvus.proto.index.StatsResult.state:type_name -> milvus.proto.index.JobState + 27, // 29: milvus.proto.index.StatsResult.insert_logs:type_name -> milvus.proto.data.FieldBinlog + 27, // 30: milvus.proto.index.StatsResult.stats_logs:type_name -> milvus.proto.data.FieldBinlog + 19, // 31: milvus.proto.index.StatsResult.text_stats_logs:type_name -> milvus.proto.index.StatsResult.TextStatsLogsEntry + 27, // 32: milvus.proto.index.StatsResult.bm25_logs:type_name -> milvus.proto.data.FieldBinlog + 14, // 33: milvus.proto.index.StatsResults.results:type_name -> milvus.proto.index.StatsResult + 25, // 34: milvus.proto.index.QueryJobsV2Response.status:type_name -> milvus.proto.common.Status + 11, // 35: milvus.proto.index.QueryJobsV2Response.index_job_results:type_name -> milvus.proto.index.IndexJobResults + 13, // 36: milvus.proto.index.QueryJobsV2Response.analyze_job_results:type_name -> milvus.proto.index.AnalyzeResults + 15, // 37: milvus.proto.index.QueryJobsV2Response.stats_job_results:type_name -> milvus.proto.index.StatsResults + 30, // 38: milvus.proto.index.DropJobsV2Request.job_type:type_name -> milvus.proto.index.JobType + 33, // 39: milvus.proto.index.AnalyzeRequest.SegmentStatsEntry.value:type_name -> milvus.proto.index.SegmentStats + 34, // 40: milvus.proto.index.StatsResult.TextStatsLogsEntry.value:type_name -> milvus.proto.data.TextIndexStats + 35, // 41: milvus.proto.index.IndexNode.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest + 36, // 42: milvus.proto.index.IndexNode.GetStatisticsChannel:input_type -> milvus.proto.internal.GetStatisticsChannelRequest + 0, // 43: milvus.proto.index.IndexNode.CreateJob:input_type -> milvus.proto.index.CreateJobRequest + 1, // 44: milvus.proto.index.IndexNode.QueryJobs:input_type -> milvus.proto.index.QueryJobsRequest + 3, // 45: milvus.proto.index.IndexNode.DropJobs:input_type -> milvus.proto.index.DropJobsRequest + 4, // 46: milvus.proto.index.IndexNode.GetJobStats:input_type -> milvus.proto.index.GetJobStatsRequest + 37, // 47: milvus.proto.index.IndexNode.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest + 38, // 48: milvus.proto.index.IndexNode.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest + 8, // 49: milvus.proto.index.IndexNode.CreateJobV2:input_type -> milvus.proto.index.CreateJobV2Request + 9, // 50: milvus.proto.index.IndexNode.QueryJobsV2:input_type -> milvus.proto.index.QueryJobsV2Request + 17, // 51: milvus.proto.index.IndexNode.DropJobsV2:input_type -> milvus.proto.index.DropJobsV2Request + 39, // 52: milvus.proto.index.IndexNode.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates + 40, // 53: milvus.proto.index.IndexNode.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse + 25, // 54: milvus.proto.index.IndexNode.CreateJob:output_type -> milvus.proto.common.Status + 2, // 55: milvus.proto.index.IndexNode.QueryJobs:output_type -> milvus.proto.index.QueryJobsResponse + 25, // 56: milvus.proto.index.IndexNode.DropJobs:output_type -> milvus.proto.common.Status + 5, // 57: milvus.proto.index.IndexNode.GetJobStats:output_type -> milvus.proto.index.GetJobStatsResponse + 41, // 58: milvus.proto.index.IndexNode.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse + 42, // 59: milvus.proto.index.IndexNode.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse + 25, // 60: milvus.proto.index.IndexNode.CreateJobV2:output_type -> milvus.proto.common.Status + 16, // 61: milvus.proto.index.IndexNode.QueryJobsV2:output_type -> milvus.proto.index.QueryJobsV2Response + 25, // 62: milvus.proto.index.IndexNode.DropJobsV2:output_type -> milvus.proto.common.Status + 52, // [52:63] is the sub-list for method output_type + 41, // [41:52] is the sub-list for method input_type + 41, // [41:41] is the sub-list for extension type_name + 41, // [41:41] is the sub-list for extension extendee + 0, // [0:41] is the sub-list for field type_name +} + +func init() { file_worker_proto_init() } +func file_worker_proto_init() { + if File_worker_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_worker_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CreateJobRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_worker_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QueryJobsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_worker_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QueryJobsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_worker_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DropJobsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_worker_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetJobStatsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_worker_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetJobStatsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_worker_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AnalyzeRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_worker_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CreateStatsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_worker_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CreateJobV2Request); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_worker_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QueryJobsV2Request); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_worker_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*IndexTaskInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_worker_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*IndexJobResults); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_worker_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AnalyzeResult); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_worker_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AnalyzeResults); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_worker_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*StatsResult); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_worker_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*StatsResults); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_worker_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QueryJobsV2Response); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_worker_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DropJobsV2Request); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + file_worker_proto_msgTypes[8].OneofWrappers = []interface{}{ + (*CreateJobV2Request_AnalyzeRequest)(nil), + (*CreateJobV2Request_IndexRequest)(nil), + (*CreateJobV2Request_StatsRequest)(nil), + } + file_worker_proto_msgTypes[16].OneofWrappers = []interface{}{ + (*QueryJobsV2Response_IndexJobResults)(nil), + (*QueryJobsV2Response_AnalyzeJobResults)(nil), + (*QueryJobsV2Response_StatsJobResults)(nil), + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_worker_proto_rawDesc, + NumEnums: 0, + NumMessages: 20, + NumExtensions: 0, + NumServices: 1, + }, + GoTypes: file_worker_proto_goTypes, + DependencyIndexes: file_worker_proto_depIdxs, + MessageInfos: file_worker_proto_msgTypes, + }.Build() + File_worker_proto = out.File + file_worker_proto_rawDesc = nil + file_worker_proto_goTypes = nil + file_worker_proto_depIdxs = nil +} diff --git a/pkg/proto/workerpb/worker_grpc.pb.go b/pkg/proto/workerpb/worker_grpc.pb.go new file mode 100644 index 0000000000000..a13c2bb5538c7 --- /dev/null +++ b/pkg/proto/workerpb/worker_grpc.pb.go @@ -0,0 +1,482 @@ +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. +// versions: +// - protoc-gen-go-grpc v1.3.0 +// - protoc v3.21.4 +// source: worker.proto + +package workerpb + +import ( + context "context" + commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" +) + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +// Requires gRPC-Go v1.32.0 or later. +const _ = grpc.SupportPackageIsVersion7 + +const ( + IndexNode_GetComponentStates_FullMethodName = "/milvus.proto.index.IndexNode/GetComponentStates" + IndexNode_GetStatisticsChannel_FullMethodName = "/milvus.proto.index.IndexNode/GetStatisticsChannel" + IndexNode_CreateJob_FullMethodName = "/milvus.proto.index.IndexNode/CreateJob" + IndexNode_QueryJobs_FullMethodName = "/milvus.proto.index.IndexNode/QueryJobs" + IndexNode_DropJobs_FullMethodName = "/milvus.proto.index.IndexNode/DropJobs" + IndexNode_GetJobStats_FullMethodName = "/milvus.proto.index.IndexNode/GetJobStats" + IndexNode_ShowConfigurations_FullMethodName = "/milvus.proto.index.IndexNode/ShowConfigurations" + IndexNode_GetMetrics_FullMethodName = "/milvus.proto.index.IndexNode/GetMetrics" + IndexNode_CreateJobV2_FullMethodName = "/milvus.proto.index.IndexNode/CreateJobV2" + IndexNode_QueryJobsV2_FullMethodName = "/milvus.proto.index.IndexNode/QueryJobsV2" + IndexNode_DropJobsV2_FullMethodName = "/milvus.proto.index.IndexNode/DropJobsV2" +) + +// IndexNodeClient is the client API for IndexNode service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +type IndexNodeClient interface { + GetComponentStates(ctx context.Context, in *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) + GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) + CreateJob(ctx context.Context, in *CreateJobRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + QueryJobs(ctx context.Context, in *QueryJobsRequest, opts ...grpc.CallOption) (*QueryJobsResponse, error) + DropJobs(ctx context.Context, in *DropJobsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + GetJobStats(ctx context.Context, in *GetJobStatsRequest, opts ...grpc.CallOption) (*GetJobStatsResponse, error) + ShowConfigurations(ctx context.Context, in *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error) + // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy + GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) + CreateJobV2(ctx context.Context, in *CreateJobV2Request, opts ...grpc.CallOption) (*commonpb.Status, error) + QueryJobsV2(ctx context.Context, in *QueryJobsV2Request, opts ...grpc.CallOption) (*QueryJobsV2Response, error) + DropJobsV2(ctx context.Context, in *DropJobsV2Request, opts ...grpc.CallOption) (*commonpb.Status, error) +} + +type indexNodeClient struct { + cc grpc.ClientConnInterface +} + +func NewIndexNodeClient(cc grpc.ClientConnInterface) IndexNodeClient { + return &indexNodeClient{cc} +} + +func (c *indexNodeClient) GetComponentStates(ctx context.Context, in *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) { + out := new(milvuspb.ComponentStates) + err := c.cc.Invoke(ctx, IndexNode_GetComponentStates_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *indexNodeClient) GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) { + out := new(milvuspb.StringResponse) + err := c.cc.Invoke(ctx, IndexNode_GetStatisticsChannel_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *indexNodeClient) CreateJob(ctx context.Context, in *CreateJobRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, IndexNode_CreateJob_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *indexNodeClient) QueryJobs(ctx context.Context, in *QueryJobsRequest, opts ...grpc.CallOption) (*QueryJobsResponse, error) { + out := new(QueryJobsResponse) + err := c.cc.Invoke(ctx, IndexNode_QueryJobs_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *indexNodeClient) DropJobs(ctx context.Context, in *DropJobsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, IndexNode_DropJobs_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *indexNodeClient) GetJobStats(ctx context.Context, in *GetJobStatsRequest, opts ...grpc.CallOption) (*GetJobStatsResponse, error) { + out := new(GetJobStatsResponse) + err := c.cc.Invoke(ctx, IndexNode_GetJobStats_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *indexNodeClient) ShowConfigurations(ctx context.Context, in *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error) { + out := new(internalpb.ShowConfigurationsResponse) + err := c.cc.Invoke(ctx, IndexNode_ShowConfigurations_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *indexNodeClient) GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) { + out := new(milvuspb.GetMetricsResponse) + err := c.cc.Invoke(ctx, IndexNode_GetMetrics_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *indexNodeClient) CreateJobV2(ctx context.Context, in *CreateJobV2Request, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, IndexNode_CreateJobV2_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *indexNodeClient) QueryJobsV2(ctx context.Context, in *QueryJobsV2Request, opts ...grpc.CallOption) (*QueryJobsV2Response, error) { + out := new(QueryJobsV2Response) + err := c.cc.Invoke(ctx, IndexNode_QueryJobsV2_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *indexNodeClient) DropJobsV2(ctx context.Context, in *DropJobsV2Request, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, IndexNode_DropJobsV2_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// IndexNodeServer is the server API for IndexNode service. +// All implementations should embed UnimplementedIndexNodeServer +// for forward compatibility +type IndexNodeServer interface { + GetComponentStates(context.Context, *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) + GetStatisticsChannel(context.Context, *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) + CreateJob(context.Context, *CreateJobRequest) (*commonpb.Status, error) + QueryJobs(context.Context, *QueryJobsRequest) (*QueryJobsResponse, error) + DropJobs(context.Context, *DropJobsRequest) (*commonpb.Status, error) + GetJobStats(context.Context, *GetJobStatsRequest) (*GetJobStatsResponse, error) + ShowConfigurations(context.Context, *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) + // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy + GetMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) + CreateJobV2(context.Context, *CreateJobV2Request) (*commonpb.Status, error) + QueryJobsV2(context.Context, *QueryJobsV2Request) (*QueryJobsV2Response, error) + DropJobsV2(context.Context, *DropJobsV2Request) (*commonpb.Status, error) +} + +// UnimplementedIndexNodeServer should be embedded to have forward compatible implementations. +type UnimplementedIndexNodeServer struct { +} + +func (UnimplementedIndexNodeServer) GetComponentStates(context.Context, *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetComponentStates not implemented") +} +func (UnimplementedIndexNodeServer) GetStatisticsChannel(context.Context, *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetStatisticsChannel not implemented") +} +func (UnimplementedIndexNodeServer) CreateJob(context.Context, *CreateJobRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method CreateJob not implemented") +} +func (UnimplementedIndexNodeServer) QueryJobs(context.Context, *QueryJobsRequest) (*QueryJobsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryJobs not implemented") +} +func (UnimplementedIndexNodeServer) DropJobs(context.Context, *DropJobsRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method DropJobs not implemented") +} +func (UnimplementedIndexNodeServer) GetJobStats(context.Context, *GetJobStatsRequest) (*GetJobStatsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetJobStats not implemented") +} +func (UnimplementedIndexNodeServer) ShowConfigurations(context.Context, *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ShowConfigurations not implemented") +} +func (UnimplementedIndexNodeServer) GetMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetMetrics not implemented") +} +func (UnimplementedIndexNodeServer) CreateJobV2(context.Context, *CreateJobV2Request) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method CreateJobV2 not implemented") +} +func (UnimplementedIndexNodeServer) QueryJobsV2(context.Context, *QueryJobsV2Request) (*QueryJobsV2Response, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryJobsV2 not implemented") +} +func (UnimplementedIndexNodeServer) DropJobsV2(context.Context, *DropJobsV2Request) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method DropJobsV2 not implemented") +} + +// UnsafeIndexNodeServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to IndexNodeServer will +// result in compilation errors. +type UnsafeIndexNodeServer interface { + mustEmbedUnimplementedIndexNodeServer() +} + +func RegisterIndexNodeServer(s grpc.ServiceRegistrar, srv IndexNodeServer) { + s.RegisterService(&IndexNode_ServiceDesc, srv) +} + +func _IndexNode_GetComponentStates_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.GetComponentStatesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IndexNodeServer).GetComponentStates(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: IndexNode_GetComponentStates_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IndexNodeServer).GetComponentStates(ctx, req.(*milvuspb.GetComponentStatesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _IndexNode_GetStatisticsChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.GetStatisticsChannelRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IndexNodeServer).GetStatisticsChannel(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: IndexNode_GetStatisticsChannel_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IndexNodeServer).GetStatisticsChannel(ctx, req.(*internalpb.GetStatisticsChannelRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _IndexNode_CreateJob_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CreateJobRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IndexNodeServer).CreateJob(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: IndexNode_CreateJob_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IndexNodeServer).CreateJob(ctx, req.(*CreateJobRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _IndexNode_QueryJobs_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryJobsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IndexNodeServer).QueryJobs(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: IndexNode_QueryJobs_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IndexNodeServer).QueryJobs(ctx, req.(*QueryJobsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _IndexNode_DropJobs_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DropJobsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IndexNodeServer).DropJobs(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: IndexNode_DropJobs_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IndexNodeServer).DropJobs(ctx, req.(*DropJobsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _IndexNode_GetJobStats_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetJobStatsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IndexNodeServer).GetJobStats(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: IndexNode_GetJobStats_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IndexNodeServer).GetJobStats(ctx, req.(*GetJobStatsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _IndexNode_ShowConfigurations_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.ShowConfigurationsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IndexNodeServer).ShowConfigurations(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: IndexNode_ShowConfigurations_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IndexNodeServer).ShowConfigurations(ctx, req.(*internalpb.ShowConfigurationsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _IndexNode_GetMetrics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(milvuspb.GetMetricsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IndexNodeServer).GetMetrics(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: IndexNode_GetMetrics_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IndexNodeServer).GetMetrics(ctx, req.(*milvuspb.GetMetricsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _IndexNode_CreateJobV2_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CreateJobV2Request) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IndexNodeServer).CreateJobV2(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: IndexNode_CreateJobV2_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IndexNodeServer).CreateJobV2(ctx, req.(*CreateJobV2Request)) + } + return interceptor(ctx, in, info, handler) +} + +func _IndexNode_QueryJobsV2_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryJobsV2Request) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IndexNodeServer).QueryJobsV2(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: IndexNode_QueryJobsV2_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IndexNodeServer).QueryJobsV2(ctx, req.(*QueryJobsV2Request)) + } + return interceptor(ctx, in, info, handler) +} + +func _IndexNode_DropJobsV2_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DropJobsV2Request) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(IndexNodeServer).DropJobsV2(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: IndexNode_DropJobsV2_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(IndexNodeServer).DropJobsV2(ctx, req.(*DropJobsV2Request)) + } + return interceptor(ctx, in, info, handler) +} + +// IndexNode_ServiceDesc is the grpc.ServiceDesc for IndexNode service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var IndexNode_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "milvus.proto.index.IndexNode", + HandlerType: (*IndexNodeServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "GetComponentStates", + Handler: _IndexNode_GetComponentStates_Handler, + }, + { + MethodName: "GetStatisticsChannel", + Handler: _IndexNode_GetStatisticsChannel_Handler, + }, + { + MethodName: "CreateJob", + Handler: _IndexNode_CreateJob_Handler, + }, + { + MethodName: "QueryJobs", + Handler: _IndexNode_QueryJobs_Handler, + }, + { + MethodName: "DropJobs", + Handler: _IndexNode_DropJobs_Handler, + }, + { + MethodName: "GetJobStats", + Handler: _IndexNode_GetJobStats_Handler, + }, + { + MethodName: "ShowConfigurations", + Handler: _IndexNode_ShowConfigurations_Handler, + }, + { + MethodName: "GetMetrics", + Handler: _IndexNode_GetMetrics_Handler, + }, + { + MethodName: "CreateJobV2", + Handler: _IndexNode_CreateJobV2_Handler, + }, + { + MethodName: "QueryJobsV2", + Handler: _IndexNode_QueryJobsV2_Handler, + }, + { + MethodName: "DropJobsV2", + Handler: _IndexNode_DropJobsV2_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "worker.proto", +} diff --git a/pkg/streaming/util/message/builder.go b/pkg/streaming/util/message/builder.go index 0f941c6851bd4..527add53795d8 100644 --- a/pkg/streaming/util/message/builder.go +++ b/pkg/streaming/util/message/builder.go @@ -7,7 +7,7 @@ import ( "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" - "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb" + "github.com/milvus-io/milvus/pkg/proto/messagespb" "github.com/milvus-io/milvus/pkg/util/tsoutil" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/pkg/streaming/util/message/message_impl.go b/pkg/streaming/util/message/message_impl.go index 7e4a4c0be2498..25c9bcad9ea8d 100644 --- a/pkg/streaming/util/message/message_impl.go +++ b/pkg/streaming/util/message/message_impl.go @@ -3,7 +3,7 @@ package message import ( "fmt" - "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb" + "github.com/milvus-io/milvus/pkg/proto/messagespb" ) type messageImpl struct { diff --git a/pkg/streaming/util/message/message_type.go b/pkg/streaming/util/message/message_type.go index b6c74d08e2193..c09b49ba72a4b 100644 --- a/pkg/streaming/util/message/message_type.go +++ b/pkg/streaming/util/message/message_type.go @@ -3,7 +3,7 @@ package message import ( "strconv" - "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb" + "github.com/milvus-io/milvus/pkg/proto/messagespb" ) type MessageType messagespb.MessageType diff --git a/pkg/streaming/util/message/specialized_message.go b/pkg/streaming/util/message/specialized_message.go index a9fec13f4597f..780d8b0f57fcf 100644 --- a/pkg/streaming/util/message/specialized_message.go +++ b/pkg/streaming/util/message/specialized_message.go @@ -8,7 +8,7 @@ import ( "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" - "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb" + "github.com/milvus-io/milvus/pkg/proto/messagespb" ) type ( diff --git a/pkg/streaming/util/message/txn.go b/pkg/streaming/util/message/txn.go index 150f92634ac98..fe1615adcb3eb 100644 --- a/pkg/streaming/util/message/txn.go +++ b/pkg/streaming/util/message/txn.go @@ -3,7 +3,7 @@ package message import ( "time" - "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb" + "github.com/milvus-io/milvus/pkg/proto/messagespb" ) type ( diff --git a/pkg/streaming/util/message/txn_test.go b/pkg/streaming/util/message/txn_test.go index ce22af12b4494..cc6f4dc844d46 100644 --- a/pkg/streaming/util/message/txn_test.go +++ b/pkg/streaming/util/message/txn_test.go @@ -8,7 +8,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" - "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb" + "github.com/milvus-io/milvus/pkg/proto/messagespb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/walimplstest" ) diff --git a/pkg/streaming/util/options/deliver.go b/pkg/streaming/util/options/deliver.go index bb978250b854c..dfb61d61cc4d4 100644 --- a/pkg/streaming/util/options/deliver.go +++ b/pkg/streaming/util/options/deliver.go @@ -3,8 +3,8 @@ package options import ( "fmt" - "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/messagespb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" ) diff --git a/pkg/streaming/util/options/deliver_test.go b/pkg/streaming/util/options/deliver_test.go index 4c1dfad2ad377..1ac3b43b98e0c 100644 --- a/pkg/streaming/util/options/deliver_test.go +++ b/pkg/streaming/util/options/deliver_test.go @@ -6,7 +6,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/milvus-io/milvus/pkg/mocks/streaming/util/mock_message" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" ) diff --git a/pkg/streaming/util/types/pchannel_info.go b/pkg/streaming/util/types/pchannel_info.go index 12bfb7483c2d9..7db3860cddfa8 100644 --- a/pkg/streaming/util/types/pchannel_info.go +++ b/pkg/streaming/util/types/pchannel_info.go @@ -3,7 +3,7 @@ package types import ( "fmt" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) const ( diff --git a/pkg/streaming/util/types/pchannel_info_test.go b/pkg/streaming/util/types/pchannel_info_test.go index 0cf52829dffe3..4afcdd9e9dce8 100644 --- a/pkg/streaming/util/types/pchannel_info_test.go +++ b/pkg/streaming/util/types/pchannel_info_test.go @@ -5,7 +5,7 @@ import ( "github.com/stretchr/testify/assert" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" ) func TestPChannelInfo(t *testing.T) { diff --git a/pkg/streaming/util/types/streaming_node.go b/pkg/streaming/util/types/streaming_node.go index 0cca5798e19d1..bc4118cb30a65 100644 --- a/pkg/streaming/util/types/streaming_node.go +++ b/pkg/streaming/util/types/streaming_node.go @@ -7,8 +7,8 @@ import ( "google.golang.org/protobuf/proto" "google.golang.org/protobuf/types/known/anypb" - "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/messagespb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/util/typeutil" ) diff --git a/pkg/streaming/walimpls/impls/kafka/wal.go b/pkg/streaming/walimpls/impls/kafka/wal.go index 63d0bcb492044..3540beba21898 100644 --- a/pkg/streaming/walimpls/impls/kafka/wal.go +++ b/pkg/streaming/walimpls/impls/kafka/wal.go @@ -6,7 +6,7 @@ import ( "github.com/cockroachdb/errors" "github.com/confluentinc/confluent-kafka-go/kafka" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/walimpls" "github.com/milvus-io/milvus/pkg/streaming/walimpls/helper" diff --git a/pkg/streaming/walimpls/impls/pulsar/wal.go b/pkg/streaming/walimpls/impls/pulsar/wal.go index 65d33fdfc1246..6b2d14e90f7d1 100644 --- a/pkg/streaming/walimpls/impls/pulsar/wal.go +++ b/pkg/streaming/walimpls/impls/pulsar/wal.go @@ -6,7 +6,7 @@ import ( "github.com/apache/pulsar-client-go/pulsar" "go.uber.org/zap" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/walimpls" "github.com/milvus-io/milvus/pkg/streaming/walimpls/helper" diff --git a/pkg/streaming/walimpls/impls/rmq/wal.go b/pkg/streaming/walimpls/impls/rmq/wal.go index 6e22d23859249..d2f157fd36ec1 100644 --- a/pkg/streaming/walimpls/impls/rmq/wal.go +++ b/pkg/streaming/walimpls/impls/rmq/wal.go @@ -7,7 +7,7 @@ import ( "github.com/milvus-io/milvus/pkg/mq/common" "github.com/milvus-io/milvus/pkg/mq/mqimpl/rocksmq/client" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/walimpls" "github.com/milvus-io/milvus/pkg/streaming/walimpls/helper" diff --git a/pkg/streaming/walimpls/impls/walimplstest/wal.go b/pkg/streaming/walimpls/impls/walimplstest/wal.go index 7505754b6fbb4..68d18ab081fda 100644 --- a/pkg/streaming/walimpls/impls/walimplstest/wal.go +++ b/pkg/streaming/walimpls/impls/walimplstest/wal.go @@ -6,7 +6,7 @@ package walimplstest import ( "context" - "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/walimpls" "github.com/milvus-io/milvus/pkg/streaming/walimpls/helper" diff --git a/scripts/generate_proto.sh b/scripts/generate_proto.sh index 1b087e8360a10..c8851fac705ab 100755 --- a/scripts/generate_proto.sh +++ b/scripts/generate_proto.sh @@ -24,7 +24,7 @@ while [ -h "$SOURCE" ]; do # resolve $SOURCE until the file is no longer a symli done ROOT_DIR="$( cd -P "$( dirname "$SOURCE" )/.." && pwd )" -PROTO_DIR=$ROOT_DIR/internal/proto/ +PROTO_DIR=$ROOT_DIR/pkg/proto API_PROTO_DIR=$ROOT_DIR/cmake_build/thirdparty/milvus-proto/proto CPP_SRC_DIR=$ROOT_DIR/internal/core PROTOC_BIN=$ROOT_DIR/cmake_build/bin/protoc @@ -47,24 +47,21 @@ echo "using protoc-gen-go-grpc: $(which protoc-gen-go-grpc)" # official go code ship with the crate, so we need to generate it manually. pushd ${PROTO_DIR} -mkdir -p etcdpb -mkdir -p indexcgopb -mkdir -p cgopb - -mkdir -p internalpb -mkdir -p rootcoordpb - -mkdir -p segcorepb -mkdir -p clusteringpb -mkdir -p proxypb - -mkdir -p indexpb -mkdir -p datapb -mkdir -p querypb -mkdir -p planpb -mkdir -p streamingpb -mkdir -p workerpb - +mkdir -p ./etcdpb +mkdir -p ./indexcgopb +mkdir -p ./cgopb +mkdir -p ./internalpb +mkdir -p ./rootcoordpb +mkdir -p ./segcorepb +mkdir -p ./clusteringpb +mkdir -p ./proxypb +mkdir -p ./indexpb +mkdir -p ./datapb +mkdir -p ./querypb +mkdir -p ./planpb +mkdir -p ./workerpb +mkdir -p ./messagespb +mkdir -p ./streamingpb mkdir -p $ROOT_DIR/cmd/tools/migration/legacy/legacypb protoc_opt="${PROTOC_BIN} --proto_path=${API_PROTO_DIR} --proto_path=." @@ -81,7 +78,8 @@ ${protoc_opt} --go_out=paths=source_relative:./querypb --go-grpc_out=require_uni ${protoc_opt} --go_out=paths=source_relative:./planpb --go-grpc_out=require_unimplemented_servers=false,paths=source_relative:./planpb plan.proto|| { echo 'generate plan.proto failed'; exit 1; } ${protoc_opt} --go_out=paths=source_relative:./segcorepb --go-grpc_out=require_unimplemented_servers=false,paths=source_relative:./segcorepb segcore.proto|| { echo 'generate segcore.proto failed'; exit 1; } ${protoc_opt} --go_out=paths=source_relative:./clusteringpb --go-grpc_out=require_unimplemented_servers=false,paths=source_relative:./clusteringpb clustering.proto|| { echo 'generate clustering.proto failed'; exit 1; } - +${protoc_opt} --go_out=paths=source_relative:./messagespb --go-grpc_out=require_unimplemented_servers=false,paths=source_relative:./messagespb messages.proto || { echo 'generate messages.proto failed'; exit 1; } +${protoc_opt} --go_out=paths=source_relative:./streamingpb --go-grpc_out=require_unimplemented_servers=false,paths=source_relative:./streamingpb streaming.proto || { echo 'generate streamingpb.proto failed'; exit 1; } ${protoc_opt} --go_out=paths=source_relative:./workerpb --go-grpc_out=require_unimplemented_servers=false,paths=source_relative:./workerpb worker.proto|| { echo 'generate worker.proto failed'; exit 1; } ${protoc_opt} --proto_path=$ROOT_DIR/pkg/eventlog/ --go_out=paths=source_relative:../../pkg/eventlog/ --go-grpc_out=require_unimplemented_servers=false,paths=source_relative:../../pkg/eventlog/ event_log.proto || { echo 'generate event_log.proto failed'; exit 1; } @@ -99,15 +97,3 @@ ${protoc_opt} --cpp_out=$CPP_SRC_DIR/src/pb cgo_msg.proto|| { echo 'generate cgo ${protoc_opt} --cpp_out=$CPP_SRC_DIR/src/pb plan.proto|| { echo 'generate plan.proto failed'; exit 1; } popd - - -pushd $ROOT_DIR/pkg/streaming/proto - -mkdir -p messagespb -mkdir -p streamingpb - -# streaming node message protobuf -${protoc_opt} --go_out=paths=source_relative:./messagespb --go-grpc_out=require_unimplemented_servers=false,paths=source_relative:./messagespb messages.proto || { echo 'generate messagespb.proto failed'; exit 1; } -${protoc_opt} --go_out=paths=source_relative:./streamingpb --go-grpc_out=require_unimplemented_servers=false,paths=source_relative:./streamingpb streaming.proto || { echo 'generate streamingpb.proto failed'; exit 1; } - -popd diff --git a/tests/integration/balance/balance_test.go b/tests/integration/balance/balance_test.go index fca63ef6e8ae5..cf2f148ae3fad 100644 --- a/tests/integration/balance/balance_test.go +++ b/tests/integration/balance/balance_test.go @@ -32,8 +32,8 @@ 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/proto/querypb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/tests/integration/balance/channel_exclusive_balance_test.go b/tests/integration/balance/channel_exclusive_balance_test.go index fd9e9aeec5176..fd5714aec08b9 100644 --- a/tests/integration/balance/channel_exclusive_balance_test.go +++ b/tests/integration/balance/channel_exclusive_balance_test.go @@ -33,9 +33,9 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" grpcquerynode "github.com/milvus-io/milvus/internal/distributed/querynode" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metric" diff --git a/tests/integration/channel_balance/channel_balance_test.go b/tests/integration/channel_balance/channel_balance_test.go index 497843ff6e2bd..f136571d803c0 100644 --- a/tests/integration/channel_balance/channel_balance_test.go +++ b/tests/integration/channel_balance/channel_balance_test.go @@ -13,9 +13,9 @@ 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/proto/datapb" "github.com/milvus-io/milvus/internal/util/streamingutil" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/tests/integration" ) diff --git a/tests/integration/compaction/clustering_compaction_null_data_test.go b/tests/integration/compaction/clustering_compaction_null_data_test.go index b286e996bce90..43e9f82b30998 100644 --- a/tests/integration/compaction/clustering_compaction_null_data_test.go +++ b/tests/integration/compaction/clustering_compaction_null_data_test.go @@ -15,9 +15,9 @@ 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/proto/datapb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metric" diff --git a/tests/integration/compaction/clustering_compaction_test.go b/tests/integration/compaction/clustering_compaction_test.go index 7928973cf39b2..f1801575bca48 100644 --- a/tests/integration/compaction/clustering_compaction_test.go +++ b/tests/integration/compaction/clustering_compaction_test.go @@ -31,9 +31,9 @@ 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/proto/datapb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metric" diff --git a/tests/integration/compaction/l0_compaction_test.go b/tests/integration/compaction/l0_compaction_test.go index 80a204322041a..f495593539358 100644 --- a/tests/integration/compaction/l0_compaction_test.go +++ b/tests/integration/compaction/l0_compaction_test.go @@ -28,9 +28,9 @@ 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/proto/datapb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metric" diff --git a/tests/integration/compaction/l2_single_compaction_test.go b/tests/integration/compaction/l2_single_compaction_test.go index 43a7bea351ff1..c608092591f85 100644 --- a/tests/integration/compaction/l2_single_compaction_test.go +++ b/tests/integration/compaction/l2_single_compaction_test.go @@ -31,9 +31,9 @@ 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/proto/datapb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metric" diff --git a/tests/integration/compaction/mix_compaction_test.go b/tests/integration/compaction/mix_compaction_test.go index 987d28b22f083..ce49192d1d796 100644 --- a/tests/integration/compaction/mix_compaction_test.go +++ b/tests/integration/compaction/mix_compaction_test.go @@ -28,9 +28,9 @@ 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/proto/datapb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metric" diff --git a/tests/integration/crossclusterrouting/cross_cluster_routing_test.go b/tests/integration/crossclusterrouting/cross_cluster_routing_test.go index bb146fcea10e5..fff9c65bfe2fc 100644 --- a/tests/integration/crossclusterrouting/cross_cluster_routing_test.go +++ b/tests/integration/crossclusterrouting/cross_cluster_routing_test.go @@ -28,10 +28,10 @@ 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/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/proxypb" - "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/proto/workerpb" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/proxypb" + "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/proto/workerpb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/tests/integration/datanode/compaction_test.go b/tests/integration/datanode/compaction_test.go index b710977a0afe9..9944434d334cc 100644 --- a/tests/integration/datanode/compaction_test.go +++ b/tests/integration/datanode/compaction_test.go @@ -14,8 +14,8 @@ 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/proto/datapb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metric" "github.com/milvus-io/milvus/tests/integration" diff --git a/tests/integration/import/binlog_test.go b/tests/integration/import/binlog_test.go index f1a32c4502afa..61d2c6d746992 100644 --- a/tests/integration/import/binlog_test.go +++ b/tests/integration/import/binlog_test.go @@ -28,10 +28,10 @@ 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/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metric" diff --git a/tests/integration/import/dynamic_field_test.go b/tests/integration/import/dynamic_field_test.go index 93b522814405b..14b1ab1c7d89f 100644 --- a/tests/integration/import/dynamic_field_test.go +++ b/tests/integration/import/dynamic_field_test.go @@ -31,10 +31,10 @@ 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/proto/internalpb" "github.com/milvus-io/milvus/internal/util/importutilv2" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metric" diff --git a/tests/integration/import/import_test.go b/tests/integration/import/import_test.go index 415489eeedea9..2cfe97f18a07b 100644 --- a/tests/integration/import/import_test.go +++ b/tests/integration/import/import_test.go @@ -31,11 +31,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/proto/internalpb" "github.com/milvus-io/milvus/internal/util/importutilv2" "github.com/milvus-io/milvus/internal/util/indexparamcheck" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/metric" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/tests/integration/import/multi_vector_test.go b/tests/integration/import/multi_vector_test.go index b6920f099ed46..d11baa0217563 100644 --- a/tests/integration/import/multi_vector_test.go +++ b/tests/integration/import/multi_vector_test.go @@ -31,10 +31,10 @@ 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/proto/internalpb" "github.com/milvus-io/milvus/internal/util/importutilv2" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metric" diff --git a/tests/integration/import/partition_key_test.go b/tests/integration/import/partition_key_test.go index de1b71dc11776..8ff58df999160 100644 --- a/tests/integration/import/partition_key_test.go +++ b/tests/integration/import/partition_key_test.go @@ -31,9 +31,9 @@ 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/proto/internalpb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metric" diff --git a/tests/integration/import/util_test.go b/tests/integration/import/util_test.go index 5eb63a847493b..3a4e2072c56d5 100644 --- a/tests/integration/import/util_test.go +++ b/tests/integration/import/util_test.go @@ -34,12 +34,12 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/storage" pq "github.com/milvus-io/milvus/internal/util/importutilv2/parquet" "github.com/milvus-io/milvus/internal/util/testutil" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/tests/integration" ) diff --git a/tests/integration/levelzero/delete_on_growing_test.go b/tests/integration/levelzero/delete_on_growing_test.go index e96670b9ae881..a7bafdb895a03 100644 --- a/tests/integration/levelzero/delete_on_growing_test.go +++ b/tests/integration/levelzero/delete_on_growing_test.go @@ -23,7 +23,7 @@ import ( "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/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metric" diff --git a/tests/integration/levelzero/delete_partition_key_test.go b/tests/integration/levelzero/delete_partition_key_test.go index 426356c0cbfcc..e2a536e66a392 100644 --- a/tests/integration/levelzero/delete_partition_key_test.go +++ b/tests/integration/levelzero/delete_partition_key_test.go @@ -10,7 +10,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metric" diff --git a/tests/integration/meta_watcher.go b/tests/integration/meta_watcher.go index 1fe255d8a041f..2f7fdbbbe0994 100644 --- a/tests/integration/meta_watcher.go +++ b/tests/integration/meta_watcher.go @@ -28,10 +28,10 @@ import ( "go.uber.org/zap" "google.golang.org/protobuf/proto" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/querypb" ) // MetaWatcher to observe meta data of milvus cluster diff --git a/tests/integration/ops/suspend_node_test.go b/tests/integration/ops/suspend_node_test.go index 02ca0634f7bf1..a89c2fc7e4b18 100644 --- a/tests/integration/ops/suspend_node_test.go +++ b/tests/integration/ops/suspend_node_test.go @@ -24,9 +24,9 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" grpcquerynode "github.com/milvus-io/milvus/internal/distributed/querynode" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/internal/querycoordv2/session" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/tests/integration" diff --git a/tests/integration/partialsearch/partial_search_test.go b/tests/integration/partialsearch/partial_search_test.go index 2040a3c8bd584..c5fdd1d8d264c 100644 --- a/tests/integration/partialsearch/partial_search_test.go +++ b/tests/integration/partialsearch/partial_search_test.go @@ -30,9 +30,9 @@ 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/proto/querypb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" diff --git a/tests/integration/replicas/load/load_test.go b/tests/integration/replicas/load/load_test.go index 3609441cf52ba..7bd7ba22ad6c4 100644 --- a/tests/integration/replicas/load/load_test.go +++ b/tests/integration/replicas/load/load_test.go @@ -30,10 +30,10 @@ 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/rgpb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/tests/integration/rollingupgrade/manual_rolling_upgrade_test.go b/tests/integration/rollingupgrade/manual_rolling_upgrade_test.go index 44e0265230946..faaab97db1c7a 100644 --- a/tests/integration/rollingupgrade/manual_rolling_upgrade_test.go +++ b/tests/integration/rollingupgrade/manual_rolling_upgrade_test.go @@ -29,8 +29,8 @@ import ( "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/proto/querypb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metric" diff --git a/tests/integration/sealpolicies/seal_by_total_growing_test.go b/tests/integration/sealpolicies/seal_by_total_growing_test.go index d5f8a92440190..33253bab3b92b 100644 --- a/tests/integration/sealpolicies/seal_by_total_growing_test.go +++ b/tests/integration/sealpolicies/seal_by_total_growing_test.go @@ -27,9 +27,9 @@ 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/proto/datapb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/tests/integration/stats_task/stats_task_test.go b/tests/integration/stats_task/stats_task_test.go index d4616a2255311..dbcb10af0fa20 100644 --- a/tests/integration/stats_task/stats_task_test.go +++ b/tests/integration/stats_task/stats_task_test.go @@ -32,11 +32,11 @@ import ( "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/metastore/kv/binlog" - "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/hookutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metric" diff --git a/tests/integration/streaming/hello_streaming_test.go b/tests/integration/streaming/hello_streaming_test.go index d84d6518dd2c1..77549f027440c 100644 --- a/tests/integration/streaming/hello_streaming_test.go +++ b/tests/integration/streaming/hello_streaming_test.go @@ -30,10 +30,10 @@ 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/proto/datapb" "github.com/milvus-io/milvus/internal/util/streamingutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metric" diff --git a/tests/integration/target/target_test.go b/tests/integration/target/target_test.go index 43166330503f2..774cefe04e620 100644 --- a/tests/integration/target/target_test.go +++ b/tests/integration/target/target_test.go @@ -30,8 +30,8 @@ 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/proto/querypb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" From 7d32603d4db66986b054f2bf08c495a9ba15ee4f Mon Sep 17 00:00:00 2001 From: wt Date: Fri, 10 Jan 2025 10:54:57 +0800 Subject: [PATCH 09/34] test: add more varchar expressions for query (#39122) Signed-off-by: wangting0128 --- tests/python_client/common/common_func.py | 33 ++ .../testcases/test_mix_scenes.py | 282 ++++++++++++++---- 2 files changed, 257 insertions(+), 58 deletions(-) diff --git a/tests/python_client/common/common_func.py b/tests/python_client/common/common_func.py index c808872467a0a..ad1e248143bfe 100644 --- a/tests/python_client/common/common_func.py +++ b/tests/python_client/common/common_func.py @@ -2537,6 +2537,26 @@ def gen_modulo_expression(expr_fields): return exprs +def count_match_expr(values_l: list, rex_l: str, op: str, values_r: list, rex_r: str) -> list: + if len(values_l) != len(values_r): + raise ValueError(f"[count_match_expr] values not equal: {len(values_l)} != {len(values_r)}") + + res = [] + if op in ['and', '&&']: + for i in range(len(values_l)): + if re.search(rex_l, values_l[i]) and re.search(rex_r, values_r[i]): + res.append(i) + + elif op in ['or', '||']: + for i in range(len(values_l)): + if re.search(rex_l, values_l[i]) or re.search(rex_r, values_r[i]): + res.append(i) + + else: + raise ValueError(f"[count_match_expr] Not support op: {op}") + return res + + def gen_varchar_expression(expr_fields): exprs = [] for field in expr_fields: @@ -2551,6 +2571,19 @@ def gen_varchar_expression(expr_fields): return exprs +def gen_varchar_operation(expr_fields): + exprs = [] + for field in expr_fields: + exprs.extend([ + (Expr.EQ(field, '"a"').value, field, r'a'), + (Expr.GT(field, '"a"').value, field, r'[^a]'), + (Expr.GE(field, '"a"').value, field, r'.*'), + (Expr.LT(field, '"z"').value, field, r'[^z]'), + (Expr.LE(field, '"z"').value, field, r'.*') + ]) + return exprs + + def gen_varchar_unicode_expression(expr_fields): exprs = [] for field in expr_fields: diff --git a/tests/python_client/testcases/test_mix_scenes.py b/tests/python_client/testcases/test_mix_scenes.py index 800faae586ff7..56ea0e35d2bd9 100644 --- a/tests/python_client/testcases/test_mix_scenes.py +++ b/tests/python_client/testcases/test_mix_scenes.py @@ -41,7 +41,8 @@ def setup_class(self): name=cf.gen_unique_str("test_no_index_dql_expr"), schema=cf.set_collection_schema( fields=[self.primary_field, DataType.FLOAT16_VECTOR.name, DataType.BFLOAT16_VECTOR.name, - DataType.SPARSE_FLOAT_VECTOR.name, DataType.BINARY_VECTOR.name, *self().all_scalar_fields], + DataType.SPARSE_FLOAT_VECTOR.name, DataType.BINARY_VECTOR.name, + 'VARCHAR_1', *self().all_scalar_fields], field_params={ self.primary_field: FieldParams(is_primary=True).to_dict, DataType.FLOAT16_VECTOR.name: FieldParams(dim=3).to_dict, @@ -52,7 +53,9 @@ def setup_class(self): ) # prepare data (> 1024 triggering index building) - self.insert_data = cf.gen_field_values(self.collection_wrap.schema, nb=self.nb) + self.insert_data = cf.gen_field_values(self.collection_wrap.schema, nb=self.nb, default_values={ + 'VARCHAR_1': cf.gen_varchar_data(1, self.nb) + }) @pytest.fixture(scope="class", autouse=True) def prepare_data(self): @@ -88,7 +91,7 @@ def check_query_res(self, res, expr_field: str) -> list: return [(r[self.primary_field], r[expr_field], real_data[r[self.primary_field]]) for r in res if r[expr_field] != real_data[r[self.primary_field]]] - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize("expr, output_fields", [ (Expr.In(Expr.MOD('INT8', 13).subset, [0, 1, 2]).value, ['INT8']), (Expr.Nin(Expr.MOD('INT16', 100).subset, [10, 20, 30, 40]).value, ['INT16']), @@ -133,8 +136,9 @@ def test_no_index_query_with_modulo(self, expr, expr_field, limit): # check query response data assert self.check_query_res(res=res, expr_field=expr_field) == [] - @pytest.mark.tags(CaseLabel.L2) - @pytest.mark.parametrize("expr, expr_field, rex", cf.gen_varchar_expression(['VARCHAR'])) + @pytest.mark.tags(CaseLabel.L1) + @pytest.mark.parametrize("expr, expr_field, rex", + cf.gen_varchar_expression(['VARCHAR']) + cf.gen_varchar_operation(['VARCHAR_1'])) @pytest.mark.parametrize("limit", [1, 10, 3000]) def test_no_index_query_with_string(self, expr, expr_field, limit, rex): """ @@ -157,7 +161,37 @@ def test_no_index_query_with_string(self, expr, expr_field, limit, rex): # check query response data assert self.check_query_res(res=res, expr_field=expr_field) == [] - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) + @pytest.mark.parametrize("expr_l, expr_field_l, rex_l", cf.gen_varchar_expression(['VARCHAR'])) + @pytest.mark.parametrize("expr_r, expr_field_r, rex_r", cf.gen_varchar_operation(['VARCHAR_1'])) + @pytest.mark.parametrize("expr_obj, op", [(Expr.AND, 'and'), (Expr.OR, 'or')]) + @pytest.mark.parametrize("limit", [1, 10, 3000]) + def test_no_index_query_with_mix_string( + self, expr_l, expr_field_l, rex_l, expr_r, expr_field_r, rex_r, expr_obj, op, limit): + """ + target: + 1. check mix string fields expression + method: + 1. prepare some data + 2. query with the different expr and limit + 3. check query result + expected: + 1. query response equal to min(insert data, limit) + """ + # the total number of inserted data that matches the expression + expr_count = len(cf.count_match_expr(self.insert_data.get(expr_field_l, []), rex_l, op, + self.insert_data.get(expr_field_r, []), rex_r)) + + # query + res, _ = self.collection_wrap.query(expr=expr_obj(f"({expr_l})", f"({expr_r})").value, limit=limit, + output_fields=[expr_field_l, expr_field_r]) + assert len(res) == min(expr_count, limit), f"actual: {len(res)} == expect: {min(expr_count, limit)}" + + # check query response data + assert self.check_query_res(res=res, expr_field=expr_field_l) == [] + assert self.check_query_res(res=res, expr_field=expr_field_r) == [] + + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize( "expr, expr_field", cf.gen_number_operation(['INT8', 'INT16', 'INT32', 'INT64', 'FLOAT', 'DOUBLE'])) @pytest.mark.parametrize("limit", [1, 10, 3000]) @@ -182,7 +216,7 @@ def test_no_index_query_with_operation(self, expr, expr_field, limit): # check query response data assert self.check_query_res(res=res, expr_field=expr_field) == [] - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize("range_num, counts", [([-100, 200], 10), ([2000, 5000], 10), ([3000, 4000], 5)]) @pytest.mark.parametrize("expr_field", ['INT8', 'INT16', 'INT32', 'INT64']) @pytest.mark.parametrize("limit", [1, 10, 3000]) @@ -246,7 +280,8 @@ def setup_class(self): # init params self.primary_field, self.nb = "int64_pk", 3000 self.all_fields = [self.primary_field, DataType.FLOAT16_VECTOR.name, DataType.BFLOAT16_VECTOR.name, - DataType.SPARSE_FLOAT_VECTOR.name, DataType.BINARY_VECTOR.name, *self().all_scalar_fields] + DataType.SPARSE_FLOAT_VECTOR.name, DataType.BINARY_VECTOR.name, + 'VARCHAR_1', *self().all_scalar_fields] # create a collection with fields self.collection_wrap.init_collection( @@ -265,6 +300,7 @@ def setup_class(self): # prepare data (> 1024 triggering index building) self.insert_data = cf.gen_field_values(self.collection_wrap.schema, nb=self.nb, default_values={ 'VARCHAR': cf.gen_varchar_data(3, self.nb), + 'VARCHAR_1': cf.gen_varchar_data(1, self.nb), 'ARRAY_VARCHAR': [cf.gen_varchar_data(length=2, nb=random.randint(0, 10)) for _ in range(self.nb)] }) @@ -282,7 +318,7 @@ def prepare_data(self): **DefaultVectorIndexParams.SPARSE_INVERTED_INDEX(DataType.SPARSE_FLOAT_VECTOR.name), **DefaultVectorIndexParams.BIN_IVF_FLAT(DataType.BINARY_VECTOR.name), # build Hybrid index - **DefaultScalarIndexParams.list_default([self.primary_field] + self.all_index_scalar_fields) + **DefaultScalarIndexParams.list_default([self.primary_field, 'VARCHAR_1'] + self.all_index_scalar_fields) } self.build_multi_index(index_params=index_params) assert sorted([n.field_name for n in self.collection_wrap.indexes]) == sorted(index_params.keys()) @@ -330,8 +366,9 @@ def test_hybrid_index_query_with_modulo(self, expr, expr_field, limit): # check query response data assert self.check_query_res(res=res, expr_field=expr_field) == [] - @pytest.mark.tags(CaseLabel.L2) - @pytest.mark.parametrize("expr, expr_field, rex", cf.gen_varchar_expression(['VARCHAR'])) + @pytest.mark.tags(CaseLabel.L1) + @pytest.mark.parametrize("expr, expr_field, rex", + cf.gen_varchar_expression(['VARCHAR']) + cf.gen_varchar_operation(['VARCHAR_1'])) @pytest.mark.parametrize("limit", [1, 10, 3000]) def test_hybrid_index_query_with_string(self, expr, expr_field, limit, rex): """ @@ -354,7 +391,37 @@ def test_hybrid_index_query_with_string(self, expr, expr_field, limit, rex): # check query response data assert self.check_query_res(res=res, expr_field=expr_field) == [] - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) + @pytest.mark.parametrize("expr_l, expr_field_l, rex_l", cf.gen_varchar_expression(['VARCHAR'])) + @pytest.mark.parametrize("expr_r, expr_field_r, rex_r", cf.gen_varchar_operation(['VARCHAR_1'])) + @pytest.mark.parametrize("expr_obj, op", [(Expr.AND, 'and'), (Expr.OR, 'or')]) + @pytest.mark.parametrize("limit", [1, 10, 3000]) + def test_hybrid_index_query_with_mix_string( + self, expr_l, expr_field_l, rex_l, expr_r, expr_field_r, rex_r, expr_obj, op, limit): + """ + target: + 1. check mix string fields expression + method: + 1. prepare some data + 2. query with the different expr and limit + 3. check query result + expected: + 1. query response equal to min(insert data, limit) + """ + # the total number of inserted data that matches the expression + expr_count = len(cf.count_match_expr(self.insert_data.get(expr_field_l, []), rex_l, op, + self.insert_data.get(expr_field_r, []), rex_r)) + + # query + res, _ = self.collection_wrap.query(expr=expr_obj(f"({expr_l})", f"({expr_r})").value, limit=limit, + output_fields=[expr_field_l, expr_field_r]) + assert len(res) == min(expr_count, limit), f"actual: {len(res)} == expect: {min(expr_count, limit)}" + + # check query response data + assert self.check_query_res(res=res, expr_field=expr_field_l) == [] + assert self.check_query_res(res=res, expr_field=expr_field_r) == [] + + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize( "expr, expr_field", cf.gen_number_operation(['INT8', 'INT16', 'INT32', 'INT64', 'FLOAT', 'DOUBLE'])) @pytest.mark.parametrize("limit", [1, 10, 3000]) @@ -379,7 +446,7 @@ def test_hybrid_index_query_with_operation(self, expr, expr_field, limit): # check query response data assert self.check_query_res(res=res, expr_field=expr_field) == [] - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize("range_num, counts", [([-100, 200], 10), ([2000, 5000], 10), ([3000, 4000], 5)]) @pytest.mark.parametrize("expr_field", ['INT8', 'INT16', 'INT32', 'INT64']) @pytest.mark.parametrize("limit", [1, 10, 3000]) @@ -425,7 +492,7 @@ def test_hybrid_index_query_with_int_in(self, range_num, counts, expr_field, lim check_task=CheckTasks.check_query_results, check_items={"exp_res": [{"count(*)": not_in_count}]}) - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize("range_num, counts", [([1, 3], 50), ([2, 5], 50), ([3, 3], 100)]) @pytest.mark.parametrize("limit", [1, 10, 3000]) @pytest.mark.parametrize("expr_field", ['VARCHAR']) @@ -471,7 +538,7 @@ def test_hybrid_index_query_with_varchar_in(self, range_num, counts, limit, expr check_task=CheckTasks.check_query_results, check_items={"exp_res": [{"count(*)": not_in_count}]}) - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize("length", [0, 5, 11]) @pytest.mark.parametrize("expr_obj", [Expr.array_length, Expr.ARRAY_LENGTH]) @pytest.mark.parametrize("expr_field", ['ARRAY_VARCHAR']) @@ -511,7 +578,7 @@ def test_hybrid_index_query_count(self): self.collection_wrap.query(expr='', output_fields=['count(*)'], check_task=CheckTasks.check_query_results, check_items={"exp_res": [{"count(*)": self.nb}]}) - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) def test_hybrid_index_search_output_fields(self): """ target: @@ -548,7 +615,8 @@ def setup_class(self): # init params self.primary_field, self.nb = "int64_pk", 3000 self.all_fields = [self.primary_field, DataType.FLOAT16_VECTOR.name, DataType.BFLOAT16_VECTOR.name, - DataType.SPARSE_FLOAT_VECTOR.name, DataType.BINARY_VECTOR.name, *self().all_scalar_fields] + DataType.SPARSE_FLOAT_VECTOR.name, DataType.BINARY_VECTOR.name, + 'VARCHAR_1', *self().all_scalar_fields] # create a collection with fields self.collection_wrap.init_collection( @@ -567,6 +635,7 @@ def setup_class(self): # prepare data (> 1024 triggering index building) self.insert_data = cf.gen_field_values(self.collection_wrap.schema, nb=self.nb, default_values={ 'VARCHAR': cf.gen_varchar_data(3, self.nb), + 'VARCHAR_1': cf.gen_varchar_data(1, self.nb), 'ARRAY_VARCHAR': [cf.gen_varchar_data(length=2, nb=random.randint(0, 10)) for _ in range(self.nb)] }) @@ -584,7 +653,8 @@ def prepare_data(self): **DefaultVectorIndexParams.SPARSE_WAND(DataType.SPARSE_FLOAT_VECTOR.name), **DefaultVectorIndexParams.BIN_FLAT(DataType.BINARY_VECTOR.name), # build INVERTED index - **DefaultScalarIndexParams.list_inverted([self.primary_field] + self.inverted_support_dtype_names) + **DefaultScalarIndexParams.list_inverted( + [self.primary_field, 'VARCHAR_1'] + self.inverted_support_dtype_names) } self.build_multi_index(index_params=index_params) assert sorted([n.field_name for n in self.collection_wrap.indexes]) == sorted(index_params.keys()) @@ -632,8 +702,9 @@ def test_inverted_index_query_with_modulo(self, expr, expr_field, limit): # check query response data assert self.check_query_res(res=res, expr_field=expr_field) == [] - @pytest.mark.tags(CaseLabel.L2) - @pytest.mark.parametrize("expr, expr_field, rex", cf.gen_varchar_expression(['VARCHAR'])) + @pytest.mark.tags(CaseLabel.L1) + @pytest.mark.parametrize("expr, expr_field, rex", + cf.gen_varchar_expression(['VARCHAR']) + cf.gen_varchar_operation(['VARCHAR_1'])) @pytest.mark.parametrize("limit", [1, 10, 3000]) def test_inverted_index_query_with_string(self, expr, expr_field, limit, rex): """ @@ -656,7 +727,37 @@ def test_inverted_index_query_with_string(self, expr, expr_field, limit, rex): # check query response data assert self.check_query_res(res=res, expr_field=expr_field) == [] - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) + @pytest.mark.parametrize("expr_l, expr_field_l, rex_l", cf.gen_varchar_expression(['VARCHAR'])) + @pytest.mark.parametrize("expr_r, expr_field_r, rex_r", cf.gen_varchar_operation(['VARCHAR_1'])) + @pytest.mark.parametrize("expr_obj, op", [(Expr.AND, 'and'), (Expr.OR, 'or')]) + @pytest.mark.parametrize("limit", [1, 10, 3000]) + def test_inverted_index_query_with_mix_string( + self, expr_l, expr_field_l, rex_l, expr_r, expr_field_r, rex_r, expr_obj, op, limit): + """ + target: + 1. check mix string fields expression + method: + 1. prepare some data + 2. query with the different expr and limit + 3. check query result + expected: + 1. query response equal to min(insert data, limit) + """ + # the total number of inserted data that matches the expression + expr_count = len(cf.count_match_expr(self.insert_data.get(expr_field_l, []), rex_l, op, + self.insert_data.get(expr_field_r, []), rex_r)) + + # query + res, _ = self.collection_wrap.query(expr=expr_obj(f"({expr_l})", f"({expr_r})").value, limit=limit, + output_fields=[expr_field_l, expr_field_r]) + assert len(res) == min(expr_count, limit), f"actual: {len(res)} == expect: {min(expr_count, limit)}" + + # check query response data + assert self.check_query_res(res=res, expr_field=expr_field_l) == [] + assert self.check_query_res(res=res, expr_field=expr_field_r) == [] + + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize( "expr, expr_field", cf.gen_number_operation(['INT8', 'INT16', 'INT32', 'INT64', 'FLOAT', 'DOUBLE'])) @pytest.mark.parametrize("limit", [1, 10, 3000]) @@ -681,7 +782,7 @@ def test_inverted_index_query_with_operation(self, expr, expr_field, limit): # check query response data assert self.check_query_res(res=res, expr_field=expr_field) == [] - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize("range_num, counts", [([-100, 200], 10), ([2000, 5000], 10), ([3000, 4000], 5)]) @pytest.mark.parametrize("expr_field", ['INT8', 'INT16', 'INT32', 'INT64']) @pytest.mark.parametrize("limit", [1, 10, 3000]) @@ -727,7 +828,7 @@ def test_inverted_index_query_with_int_in(self, range_num, counts, expr_field, l check_task=CheckTasks.check_query_results, check_items={"exp_res": [{"count(*)": not_in_count}]}) - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize("range_num, counts", [([1, 3], 50), ([2, 5], 50), ([3, 3], 100)]) @pytest.mark.parametrize("limit", [1, 10, 3000]) @pytest.mark.parametrize("expr_field", ['VARCHAR']) @@ -773,7 +874,7 @@ def test_inverted_index_query_with_varchar_in(self, range_num, counts, limit, ex check_task=CheckTasks.check_query_results, check_items={"exp_res": [{"count(*)": not_in_count}]}) - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize("length", [0, 5, 11]) @pytest.mark.parametrize("expr_obj", [Expr.array_length, Expr.ARRAY_LENGTH]) @pytest.mark.parametrize("expr_field", ['ARRAY_VARCHAR']) @@ -815,7 +916,8 @@ def setup_class(self): # init params self.primary_field, self.nb = "int64_pk", 3000 self.all_fields = [self.primary_field, DataType.FLOAT16_VECTOR.name, DataType.BFLOAT16_VECTOR.name, - DataType.SPARSE_FLOAT_VECTOR.name, DataType.BINARY_VECTOR.name, *self().all_scalar_fields] + DataType.SPARSE_FLOAT_VECTOR.name, DataType.BINARY_VECTOR.name, + "VARCHAR_1", *self().all_scalar_fields] # create a collection with fields self.collection_wrap.init_collection( @@ -834,6 +936,7 @@ def setup_class(self): # prepare data (> 1024 triggering index building) self.insert_data = cf.gen_field_values(self.collection_wrap.schema, nb=self.nb, default_values={ 'VARCHAR': cf.gen_varchar_data(3, self.nb), + 'VARCHAR_1': cf.gen_varchar_data(1, self.nb), 'ARRAY_VARCHAR': [cf.gen_varchar_data(length=2, nb=random.randint(0, 10)) for _ in range(self.nb)] }) @@ -851,7 +954,7 @@ def prepare_data(self): **DefaultVectorIndexParams.SPARSE_WAND(DataType.SPARSE_FLOAT_VECTOR.name), **DefaultVectorIndexParams.BIN_IVF_FLAT(DataType.BINARY_VECTOR.name), # build BITMAP index - **DefaultScalarIndexParams.list_bitmap(self.bitmap_support_dtype_names) + **DefaultScalarIndexParams.list_bitmap(["VARCHAR_1"] + self.bitmap_support_dtype_names) } self.build_multi_index(index_params=index_params) assert sorted([n.field_name for n in self.collection_wrap.indexes]) == sorted(index_params.keys()) @@ -923,8 +1026,9 @@ def test_bitmap_index_query_with_modulo(self, expr, expr_field, limit): # check query response data assert self.check_query_res(res=res, expr_field=expr_field) == [] - @pytest.mark.tags(CaseLabel.L2) - @pytest.mark.parametrize("expr, expr_field, rex", cf.gen_varchar_expression(['VARCHAR'])) + @pytest.mark.tags(CaseLabel.L1) + @pytest.mark.parametrize("expr, expr_field, rex", + cf.gen_varchar_expression(['VARCHAR']) + cf.gen_varchar_operation(['VARCHAR_1'])) @pytest.mark.parametrize("limit", [1, 10, 3000]) def test_bitmap_index_query_with_string(self, expr, expr_field, limit, rex): """ @@ -947,7 +1051,37 @@ def test_bitmap_index_query_with_string(self, expr, expr_field, limit, rex): # check query response data assert self.check_query_res(res=res, expr_field=expr_field) == [] - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) + @pytest.mark.parametrize("expr_l, expr_field_l, rex_l", cf.gen_varchar_expression(['VARCHAR'])) + @pytest.mark.parametrize("expr_r, expr_field_r, rex_r", cf.gen_varchar_operation(['VARCHAR_1'])) + @pytest.mark.parametrize("expr_obj, op", [(Expr.AND, 'and'), (Expr.OR, 'or')]) + @pytest.mark.parametrize("limit", [1, 10, 3000]) + def test_bitmap_index_query_with_mix_string( + self, expr_l, expr_field_l, rex_l, expr_r, expr_field_r, rex_r, expr_obj, op, limit): + """ + target: + 1. check mix string fields expression + method: + 1. prepare some data + 2. query with the different expr and limit + 3. check query result + expected: + 1. query response equal to min(insert data, limit) + """ + # the total number of inserted data that matches the expression + expr_count = len(cf.count_match_expr(self.insert_data.get(expr_field_l, []), rex_l, op, + self.insert_data.get(expr_field_r, []), rex_r)) + + # query + res, _ = self.collection_wrap.query(expr=expr_obj(f"({expr_l})", f"({expr_r})").value, limit=limit, + output_fields=[expr_field_l, expr_field_r]) + assert len(res) == min(expr_count, limit), f"actual: {len(res)} == expect: {min(expr_count, limit)}" + + # check query response data + assert self.check_query_res(res=res, expr_field=expr_field_l) == [] + assert self.check_query_res(res=res, expr_field=expr_field_r) == [] + + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize( "expr, expr_field", cf.gen_number_operation(['INT8', 'INT16', 'INT32', 'INT64', 'FLOAT', 'DOUBLE'])) @pytest.mark.parametrize("limit", [1, 10, 3000]) @@ -972,7 +1106,7 @@ def test_bitmap_index_query_with_operation(self, expr, expr_field, limit): # check query response data assert self.check_query_res(res=res, expr_field=expr_field) == [] - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize("range_num, counts", [([-100, 200], 10), ([2000, 5000], 10), ([3000, 4000], 5)]) @pytest.mark.parametrize("expr_field", ['INT8', 'INT16', 'INT32', 'INT64']) @pytest.mark.parametrize("limit", [1, 10, 3000]) @@ -1018,7 +1152,7 @@ def test_bitmap_index_query_with_int_in(self, range_num, counts, expr_field, lim check_task=CheckTasks.check_query_results, check_items={"exp_res": [{"count(*)": not_in_count}]}) - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize("range_num, counts", [([1, 3], 50), ([2, 5], 50), ([3, 3], 100)]) @pytest.mark.parametrize("limit", [1, 10, 3000]) @pytest.mark.parametrize("expr_field", ['VARCHAR']) @@ -1064,7 +1198,7 @@ def test_bitmap_index_query_with_varchar_in(self, range_num, counts, limit, expr check_task=CheckTasks.check_query_results, check_items={"exp_res": [{"count(*)": not_in_count}]}) - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize("length", [0, 5, 11]) @pytest.mark.parametrize("expr_obj", [Expr.array_length, Expr.ARRAY_LENGTH]) @pytest.mark.parametrize("expr_field", ['ARRAY_VARCHAR']) @@ -1104,7 +1238,7 @@ def test_bitmap_index_query_count(self): self.collection_wrap.query(expr='', output_fields=['count(*)'], check_task=CheckTasks.check_query_results, check_items={"exp_res": [{"count(*)": self.nb}]}) - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize("limit", [10, 1000]) @pytest.mark.parametrize("group_by_field", ['INT8', 'INT16', 'INT32', 'INT64', 'BOOL', 'VARCHAR']) @pytest.mark.parametrize( @@ -1134,7 +1268,7 @@ def test_bitmap_index_search_group_by(self, limit, group_by_field, dim, search_p assert len(values) == len(set(values)), f"values: {values}, output_values:{output_values}" - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize("batch_size", [10, 1000]) def test_bitmap_index_search_iterator(self, batch_size): """ @@ -1151,7 +1285,7 @@ def test_bitmap_index_search_iterator(self, batch_size): cf.gen_vectors(nb=1, dim=3, vector_data_type=vector_field), vector_field, search_params, batch_size, expr='INT16 > 15', check_task=CheckTasks.check_search_iterator, check_items={"batch_size": batch_size}) - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) def test_bitmap_index_search_output_fields(self): """ target: @@ -1170,7 +1304,7 @@ def test_bitmap_index_search_output_fields(self): check_items={"nq": nq, "ids": self.insert_data.get(self.primary_field), "limit": limit, "output_fields": self.all_fields}) - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) def test_bitmap_index_hybrid_search(self): """ target: @@ -1223,7 +1357,7 @@ def setup_class(self): # init params self.primary_field, self.nb = "int64_pk", 3000 - self.all_fields = [self.primary_field, DataType.FLOAT_VECTOR.name, *self().all_scalar_fields] + self.all_fields = [self.primary_field, DataType.FLOAT_VECTOR.name, 'VARCHAR_1', *self().all_scalar_fields] # create a collection with fields self.collection_wrap.init_collection( @@ -1239,6 +1373,7 @@ def setup_class(self): # prepare data (> 1024 triggering index building) self.insert_data = cf.gen_field_values(self.collection_wrap.schema, nb=self.nb, default_values={ 'VARCHAR': cf.gen_varchar_data(3, self.nb), + 'VARCHAR_1': cf.gen_varchar_data(1, self.nb), 'ARRAY_VARCHAR': [cf.gen_varchar_data(length=2, nb=random.randint(0, 10)) for _ in range(self.nb)] }) @@ -1253,13 +1388,13 @@ def prepare_data(self): index_params = { **DefaultVectorIndexParams.HNSW(DataType.FLOAT_VECTOR.name), # build BITMAP index - **DefaultScalarIndexParams.list_bitmap(self.bitmap_support_dtype_names) + **DefaultScalarIndexParams.list_bitmap(['VARCHAR_1'] + self.bitmap_support_dtype_names) } self.build_multi_index(index_params=index_params) assert sorted([n.field_name for n in self.collection_wrap.indexes]) == sorted(index_params.keys()) # enable offset cache - for index_name in self.bitmap_support_dtype_names: + for index_name in ['VARCHAR_1'] + self.bitmap_support_dtype_names: self.collection_wrap.alter_index(index_name=index_name, extra_params=AlterIndexParams.index_offset_cache()) # load collection @@ -1279,7 +1414,7 @@ def check_query_res(self, res, expr_field: str) -> list: return [(r[self.primary_field], r[expr_field], real_data[r[self.primary_field]]) for r in res if r[expr_field] != real_data[r[self.primary_field]]] - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize("expr, expr_field", cf.gen_modulo_expression(['INT8', 'INT16', 'INT32', 'INT64'])) @pytest.mark.parametrize("limit", [1, 10]) def test_bitmap_offset_cache_query_with_modulo(self, expr, expr_field, limit): @@ -1304,8 +1439,9 @@ def test_bitmap_offset_cache_query_with_modulo(self, expr, expr_field, limit): # check query response data assert self.check_query_res(res=res, expr_field=expr_field) == [] - @pytest.mark.tags(CaseLabel.L2) - @pytest.mark.parametrize("expr, expr_field, rex", cf.gen_varchar_expression(['VARCHAR'])) + @pytest.mark.tags(CaseLabel.L1) + @pytest.mark.parametrize("expr, expr_field, rex", + cf.gen_varchar_expression(['VARCHAR']) + cf.gen_varchar_operation(['VARCHAR_1'])) @pytest.mark.parametrize("limit", [1, 10]) def test_bitmap_offset_cache_query_with_string(self, expr, expr_field, limit, rex): """ @@ -1328,7 +1464,37 @@ def test_bitmap_offset_cache_query_with_string(self, expr, expr_field, limit, re # check query response data assert self.check_query_res(res=res, expr_field=expr_field) == [] - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) + @pytest.mark.parametrize("expr_l, expr_field_l, rex_l", cf.gen_varchar_expression(['VARCHAR'])) + @pytest.mark.parametrize("expr_r, expr_field_r, rex_r", cf.gen_varchar_operation(['VARCHAR_1'])) + @pytest.mark.parametrize("expr_obj, op", [(Expr.AND, 'and'), (Expr.OR, 'or')]) + @pytest.mark.parametrize("limit", [1, 10, 3000]) + def test_bitmap_offset_cache_query_with_mix_string( + self, expr_l, expr_field_l, rex_l, expr_r, expr_field_r, rex_r, expr_obj, op, limit): + """ + target: + 1. check mix string fields expression + method: + 1. prepare some data + 2. query with the different expr and limit + 3. check query result + expected: + 1. query response equal to min(insert data, limit) + """ + # the total number of inserted data that matches the expression + expr_count = len(cf.count_match_expr(self.insert_data.get(expr_field_l, []), rex_l, op, + self.insert_data.get(expr_field_r, []), rex_r)) + + # query + res, _ = self.collection_wrap.query(expr=expr_obj(f"({expr_l})", f"({expr_r})").value, limit=limit, + output_fields=[expr_field_l, expr_field_r]) + assert len(res) == min(expr_count, limit), f"actual: {len(res)} == expect: {min(expr_count, limit)}" + + # check query response data + assert self.check_query_res(res=res, expr_field=expr_field_l) == [] + assert self.check_query_res(res=res, expr_field=expr_field_r) == [] + + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize( "expr, expr_field", cf.gen_number_operation(['INT8', 'INT16', 'INT32', 'INT64', 'FLOAT', 'DOUBLE'])) @pytest.mark.parametrize("limit", [1, 10]) @@ -1353,7 +1519,7 @@ def test_bitmap_offset_cache_query_with_operation(self, expr, expr_field, limit) # check query response data assert self.check_query_res(res=res, expr_field=expr_field) == [] - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize("range_num, counts", [([-100, 200], 10), ([2000, 5000], 10), ([3000, 4000], 5)]) @pytest.mark.parametrize("expr_field", ['INT8', 'INT16', 'INT32', 'INT64']) @pytest.mark.parametrize("limit", [1, 10, 3000]) @@ -1399,7 +1565,7 @@ def test_bitmap_offset_cache_query_with_int_in(self, range_num, counts, expr_fie check_task=CheckTasks.check_query_results, check_items={"exp_res": [{"count(*)": not_in_count}]}) - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize("range_num, counts", [([1, 3], 50), ([2, 5], 50), ([3, 3], 100)]) @pytest.mark.parametrize("limit", [1, 10, 3000]) @pytest.mark.parametrize("expr_field", ['VARCHAR']) @@ -1445,7 +1611,7 @@ def test_bitmap_offset_cache_query_with_varchar_in(self, range_num, counts, limi check_task=CheckTasks.check_query_results, check_items={"exp_res": [{"count(*)": not_in_count}]}) - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize("length", [0, 5, 11]) @pytest.mark.parametrize("expr_obj", [Expr.array_length, Expr.ARRAY_LENGTH]) @pytest.mark.parametrize("expr_field", ['ARRAY_VARCHAR']) @@ -1469,7 +1635,7 @@ def test_bitmap_offset_cache_query_array_length_count(self, length, expr_obj, ex self.collection_wrap.query(expr=expr, output_fields=['count(*)'], check_task=CheckTasks.check_query_results, check_items={"exp_res": [{"count(*)": expr_count}]}) - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) def test_bitmap_offset_cache_query_count(self): """ target: @@ -1485,7 +1651,7 @@ def test_bitmap_offset_cache_query_count(self): self.collection_wrap.query(expr='', output_fields=['count(*)'], check_task=CheckTasks.check_query_results, check_items={"exp_res": [{"count(*)": self.nb}]}) - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) def test_bitmap_offset_cache_search_output_fields(self): """ target: @@ -1504,7 +1670,7 @@ def test_bitmap_offset_cache_search_output_fields(self): check_items={"nq": nq, "ids": self.insert_data.get(self.primary_field), "limit": limit, "output_fields": self.all_fields}) - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) def test_bitmap_offset_cache_hybrid_search(self): """ target: @@ -1604,7 +1770,7 @@ def check_query_res(self, res, expr_field: str) -> list: return [(r[self.primary_field], r[expr_field], real_data[r[self.primary_field]]) for r in res if r[expr_field] != real_data[r[self.primary_field]]] - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize("expr, expr_field", cf.gen_modulo_expression(['INT8', 'INT16', 'INT32', 'INT64'])) @pytest.mark.parametrize("limit", [1, 10]) def test_bitmap_mmap_query_with_modulo(self, expr, expr_field, limit): @@ -1629,7 +1795,7 @@ def test_bitmap_mmap_query_with_modulo(self, expr, expr_field, limit): # check query response data assert self.check_query_res(res=res, expr_field=expr_field) == [] - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize("expr, expr_field, rex", cf.gen_varchar_expression(['VARCHAR'])) @pytest.mark.parametrize("limit", [1, 10]) def test_bitmap_mmap_query_with_string(self, expr, expr_field, limit, rex): @@ -1653,7 +1819,7 @@ def test_bitmap_mmap_query_with_string(self, expr, expr_field, limit, rex): # check query response data assert self.check_query_res(res=res, expr_field=expr_field) == [] - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize( "expr, expr_field", cf.gen_number_operation(['INT8', 'INT16', 'INT32', 'INT64', 'FLOAT', 'DOUBLE'])) @pytest.mark.parametrize("limit", [1, 10]) @@ -1678,7 +1844,7 @@ def test_bitmap_mmap_query_with_operation(self, expr, expr_field, limit): # check query response data assert self.check_query_res(res=res, expr_field=expr_field) == [] - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize("range_num, counts", [([-100, 200], 10), ([2000, 5000], 10), ([3000, 4000], 5)]) @pytest.mark.parametrize("expr_field", ['INT8', 'INT16', 'INT32', 'INT64']) @pytest.mark.parametrize("limit", [1, 10, 3000]) @@ -1724,7 +1890,7 @@ def test_bitmap_mmap_query_with_int_in(self, range_num, counts, expr_field, limi check_task=CheckTasks.check_query_results, check_items={"exp_res": [{"count(*)": not_in_count}]}) - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) def test_bitmap_mmap_query_count(self): """ target: @@ -1740,7 +1906,7 @@ def test_bitmap_mmap_query_count(self): self.collection_wrap.query(expr='', output_fields=['count(*)'], check_task=CheckTasks.check_query_results, check_items={"exp_res": [{"count(*)": self.nb}]}) - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) def test_bitmap_mmap_search_output_fields(self): """ target: @@ -1759,7 +1925,7 @@ def test_bitmap_mmap_search_output_fields(self): check_items={"nq": nq, "ids": self.insert_data.get(self.primary_field), "limit": limit, "output_fields": self.all_fields}) - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) def test_bitmap_mmap_hybrid_search(self): """ target: @@ -1858,7 +2024,7 @@ def prepare_data(self): # load collection self.collection_wrap.load() - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize("expr, expr_field, rex", cf.gen_varchar_unicode_expression(['VARCHAR_BITMAP', 'VARCHAR_INVERTED'])) @pytest.mark.parametrize("limit", [1, 10, 3000]) @@ -1880,7 +2046,7 @@ def test_index_unicode_string_query(self, expr, expr_field, limit, rex): res, _ = self.collection_wrap.query(expr=expr, limit=limit, output_fields=[expr_field]) assert len(res) == min(expr_count, limit), f"actual: {len(res)} == expect: {min(expr_count, limit)}" - @pytest.mark.tags(CaseLabel.L2) + @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize("obj", cf.gen_varchar_unicode_expression_array( ['ARRAY_VARCHAR_BITMAP', 'ARRAY_VARCHAR_INVERTED', 'ARRAY_VARCHAR_NoIndex'])) @pytest.mark.parametrize("limit", [1, 10, 3000]) From e5eb1159e2c3dd0dd1561c32991d2d9b95a2fc5d Mon Sep 17 00:00:00 2001 From: Xianhui Lin <35839735+JsDove@users.noreply.github.com> Date: Fri, 10 Jan 2025 12:08:57 +0800 Subject: [PATCH 10/34] fix: Refine the err msg of alter index invalid (#39137) fix: Refine the err msg of alter index invalid issue: https://github.com/milvus-io/milvus/issues/38517 Signed-off-by: Xianhui.Lin --- internal/proxy/task_index.go | 4 ++-- tests/python_client/common/code_mapping.py | 2 +- tests/python_client/milvus_client/test_milvus_client_alter.py | 2 +- tests/python_client/testcases/test_index.py | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/internal/proxy/task_index.go b/internal/proxy/task_index.go index 5afeae1a17c7d..29f28c154a513 100644 --- a/internal/proxy/task_index.go +++ b/internal/proxy/task_index.go @@ -619,13 +619,13 @@ func (t *alterIndexTask) PreExecute(ctx context.Context) error { if len(t.req.GetExtraParams()) > 0 { for _, param := range t.req.GetExtraParams() { if !indexparams.IsConfigableIndexParam(param.GetKey()) { - return merr.WrapErrParameterInvalidMsg("%s is not a configable index proptery", param.GetKey()) + return merr.WrapErrParameterInvalidMsg("%s is not a configable index property", param.GetKey()) } } } else if len(t.req.GetDeleteKeys()) > 0 { for _, param := range t.req.GetDeleteKeys() { if !indexparams.IsConfigableIndexParam(param) { - return merr.WrapErrParameterInvalidMsg("%s is not a configable index proptery", param) + return merr.WrapErrParameterInvalidMsg("%s is not a configable index property", param) } } } diff --git a/tests/python_client/common/code_mapping.py b/tests/python_client/common/code_mapping.py index 337fffa07669d..d1d9abca14845 100644 --- a/tests/python_client/common/code_mapping.py +++ b/tests/python_client/common/code_mapping.py @@ -40,7 +40,7 @@ class IndexErrorMessage(ExceptionsMessage): CheckBitmapIndex = "bitmap index are only supported on bool, int, string" CheckBitmapOnPK = "create bitmap index on primary key not supported" CheckBitmapCardinality = "failed to check bitmap cardinality limit, should be larger than 0 and smaller than 1000" - NotConfigable = "{0} is not a configable index proptery" + NotConfigable = "{0} is not a configable index property" InvalidOffsetCache = "invalid offset cache index params" OneIndexPerField = "at most one distinct index is allowed per field" AlterOnLoadedCollection = "can't alter index on loaded collection, please release the collection first" diff --git a/tests/python_client/milvus_client/test_milvus_client_alter.py b/tests/python_client/milvus_client/test_milvus_client_alter.py index 7af1eb1559b5d..7e883e0afe4de 100644 --- a/tests/python_client/milvus_client/test_milvus_client_alter.py +++ b/tests/python_client/milvus_client/test_milvus_client_alter.py @@ -85,7 +85,7 @@ def test_milvus_client_alter_index_unsupported_properties(self): for p in properties.items(): if p[0] not in ["mmap.enabled"]: log.debug(f"try to alter index property: {p[0]}") - error = {ct.err_code: 1, ct.err_msg: f"{p[0]} is not a configable index proptery"} + error = {ct.err_code: 1, ct.err_msg: f"{p[0]} is not a configable index property"} new_value = p[1] + 1 if isinstance(p[1], numbers.Number) else "new_value" self.alter_index_properties(client, collection_name, vector_field_name, properties={p[0]: new_value}, diff --git a/tests/python_client/testcases/test_index.py b/tests/python_client/testcases/test_index.py index 66afe37830e8f..3502f6de1c587 100644 --- a/tests/python_client/testcases/test_index.py +++ b/tests/python_client/testcases/test_index.py @@ -1428,7 +1428,7 @@ def test_alter_index_invalid(self): collection_w.alter_index(ct.default_index_name, {"error_param_key": 123}, check_task=CheckTasks.err_res, check_items={ct.err_code: 1100, - ct.err_msg: "error_param_key is not a configable index proptery:"}) + ct.err_msg: "error_param_key is not a configable index property:"}) collection_w.alter_index(ct.default_index_name, ["error_param_type"], check_task=CheckTasks.err_res, check_items={ct.err_code: 1, From 4355b485e54f254c30ca1fe67c1262fca6d6c649 Mon Sep 17 00:00:00 2001 From: Ted Xu Date: Fri, 10 Jan 2025 13:23:00 +0800 Subject: [PATCH 11/34] enhance: remove compaction parallelism control (#39081) See #39080 --------- Signed-off-by: Ted Xu --- configs/milvus.yaml | 2 +- internal/datacoord/compaction.go | 184 +++++++++++-------------- internal/datacoord/compaction_test.go | 103 ++++++++------ pkg/util/paramtable/component_param.go | 3 +- 4 files changed, 148 insertions(+), 144 deletions(-) diff --git a/configs/milvus.yaml b/configs/milvus.yaml index 3f574165f1b64..305db189ef1be 100644 --- a/configs/milvus.yaml +++ b/configs/milvus.yaml @@ -567,7 +567,7 @@ dataCoord: taskPrioritizer: default taskQueueCapacity: 100000 # compaction task queue size rpcTimeout: 10 - maxParallelTaskNum: 10 + maxParallelTaskNum: -1 # Deprecated, see datanode.slot.slotCap dropTolerance: 86400 # Compaction task will be cleaned after finish longer than this time(in seconds) gcInterval: 1800 # The time interval in seconds for compaction gc mix: diff --git a/internal/datacoord/compaction.go b/internal/datacoord/compaction.go index bea8a6a29d224..a49996c6f3ea8 100644 --- a/internal/datacoord/compaction.go +++ b/internal/datacoord/compaction.go @@ -78,6 +78,72 @@ type compactionInfo struct { mergeInfos map[int64]*milvuspb.CompactionMergeInfo } +type NodeAssigner interface { + assign(t CompactionTask) bool +} + +type SlotBasedNodeAssigner struct { + cluster Cluster + + slots map[int64]int64 +} + +var _ NodeAssigner = (*SlotBasedNodeAssigner)(nil) + +func newSlotBasedNodeAssigner(cluster Cluster) *SlotBasedNodeAssigner { + return &SlotBasedNodeAssigner{ + cluster: cluster, + } +} + +func (sna *SlotBasedNodeAssigner) assign(t CompactionTask) bool { + if sna.slots == nil { + sna.slots = sna.cluster.QuerySlots() + } + + logger := log.With( + zap.Int64("planID", t.GetTaskProto().GetPlanID()), + zap.String("type", t.GetTaskProto().GetType().String()), + zap.String("vchannel", t.GetTaskProto().GetChannel())) + + nodeID, useSlot := sna.pickAnyNode(t) + if nodeID == NullNodeID { + logger.RatedWarn(10, "cannot find datanode for compaction task", + zap.Int64("required", t.GetSlotUsage()), zap.Any("available", sna.slots)) + return false + } + err := t.SetNodeID(nodeID) + if err != nil { + logger.Warn("assignNodeID failed", zap.Error(err)) + return false + } + // update the input nodeSlots + sna.slots[nodeID] = sna.slots[nodeID] - useSlot + logger.Debug("assignNodeID success", zap.Any("nodeID", nodeID)) + return true +} + +func (sna *SlotBasedNodeAssigner) pickAnyNode(task CompactionTask) (nodeID int64, useSlot int64) { + nodeID = NullNodeID + var maxSlots int64 = -1 + + useSlot = task.GetSlotUsage() + if useSlot <= 0 { + log.Warn("task slot should not be 0", + zap.Int64("planID", task.GetTaskProto().GetPlanID()), + zap.String("type", task.GetTaskProto().GetType().String())) + return NullNodeID, useSlot + } + + for id, slots := range sna.slots { + if slots >= useSlot && slots > maxSlots { + nodeID = id + maxSlots = slots + } + } + return nodeID, useSlot +} + type compactionPlanHandler struct { queueTasks CompactionQueue @@ -202,15 +268,21 @@ func newCompactionPlanHandler(cluster Cluster, sessions session.DataNodeManager, } } -func (c *compactionPlanHandler) schedule() []CompactionTask { +func (c *compactionPlanHandler) checkSchedule() { + assigner := newSlotBasedNodeAssigner(c.cluster) + err := c.checkCompaction(assigner) + if err != nil { + log.Info("fail to update compaction", zap.Error(err)) + } + c.cleanFailedTasks() + c.schedule(assigner) +} + +func (c *compactionPlanHandler) schedule(assigner NodeAssigner) []CompactionTask { selected := make([]CompactionTask, 0) if c.queueTasks.Len() == 0 { return selected } - var ( - parallelism = Params.DataCoordCfg.CompactionMaxParallelTasks.GetAsInt() - slots map[int64]int64 - ) l0ChannelExcludes := typeutil.NewSet[string]() mixChannelExcludes := typeutil.NewSet[string]() @@ -219,11 +291,6 @@ func (c *compactionPlanHandler) schedule() []CompactionTask { clusterLabelExcludes := typeutil.NewSet[string]() c.executingGuard.RLock() - if len(c.executingTasks) >= parallelism { - c.executingGuard.RUnlock() - return selected - } - for _, t := range c.executingTasks { switch t.GetTaskProto().GetType() { case datapb.CompactionType_Level0DeleteCompaction: @@ -253,8 +320,7 @@ func (c *compactionPlanHandler) schedule() []CompactionTask { // The schedule loop will stop if either: // 1. no more task to schedule (the task queue is empty) - // 2. the parallelism of running tasks is reached - // 3. no avaiable slots + // 2. no avaiable slots for { t, err := c.queueTasks.Dequeue() if err != nil { @@ -291,24 +357,15 @@ func (c *compactionPlanHandler) schedule() []CompactionTask { } if t.NeedReAssignNodeID() { - if slots == nil { - slots = c.cluster.QuerySlots() - } - id := assignNodeID(slots, t) - if id == NullNodeID { - log.RatedWarn(10, "not enough slots for compaction task", zap.Int64("planID", t.GetTaskProto().GetPlanID())) + if ok := assigner.assign(t); !ok { selected = selected[:len(selected)-1] excluded = append(excluded, t) - break // 3. no avaiable slots + break // 2. no avaiable slots } } c.executingGuard.Lock() c.executingTasks[t.GetTaskProto().GetPlanID()] = t - if len(c.executingTasks) >= parallelism { - c.executingGuard.Unlock() - break // 2. the parallelism of running tasks is reached - } c.executingGuard.Unlock() metrics.DataCoordCompactionTaskNum.WithLabelValues(fmt.Sprintf("%d", NullNodeID), t.GetTaskProto().GetType().String(), metrics.Pending).Dec() metrics.DataCoordCompactionTaskNum.WithLabelValues(fmt.Sprintf("%d", t.GetTaskProto().GetNodeID()), t.GetTaskProto().GetType().String(), metrics.Executing).Inc() @@ -318,9 +375,8 @@ func (c *compactionPlanHandler) schedule() []CompactionTask { func (c *compactionPlanHandler) start() { c.loadMeta() - c.stopWg.Add(3) + c.stopWg.Add(2) go c.loopSchedule() - go c.loopCheck() go c.loopClean() } @@ -397,29 +453,7 @@ func (c *compactionPlanHandler) loopSchedule() { return case <-scheduleTicker.C: - c.schedule() - } - } -} - -func (c *compactionPlanHandler) loopCheck() { - interval := Params.DataCoordCfg.CompactionCheckIntervalInSeconds.GetAsDuration(time.Second) - log.Info("compactionPlanHandler start loop check", zap.Any("check result interval", interval)) - defer c.stopWg.Done() - checkResultTicker := time.NewTicker(interval) - defer checkResultTicker.Stop() - for { - select { - case <-c.stopCh: - log.Info("compactionPlanHandler quit loop check") - return - - case <-checkResultTicker.C: - err := c.checkCompaction() - if err != nil { - log.Info("fail to update compaction", zap.Error(err)) - } - c.cleanFailedTasks() + c.checkSchedule() } } } @@ -647,50 +681,20 @@ func (c *compactionPlanHandler) createCompactTask(t *datapb.CompactionTask) (Com return task, nil } -func assignNodeID(slots map[int64]int64, t CompactionTask) int64 { - if len(slots) == 0 { - return NullNodeID - } - - log := log.Ctx(context.TODO()) - nodeID, useSlot := pickAnyNode(slots, t) - if nodeID == NullNodeID { - log.Info("compactionHandler cannot find datanode for compaction task", - zap.Int64("planID", t.GetTaskProto().GetPlanID()), zap.String("type", t.GetTaskProto().GetType().String()), zap.String("vchannel", t.GetTaskProto().GetChannel())) - return NullNodeID - } - err := t.SetNodeID(nodeID) - if err != nil { - log.Info("compactionHandler assignNodeID failed", - zap.Int64("planID", t.GetTaskProto().GetPlanID()), zap.String("vchannel", t.GetTaskProto().GetChannel()), zap.Error(err)) - return NullNodeID - } - // update the input nodeSlots - slots[nodeID] = slots[nodeID] - useSlot - log.Info("compactionHandler assignNodeID success", - zap.Int64("planID", t.GetTaskProto().GetPlanID()), zap.String("vchannel", t.GetTaskProto().GetChannel()), zap.Any("nodeID", nodeID)) - return nodeID -} - // checkCompaction retrieves executing tasks and calls each task's Process() method // to evaluate its state and progress through the state machine. // Completed tasks are removed from executingTasks. // Tasks that fail or timeout are moved from executingTasks to cleaningTasks, // where task-specific clean logic is performed asynchronously. -func (c *compactionPlanHandler) checkCompaction() error { +func (c *compactionPlanHandler) checkCompaction(assigner NodeAssigner) error { // Get executing executingTasks before GetCompactionState from DataNode to prevent false failure, // for DC might add new task while GetCompactionState. // Assign node id if needed - var slots map[int64]int64 c.executingGuard.RLock() for _, t := range c.executingTasks { if t.NeedReAssignNodeID() { - if slots == nil { - slots = c.cluster.QuerySlots() - } - id := assignNodeID(slots, t) - if id == NullNodeID { + if ok := assigner.assign(t); !ok { break } metrics.DataCoordCompactionTaskNum.WithLabelValues(fmt.Sprintf("%d", NullNodeID), t.GetTaskProto().GetType().String(), metrics.Executing).Dec() @@ -756,26 +760,6 @@ func (c *compactionPlanHandler) cleanFailedTasks() { c.cleaningGuard.Unlock() } -func pickAnyNode(nodeSlots map[int64]int64, task CompactionTask) (nodeID int64, useSlot int64) { - nodeID = NullNodeID - var maxSlots int64 = -1 - - useSlot = task.GetSlotUsage() - if useSlot <= 0 { - log.Ctx(context.TODO()).Warn("task slot should not be 0", zap.Int64("planID", task.GetTaskProto().GetPlanID()), zap.String("type", task.GetTaskProto().GetType().String())) - return NullNodeID, useSlot - } - - for id, slots := range nodeSlots { - if slots >= useSlot && slots > maxSlots { - nodeID = id - maxSlots = slots - } - } - - return nodeID, useSlot -} - // isFull return true if the task pool is full func (c *compactionPlanHandler) isFull() bool { return c.queueTasks.Len() >= c.queueTasks.capacity diff --git a/internal/datacoord/compaction_test.go b/internal/datacoord/compaction_test.go index 31800c712162a..95974b06bc7f1 100644 --- a/internal/datacoord/compaction_test.go +++ b/internal/datacoord/compaction_test.go @@ -67,7 +67,9 @@ func (s *CompactionPlanHandlerSuite) SetupTest() { func (s *CompactionPlanHandlerSuite) TestScheduleEmpty() { s.SetupTest() - s.handler.schedule() + + assigner := newSlotBasedNodeAssigner(s.cluster) + s.handler.schedule(assigner) s.Empty(s.handler.executingTasks) } @@ -228,7 +230,8 @@ func (s *CompactionPlanHandlerSuite) TestScheduleNodeWith1ParallelTask() { s.handler.submitTask(t) } - gotTasks := s.handler.schedule() + assigner := newSlotBasedNodeAssigner(s.cluster) + gotTasks := s.handler.schedule(assigner) s.Equal(test.expectedOut, lo.Map(gotTasks, func(t CompactionTask, _ int) int64 { return t.GetTaskProto().GetPlanID() })) @@ -301,8 +304,8 @@ func (s *CompactionPlanHandlerSuite) TestScheduleWithSlotLimit() { t.SetPlan(test.plans[i]) s.handler.submitTask(t) } - - gotTasks := s.handler.schedule() + assigner := newSlotBasedNodeAssigner(s.cluster) + gotTasks := s.handler.schedule(assigner) s.Equal(test.expectedOut, lo.Map(gotTasks, func(t CompactionTask, _ int) int64 { return t.GetTaskProto().GetPlanID() })) @@ -437,8 +440,8 @@ func (s *CompactionPlanHandlerSuite) TestScheduleNodeWithL0Executing() { for _, t := range test.tasks { s.handler.submitTask(t) } - - gotTasks := s.handler.schedule() + assigner := newSlotBasedNodeAssigner(s.cluster) + gotTasks := s.handler.schedule(assigner) s.Equal(test.expectedOut, lo.Map(gotTasks, func(t CompactionTask, _ int) int64 { return t.GetTaskProto().GetPlanID() })) @@ -448,39 +451,40 @@ func (s *CompactionPlanHandlerSuite) TestScheduleNodeWithL0Executing() { func (s *CompactionPlanHandlerSuite) TestPickAnyNode() { s.SetupTest() - nodeSlots := map[int64]int64{ + + assigner := newSlotBasedNodeAssigner(s.cluster) + assigner.slots = map[int64]int64{ 100: 16, 101: 23, } task1 := newMixCompactionTask(&datapb.CompactionTask{ Type: datapb.CompactionType_MixCompaction, - }, nil, nil, nil) + }, nil, s.mockMeta, nil) task1.slotUsage = paramtable.Get().DataCoordCfg.MixCompactionSlotUsage.GetAsInt64() - node, useSlot := pickAnyNode(nodeSlots, task1) - s.Equal(int64(101), node) - nodeSlots[node] = nodeSlots[node] - useSlot + ok := assigner.assign(task1) + s.Equal(true, ok) + s.Equal(int64(101), task1.GetTaskProto().GetNodeID()) task2 := newMixCompactionTask(&datapb.CompactionTask{ Type: datapb.CompactionType_MixCompaction, - }, nil, nil, nil) + }, nil, s.mockMeta, nil) task2.slotUsage = paramtable.Get().DataCoordCfg.MixCompactionSlotUsage.GetAsInt64() - - node, useSlot = pickAnyNode(nodeSlots, task2) - s.Equal(int64(100), node) - nodeSlots[node] = nodeSlots[node] - useSlot + ok = assigner.assign(task2) + s.Equal(true, ok) + s.Equal(int64(100), task2.GetTaskProto().GetNodeID()) task3 := newMixCompactionTask(&datapb.CompactionTask{ Type: datapb.CompactionType_MixCompaction, - }, nil, nil, nil) + }, nil, s.mockMeta, nil) task3.slotUsage = paramtable.Get().DataCoordCfg.MixCompactionSlotUsage.GetAsInt64() - node, useSlot = pickAnyNode(nodeSlots, task3) - s.Equal(int64(101), node) - nodeSlots[node] = nodeSlots[node] - useSlot + ok = assigner.assign(task3) + s.Equal(true, ok) + s.Equal(int64(101), task3.GetTaskProto().GetNodeID()) - node, useSlot = pickAnyNode(map[int64]int64{}, &mixCompactionTask{}) - s.Equal(int64(NullNodeID), node) + ok = assigner.assign(&mixCompactionTask{}) + s.Equal(false, ok) } func (s *CompactionPlanHandlerSuite) TestPickAnyNodeSlotUsageShouldNotBeZero() { @@ -491,11 +495,12 @@ func (s *CompactionPlanHandlerSuite) TestPickAnyNodeSlotUsageShouldNotBeZero() { } task1 := newMixCompactionTask(&datapb.CompactionTask{ Type: datapb.CompactionType_MixCompaction, - }, nil, nil, nil) + }, nil, s.mockMeta, nil) task1.slotUsage = 0 - nodeID, useSlot := pickAnyNode(nodeSlots, task1) - s.Equal(int64(NullNodeID), nodeID) - s.Equal(int64(0), useSlot) + assigner := newSlotBasedNodeAssigner(s.cluster) + assigner.slots = nodeSlots + ok := assigner.assign(task1) + s.Equal(false, ok) } func (s *CompactionPlanHandlerSuite) TestPickAnyNodeForClusteringTask() { @@ -509,23 +514,26 @@ func (s *CompactionPlanHandlerSuite) TestPickAnyNodeForClusteringTask() { task1 := newClusteringCompactionTask(&datapb.CompactionTask{ Type: datapb.CompactionType_ClusteringCompaction, - }, nil, nil, nil, nil, nil) + }, nil, s.mockMeta, nil, nil, nil) task1.slotUsage = paramtable.Get().DataCoordCfg.ClusteringCompactionSlotUsage.GetAsInt64() task2 := newClusteringCompactionTask(&datapb.CompactionTask{ Type: datapb.CompactionType_ClusteringCompaction, - }, nil, nil, nil, nil, nil) + }, nil, s.mockMeta, nil, nil, nil) task2.slotUsage = paramtable.Get().DataCoordCfg.ClusteringCompactionSlotUsage.GetAsInt64() executingTasks[1] = task1 executingTasks[2] = task2 s.handler.executingTasks = executingTasks - node, useSlot := pickAnyNode(nodeSlots, task1) - s.Equal(int64(101), node) - nodeSlots[node] = nodeSlots[node] - useSlot - node, useSlot = pickAnyNode(nodeSlots, task2) - s.Equal(int64(NullNodeID), node) + assigner := newSlotBasedNodeAssigner(s.cluster) + assigner.slots = nodeSlots + ok := assigner.assign(task1) + s.Equal(true, ok) + s.Equal(int64(101), task1.GetTaskProto().GetNodeID()) + + ok = assigner.assign(task2) + s.Equal(false, ok) } func (s *CompactionPlanHandlerSuite) TestRemoveTasksByChannel() { @@ -623,7 +631,8 @@ func (s *CompactionPlanHandlerSuite) TestGetCompactionTask() { s.handler.submitTask(t) } - s.handler.schedule() + assigner := newSlotBasedNodeAssigner(s.cluster) + s.handler.schedule(assigner) info := s.handler.getCompactionInfo(context.TODO(), 1) s.Equal(1, info.completedCnt) @@ -804,9 +813,10 @@ func (s *CompactionPlanHandlerSuite) TestCheckCompaction() { s.handler.submitTask(t) } - s.handler.schedule() + assigner := newSlotBasedNodeAssigner(s.cluster) + s.handler.schedule(assigner) // time.Sleep(2 * time.Second) - s.handler.checkCompaction() + s.handler.checkCompaction(assigner) t := s.handler.getCompactionTask(1) s.NotNil(t) @@ -938,8 +948,10 @@ func (s *CompactionPlanHandlerSuite) TestProcessCompleteCompaction() { s.mockSessMgr.EXPECT().DropCompactionPlan(mock.Anything, mock.Anything).Return(nil) s.handler.submitTask(task) - s.handler.schedule() - err := s.handler.checkCompaction() + + assigner := newSlotBasedNodeAssigner(s.cluster) + s.handler.schedule(assigner) + err := s.handler.checkCompaction(assigner) s.NoError(err) } @@ -982,7 +994,9 @@ func (s *CompactionPlanHandlerSuite) TestCleanCompaction() { s.handler.executingTasks[1] = task s.Equal(1, len(s.handler.executingTasks)) - err := s.handler.checkCompaction() + + assigner := newSlotBasedNodeAssigner(s.cluster) + err := s.handler.checkCompaction(assigner) s.NoError(err) s.Equal(0, len(s.handler.executingTasks)) s.Equal(1, len(s.handler.cleaningTasks)) @@ -1013,8 +1027,9 @@ func (s *CompactionPlanHandlerSuite) TestCleanClusteringCompaction() { s.mockSessMgr.EXPECT().DropCompactionPlan(mock.Anything, mock.Anything).Return(nil) s.handler.executingTasks[1] = task + assigner := newSlotBasedNodeAssigner(s.cluster) s.Equal(1, len(s.handler.executingTasks)) - s.handler.checkCompaction() + s.handler.checkCompaction(assigner) s.Equal(0, len(s.handler.executingTasks)) s.Equal(1, len(s.handler.cleaningTasks)) s.handler.cleanFailedTasks() @@ -1061,7 +1076,9 @@ func (s *CompactionPlanHandlerSuite) TestCleanClusteringCompactionCommitFail() { s.mockSessMgr.EXPECT().DropCompactionPlan(mock.Anything, mock.Anything).Return(nil) s.handler.executingTasks[1] = task s.Equal(1, len(s.handler.executingTasks)) - s.handler.checkCompaction() + + assigner := newSlotBasedNodeAssigner(s.cluster) + s.handler.checkCompaction(assigner) s.Equal(0, len(task.GetTaskProto().GetResultSegments())) s.Equal(datapb.CompactionTaskState_failed, task.GetTaskProto().GetState()) @@ -1104,8 +1121,10 @@ func (s *CompactionPlanHandlerSuite) TestKeepClean() { s.mockSessMgr.EXPECT().DropCompactionPlan(mock.Anything, mock.Anything).Return(nil) s.handler.executingTasks[1] = task + + assigner := newSlotBasedNodeAssigner(s.cluster) s.Equal(1, len(s.handler.executingTasks)) - s.handler.checkCompaction() + s.handler.checkCompaction(assigner) s.Equal(0, len(s.handler.executingTasks)) s.Equal(1, len(s.handler.cleaningTasks)) s.handler.cleanFailedTasks() diff --git a/pkg/util/paramtable/component_param.go b/pkg/util/paramtable/component_param.go index a01178b37ece7..37b6a2fdf7380 100644 --- a/pkg/util/paramtable/component_param.go +++ b/pkg/util/paramtable/component_param.go @@ -3630,7 +3630,8 @@ mix is prioritized by level: mix compactions first, then L0 compactions, then cl p.CompactionMaxParallelTasks = ParamItem{ Key: "dataCoord.compaction.maxParallelTaskNum", Version: "2.2.12", - DefaultValue: "10", + DefaultValue: "-1", + Doc: "Deprecated, see datanode.slot.slotCap", Export: true, } p.CompactionMaxParallelTasks.Init(base.mgr) From b8fca4f5c12bbbf27f6dd43af69af766b840b9e1 Mon Sep 17 00:00:00 2001 From: XuanYang-cn Date: Fri, 10 Jan 2025 14:07:05 +0800 Subject: [PATCH 12/34] fix: Clustering compaction ignoring deltalogs (#39132) See also: #39131 Signed-off-by: yangxuan --- .../compaction/clustering_compactor.go | 3 ++- .../compaction/clustering_compactor_test.go | 18 ++++++++++++++++++ 2 files changed, 20 insertions(+), 1 deletion(-) diff --git a/internal/datanode/compaction/clustering_compactor.go b/internal/datanode/compaction/clustering_compactor.go index 94a6004028232..62ace26650202 100644 --- a/internal/datanode/compaction/clustering_compactor.go +++ b/internal/datanode/compaction/clustering_compactor.go @@ -473,7 +473,8 @@ func (t *clusteringCompactionTask) mapping(ctx context.Context, for _, segment := range inputSegments { segmentClone := &datapb.CompactionSegmentBinlogs{ SegmentID: segment.SegmentID, - // only FieldBinlogs needed + // only FieldBinlogs and deltalogs needed + Deltalogs: segment.Deltalogs, FieldBinlogs: segment.FieldBinlogs, } future := t.mappingPool.Submit(func() (any, error) { diff --git a/internal/datanode/compaction/clustering_compactor_test.go b/internal/datanode/compaction/clustering_compactor_test.go index 1833f878711b7..9a79db3208ddc 100644 --- a/internal/datanode/compaction/clustering_compactor_test.go +++ b/internal/datanode/compaction/clustering_compactor_test.go @@ -170,6 +170,15 @@ func (s *ClusteringCompactionTaskSuite) TestCompactionInit() { } func (s *ClusteringCompactionTaskSuite) TestScalarCompactionNormal() { + dblobs, err := getInt64DeltaBlobs( + 1, + []int64{100}, + []uint64{tsoutil.ComposeTSByTime(getMilvusBirthday().Add(time.Second), 0)}, + ) + s.Require().NoError(err) + s.mockBinlogIO.EXPECT().Download(mock.Anything, []string{"1"}). + Return([][]byte{dblobs.GetValue()}, nil).Once() + schema := genCollectionSchema() var segmentID int64 = 1001 segWriter, err := NewSegmentWriter(schema, 1000, compactionBatchSize, segmentID, PartitionID, CollectionID, []int64{}) @@ -193,6 +202,9 @@ func (s *ClusteringCompactionTaskSuite) TestScalarCompactionNormal() { { SegmentID: segmentID, FieldBinlogs: lo.Values(fBinlogs), + Deltalogs: []*datapb.FieldBinlog{ + {Binlogs: []*datapb.Binlog{{LogID: 1, LogPath: "1"}}}, + }, }, } @@ -236,6 +248,12 @@ func (s *ClusteringCompactionTaskSuite) TestScalarCompactionNormal() { s.Equal(2, totalBinlogNum/len(schema.GetFields())) s.Equal(1, statsBinlogNum) s.Equal(totalRowNum, statsRowNum) + + s.EqualValues(10239, + lo.SumBy(compactionResult.GetSegments(), func(seg *datapb.CompactionSegment) int64 { + return seg.GetNumOfRows() + }), + ) } func (s *ClusteringCompactionTaskSuite) TestScalarCompactionNormalByMemoryLimit() { From 826b726c86eebaac8d8a1eb8182c998f9b804d62 Mon Sep 17 00:00:00 2001 From: wei liu Date: Fri, 10 Jan 2025 14:12:57 +0800 Subject: [PATCH 13/34] fix: Prevent leader checker from generating excessive duplicate leader tasks (#39000) issue: #39001 Background: Segment Load Version: Each segment load request assigns a timestamp as its version. When multiple copies of a segment are loaded on different QueryNodes, the leader checker uses this version to identify the latest copy and updates the routing table in the leader view to point to it. Delegator Router Version: When a delegator builds a route to a QueryNode that has loaded a segment, it also records the segment's version. Router Table Update Logic: If the leader checker detects that the version of a segment in the routing table does not match the version in the worker, it updates the routing table to point to the QueryNode with the latest version. Additionally, it updates the segment's load version in the QueryNode during this process. Issue: When a channel is undergoing load balancing, the leader checker may sync the routing table to a new delegator. This sync operation modifies the segment's load version, which invalidates the routing in the old delegator. Subsequently, the leader checker updates the routing table in the old delegator, breaking the routing in the new delegator. This cycle continues, causing repeated updates and inconsistencies. Fix: This PR introduces two changes to address the issue: 1. Use NodeID to verify whether the delegator's routing table needs an update, avoiding unnecessary modifications. 2. Ensure compatibility by using the latest segment's load version as the version recorded in the routing table. These changes resolve the cyclic updates and prevent the leader checker from generating excessive duplicate tasks, ensuring routing stability across delegators during load balancing. Signed-off-by: Wei Liu --- .../querycoordv2/checkers/leader_checker.go | 9 ++++++--- .../checkers/leader_checker_test.go | 20 ++++++++++++------- 2 files changed, 19 insertions(+), 10 deletions(-) diff --git a/internal/querycoordv2/checkers/leader_checker.go b/internal/querycoordv2/checkers/leader_checker.go index 756e842d8c250..110d58068f52d 100644 --- a/internal/querycoordv2/checkers/leader_checker.go +++ b/internal/querycoordv2/checkers/leader_checker.go @@ -171,13 +171,16 @@ func (c *LeaderChecker) findNeedLoadedSegments(ctx context.Context, replica *met continue } - // when segment's version in leader view doesn't match segment's version in dist - // which means leader view store wrong segment location in leader view, then we should update segment location and segment's version + // The routing table on the delegator points to the nodes where segments are loaded. There are two scenarios that require updating the routing table on the delegator: + // 1. Missing Segment Routing - The routing table lacks the route for a specific segment. + // 2. Outdated Segment Routing - A segment has multiple copies loaded, but the routing table points to a node that does not host the most recently loaded copy. + // This ensures the routing table remains accurate and up-to-date, reflecting the latest segment distribution. version, ok := leaderView.Segments[s.GetID()] - if !ok || version.GetVersion() != s.Version { + if !ok || version.GetNodeID() != s.Node { log.RatedDebug(10, "leader checker append a segment to set", zap.Int64("segmentID", s.GetID()), zap.Int64("nodeID", s.Node)) + action := task.NewLeaderAction(leaderView.ID, s.Node, task.ActionTypeGrow, s.GetInsertChannel(), s.GetID(), time.Now().UnixNano()) t := task.NewLeaderSegmentTask( ctx, diff --git a/internal/querycoordv2/checkers/leader_checker_test.go b/internal/querycoordv2/checkers/leader_checker_test.go index ff0daaa76e66e..1d44938f7690b 100644 --- a/internal/querycoordv2/checkers/leader_checker_test.go +++ b/internal/querycoordv2/checkers/leader_checker_test.go @@ -138,22 +138,28 @@ func (suite *LeaderCheckerTestSuite) TestSyncLoadedSegments() { suite.Equal(tasks[0].Actions()[0].(*task.LeaderAction).SegmentID(), int64(1)) suite.Equal(tasks[0].Priority(), task.TaskPriorityLow) - // test segment's version in leader view doesn't match segment's version in dist - observer.dist.SegmentDistManager.Update(1, utils.CreateTestSegment(1, 1, 1, 2, 1, "test-insert-channel")) - view = utils.CreateTestLeaderView(2, 1, "test-insert-channel", map[int64]int64{}, map[int64]*meta.Segment{}) + // Verify that the segment routing table in the leader view does not point to the most recent segment replica. + // the leader view points to the segment on querynode-2, with version 1 + // the distribution shows that the segment is on querynode-1, with latest version 2 + node1, node2 := int64(1), int64(2) + version1, version2 := int64(1), int64(2) + observer.dist.SegmentDistManager.Update(node1) + observer.dist.SegmentDistManager.Update(node2, utils.CreateTestSegment(1, 1, 1, node2, version2, "test-insert-channel")) + view = utils.CreateTestLeaderView(node2, 1, "test-insert-channel", map[int64]int64{}, map[int64]*meta.Segment{}) view.TargetVersion = observer.target.GetCollectionTargetVersion(ctx, 1, meta.CurrentTarget) view.Segments[1] = &querypb.SegmentDist{ - NodeID: 0, - Version: time.Now().UnixMilli() - 1, + NodeID: node1, + Version: version1, } - observer.dist.LeaderViewManager.Update(2, view) + observer.dist.LeaderViewManager.Update(node2, view) tasks = suite.checker.Check(context.TODO()) suite.Len(tasks, 1) suite.Equal(tasks[0].Source(), utils.LeaderChecker) suite.Len(tasks[0].Actions(), 1) suite.Equal(tasks[0].Actions()[0].Type(), task.ActionTypeGrow) - suite.Equal(tasks[0].Actions()[0].Node(), int64(1)) + suite.Equal(tasks[0].Actions()[0].Node(), node2) + suite.Equal(tasks[0].Actions()[0].(*task.LeaderAction).GetLeaderID(), node2) suite.Equal(tasks[0].Actions()[0].(*task.LeaderAction).SegmentID(), int64(1)) suite.Equal(tasks[0].Priority(), task.TaskPriorityLow) From 357eaf0d7124ae67f53038d639125fe6abe3777a Mon Sep 17 00:00:00 2001 From: SimFG Date: Fri, 10 Jan 2025 18:16:58 +0800 Subject: [PATCH 14/34] fix: use the object heap to keep the min ddl ts order (#39118) issue: #39002 Signed-off-by: SimFG --- internal/rootcoord/scheduler.go | 24 ++++++- internal/rootcoord/scheduler_test.go | 104 +++++++++++++++++++++++++++ internal/rootcoord/task.go | 25 ++++--- pkg/util/typeutil/heap.go | 64 +++++++++++++++++ pkg/util/typeutil/heap_test.go | 50 +++++++++++++ 5 files changed, 256 insertions(+), 11 deletions(-) diff --git a/internal/rootcoord/scheduler.go b/internal/rootcoord/scheduler.go index 1f5e3d0e478ec..9ce202a6eb3c7 100644 --- a/internal/rootcoord/scheduler.go +++ b/internal/rootcoord/scheduler.go @@ -28,6 +28,7 @@ import ( "github.com/milvus-io/milvus/internal/tso" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/lock" + "github.com/milvus-io/milvus/pkg/util/typeutil" ) type IScheduler interface { @@ -46,6 +47,7 @@ type scheduler struct { tsoAllocator tso.Allocator taskChan chan task + taskHeap typeutil.Heap[task] lock sync.Mutex @@ -56,16 +58,22 @@ type scheduler struct { lockMapping map[LockLevel]*lock.KeyLock[string] } +func GetTaskHeapOrder(t task) Timestamp { + return t.GetTs() +} + func newScheduler(ctx context.Context, idAllocator allocator.Interface, tsoAllocator tso.Allocator) *scheduler { ctx1, cancel := context.WithCancel(ctx) // TODO n := 1024 * 10 + taskArr := make([]task, 0) s := &scheduler{ ctx: ctx1, cancel: cancel, idAllocator: idAllocator, tsoAllocator: tsoAllocator, taskChan: make(chan task, n), + taskHeap: typeutil.NewObjectArrayBasedMinimumHeap[task, Timestamp](taskArr, GetTaskHeapOrder), minDdlTs: *atomic.NewUint64(0), clusterLock: lock.NewKeyLock[string](), databaseLock: lock.NewKeyLock[string](), @@ -93,7 +101,7 @@ func (s *scheduler) Stop() { } func (s *scheduler) execute(task task) { - defer s.setMinDdlTs(task.GetTs()) // we should update ts, whatever task succeeds or not. + defer s.setMinDdlTs() // we should update ts, whatever task succeeds or not. task.SetInQueueDuration() if err := task.Prepare(task.GetCtx()); err != nil { task.NotifyDone(err) @@ -153,6 +161,7 @@ func (s *scheduler) setTs(task task) error { return err } task.SetTs(ts) + s.taskHeap.Push(task) return nil } @@ -186,8 +195,14 @@ func (s *scheduler) GetMinDdlTs() Timestamp { return s.minDdlTs.Load() } -func (s *scheduler) setMinDdlTs(ts Timestamp) { - s.minDdlTs.Store(ts) +func (s *scheduler) setMinDdlTs() { + s.lock.Lock() + defer s.lock.Unlock() + + for s.taskHeap.Len() > 0 && s.taskHeap.Peek().IsFinished() { + t := s.taskHeap.Pop() + s.minDdlTs.Store(t.GetTs()) + } } func (s *scheduler) executeTaskWithLock(task task, lockerKey LockerKey) error { @@ -195,9 +210,12 @@ func (s *scheduler) executeTaskWithLock(task task, lockerKey LockerKey) error { if err := s.setID(task); err != nil { return err } + s.lock.Lock() if err := s.setTs(task); err != nil { + s.lock.Unlock() return err } + s.lock.Unlock() s.execute(task) return nil } diff --git a/internal/rootcoord/scheduler_test.go b/internal/rootcoord/scheduler_test.go index 99c0806baf008..a5f782d0dca6d 100644 --- a/internal/rootcoord/scheduler_test.go +++ b/internal/rootcoord/scheduler_test.go @@ -82,6 +82,29 @@ func newMockNormalTask() *mockNormalTask { return task } +type mockLockerKeyTask struct { + baseTask + lockerKey string + rw bool +} + +func (m *mockLockerKeyTask) GetLockerKey() LockerKey { + return NewLockerKeyChain( + NewClusterLockerKey(false), + NewDatabaseLockerKey(m.lockerKey, m.rw), + ) +} + +func newMockLockerKeyTask(lockerKey string, rw bool) *mockLockerKeyTask { + task := &mockLockerKeyTask{ + baseTask: newBaseTask(context.Background(), nil), + lockerKey: lockerKey, + rw: rw, + } + task.SetCtx(context.Background()) + return task +} + func Test_scheduler_Start_Stop(t *testing.T) { idAlloc := newMockIDAllocator() tsoAlloc := newMockTsoAllocator() @@ -247,6 +270,87 @@ func Test_scheduler_updateDdlMinTsLoop(t *testing.T) { assert.Zero(t, s.GetMinDdlTs()) s.Stop() }) + + t.Run("concurrent task schedule", func(t *testing.T) { + idAlloc := newMockIDAllocator() + tsoAlloc := newMockTsoAllocator() + tso := atomic.NewUint64(100) + idAlloc.AllocOneF = func() (UniqueID, error) { + return 100, nil + } + tsoAlloc.GenerateTSOF = func(count uint32) (uint64, error) { + got := tso.Inc() + return got, nil + } + ctx := context.Background() + s := newScheduler(ctx, idAlloc, tsoAlloc) + paramtable.Init() + paramtable.Get().Save(Params.ProxyCfg.TimeTickInterval.Key, "1") + s.Start() + + for i := 0; i < 100; i++ { + if s.GetMinDdlTs() > Timestamp(100) { + break + } + assert.True(t, i < 100) + time.Sleep(time.Millisecond) + } + + w := &sync.WaitGroup{} + w.Add(5) + // locker key rw true + lockerKey := "hello" + go func() { + defer w.Done() + n := 200 + for i := 0; i < n; i++ { + task := newMockLockerKeyTask(lockerKey, true) + err := s.AddTask(task) + assert.NoError(t, err) + } + }() + + // locker key rw false + go func() { + defer w.Done() + n := 200 + for i := 0; i < n; i++ { + task := newMockLockerKeyTask(lockerKey, false) + err := s.AddTask(task) + assert.NoError(t, err) + } + }() + + go func() { + defer w.Done() + n := 200 + for i := 0; i < n; i++ { + task := newMockLockerKeyTask(lockerKey, false) + err := s.AddTask(task) + assert.NoError(t, err) + } + }() + + go func() { + defer w.Done() + n := 200 + for i := 0; i < n; i++ { + task := newMockNormalTask() + err := s.AddTask(task) + assert.NoError(t, err) + } + }() + + lastMin := s.GetMinDdlTs() + go func() { + defer w.Done() + current := s.GetMinDdlTs() + assert.True(t, current >= lastMin) + lastMin = current + time.Sleep(time.Millisecond * 100) + }() + w.Wait() + }) } type WithLockKeyTask struct { diff --git a/internal/rootcoord/task.go b/internal/rootcoord/task.go index d515979f7a09f..5f9765b6b63bb 100644 --- a/internal/rootcoord/task.go +++ b/internal/rootcoord/task.go @@ -21,6 +21,7 @@ import ( "fmt" "time" + "go.uber.org/atomic" "go.uber.org/zap" "github.com/milvus-io/milvus/pkg/log" @@ -53,16 +54,18 @@ type task interface { Execute(ctx context.Context) error WaitToFinish() error NotifyDone(err error) + IsFinished() bool SetInQueueDuration() GetLockerKey() LockerKey } type baseTask struct { - ctx context.Context - core *Core - done chan error - ts Timestamp - id UniqueID + ctx context.Context + core *Core + done chan error + isFinished *atomic.Bool + ts Timestamp + id UniqueID tr *timerecord.TimeRecorder queueDur time.Duration @@ -70,9 +73,10 @@ type baseTask struct { func newBaseTask(ctx context.Context, core *Core) baseTask { b := baseTask{ - core: core, - done: make(chan error, 1), - tr: timerecord.NewTimeRecorderWithTrace(ctx, "new task"), + core: core, + done: make(chan error, 1), + tr: timerecord.NewTimeRecorderWithTrace(ctx, "new task"), + isFinished: atomic.NewBool(false), } b.SetCtx(ctx) return b @@ -116,12 +120,17 @@ func (b *baseTask) WaitToFinish() error { func (b *baseTask) NotifyDone(err error) { b.done <- err + b.isFinished.Store(true) } func (b *baseTask) SetInQueueDuration() { b.queueDur = b.tr.ElapseSpan() } +func (b *baseTask) IsFinished() bool { + return b.isFinished.Load() +} + func (b *baseTask) GetLockerKey() LockerKey { return nil } diff --git a/pkg/util/typeutil/heap.go b/pkg/util/typeutil/heap.go index 8e0d2abdf909a..94a9ab0840d3f 100644 --- a/pkg/util/typeutil/heap.go +++ b/pkg/util/typeutil/heap.go @@ -69,6 +69,39 @@ func (h *heapArray[E]) Peek() interface{} { return (*h)[0] } +type objectHeapArray[O any, E constraints.Ordered] struct { + objects []O + getOrderFunc func(O) E +} + +func (h *objectHeapArray[O, E]) Len() int { + return len(h.objects) +} + +func (h *objectHeapArray[O, E]) Less(i, j int) bool { + return h.getOrderFunc(h.objects[i]) < h.getOrderFunc(h.objects[j]) +} + +func (h *objectHeapArray[O, E]) Swap(i, j int) { + h.objects[i], h.objects[j] = h.objects[j], h.objects[i] +} + +func (h *objectHeapArray[O, E]) Push(x interface{}) { + h.objects = append(h.objects, x.(O)) +} + +func (h *objectHeapArray[O, E]) Pop() interface{} { + old := h.objects + n := len(old) + x := old[n-1] + h.objects = old[0 : n-1] + return x +} + +func (h *objectHeapArray[O, E]) Peek() interface{} { + return h.objects[0] +} + // reverseOrderedInterface is a heap base interface that reverses the order of the elements. type reverseOrderedInterface[E constraints.Ordered] struct { HeapInterface @@ -107,6 +140,37 @@ func NewArrayBasedMinimumHeap[E constraints.Ordered](initial []E) Heap[E] { } } +func NewObjectArrayBasedMaximumHeap[O any, E constraints.Ordered](initial []O, getOrderFunc func(O) E) Heap[O] { + if initial == nil { + initial = make([]O, 0) + } + ha := &objectHeapArray[O, E]{ + objects: initial, + getOrderFunc: getOrderFunc, + } + reverse := reverseOrderedInterface[E]{ + HeapInterface: ha, + } + heap.Init(reverse) + return &heapImpl[O, reverseOrderedInterface[E]]{ + inner: reverse, + } +} + +func NewObjectArrayBasedMinimumHeap[O any, E constraints.Ordered](initial []O, getOrderFunc func(O) E) Heap[O] { + if initial == nil { + initial = make([]O, 0) + } + ha := &objectHeapArray[O, E]{ + objects: initial, + getOrderFunc: getOrderFunc, + } + heap.Init(ha) + return &heapImpl[O, *objectHeapArray[O, E]]{ + inner: ha, + } +} + // heapImpl is a min-heap of E. type heapImpl[E any, H HeapInterface] struct { inner H diff --git a/pkg/util/typeutil/heap_test.go b/pkg/util/typeutil/heap_test.go index 757bec3b428c9..c4d67741023c6 100644 --- a/pkg/util/typeutil/heap_test.go +++ b/pkg/util/typeutil/heap_test.go @@ -39,3 +39,53 @@ func TestMaximumHeap(t *testing.T) { assert.Equal(t, i, heap.Pop()) } } + +type FooHeapObject struct { + value int +} + +func GetFooHeapObjectOrderFunc(obj *FooHeapObject) int { + return obj.value +} + +func TestMinimumObjectHeap(t *testing.T) { + h := []*FooHeapObject{ + {value: 4}, + {value: 5}, + {value: 2}, + } + heap := NewObjectArrayBasedMinimumHeap(h, GetFooHeapObjectOrderFunc) + assert.Equal(t, 2, heap.Peek().value) + assert.Equal(t, 3, heap.Len()) + heap.Push(&FooHeapObject{value: 3}) + assert.Equal(t, 2, heap.Peek().value) + assert.Equal(t, 4, heap.Len()) + heap.Push(&FooHeapObject{value: 1}) + assert.Equal(t, 1, heap.Peek().value) + assert.Equal(t, 5, heap.Len()) + for i := 1; i <= 5; i++ { + assert.Equal(t, i, heap.Peek().value) + assert.Equal(t, i, heap.Pop().value) + } +} + +func TestMaximumObjectHeap(t *testing.T) { + h := []*FooHeapObject{ + {value: 4}, + {value: 1}, + {value: 2}, + } + heap := NewObjectArrayBasedMaximumHeap(h, GetFooHeapObjectOrderFunc) + assert.Equal(t, 4, heap.Peek().value) + assert.Equal(t, 3, heap.Len()) + heap.Push(&FooHeapObject{value: 3}) + assert.Equal(t, 4, heap.Peek().value) + assert.Equal(t, 4, heap.Len()) + heap.Push(&FooHeapObject{value: 5}) + assert.Equal(t, 5, heap.Peek().value) + assert.Equal(t, 5, heap.Len()) + for i := 5; i >= 1; i-- { + assert.Equal(t, i, heap.Peek().value) + assert.Equal(t, i, heap.Pop().value) + } +} From 86d665a50f5322b52e82874f1c509f2814001d58 Mon Sep 17 00:00:00 2001 From: SimFG Date: Fri, 10 Jan 2025 18:20:59 +0800 Subject: [PATCH 15/34] enhance: keylock object background recycling (#38805) issue: #38587 Signed-off-by: SimFG --- pkg/go.mod | 1 + pkg/go.sum | 4 +++ pkg/util/lock/key_lock.go | 50 +++++++++++++++++++++++++++++++--- pkg/util/lock/key_lock_test.go | 15 ++++++++++ 4 files changed, 66 insertions(+), 4 deletions(-) diff --git a/pkg/go.mod b/pkg/go.mod index dd9f7bbffd09c..67bd679df4679 100644 --- a/pkg/go.mod +++ b/pkg/go.mod @@ -12,6 +12,7 @@ require ( github.com/containerd/cgroups/v3 v3.0.3 github.com/expr-lang/expr v1.15.7 github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 + github.com/jolestar/go-commons-pool/v2 v2.1.2 github.com/json-iterator/go v1.1.12 github.com/klauspost/compress v1.17.7 github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20241211060635-410431d7865b diff --git a/pkg/go.sum b/pkg/go.sum index d339ec77fba97..c1d2b2fc90359 100644 --- a/pkg/go.sum +++ b/pkg/go.sum @@ -190,6 +190,8 @@ github.com/fatih/color v1.10.0/go.mod h1:ELkj/draVOlAH/xkhN6mQ50Qd0MPOk5AAr3maGE github.com/fatih/structs v1.1.0/go.mod h1:9NiDSp5zOcgEDl+j00MP/WkGVPOlPRLejGD8Ga6PJ7M= github.com/form3tech-oss/jwt-go v3.2.3+incompatible h1:7ZaBxOI7TMoYBfyA3cQHErNNyAWIKUMIwqxEtgHOs5c= github.com/form3tech-oss/jwt-go v3.2.3+incompatible/go.mod h1:pbq4aXjuKjdthFRnoDwaVPLA+WlJuPGy+QneDUgJi2k= +github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw= +github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g= github.com/frankban/quicktest v1.2.2/go.mod h1:Qh/WofXFeiAFII1aEBu529AtJo6Zg2VHscnEsbBnJ20= github.com/frankban/quicktest v1.7.2/go.mod h1:jaStnuzAqU1AJdCO0l53JDCJrVDKcS03DbaAcR7Ks/o= github.com/frankban/quicktest v1.10.0/go.mod h1:ui7WezCLWMWxVWr1GETZY3smRy0G4KWq9vcPtJmFl7Y= @@ -402,6 +404,8 @@ github.com/jhump/goprotoc v0.5.0/go.mod h1:VrbvcYrQOrTi3i0Vf+m+oqQWk9l72mjkJCYo7 github.com/jhump/protoreflect v1.11.0/go.mod h1:U7aMIjN0NWq9swDP7xDdoMfRHb35uiuTd3Z9nFXJf5E= github.com/jhump/protoreflect v1.12.0/go.mod h1:JytZfP5d0r8pVNLZvai7U/MCuTWITgrI4tTg7puQFKI= github.com/jmespath/go-jmespath v0.3.0/go.mod h1:9QtRXoHjLGCJ5IBSaohpXITPlowMeeYCZ7fLUTSywik= +github.com/jolestar/go-commons-pool/v2 v2.1.2 h1:E+XGo58F23t7HtZiC/W6jzO2Ux2IccSH/yx4nD+J1CM= +github.com/jolestar/go-commons-pool/v2 v2.1.2/go.mod h1:r4NYccrkS5UqP1YQI1COyTZ9UjPJAAGTUxzcsK1kqhY= github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= github.com/jonboulle/clockwork v0.2.2 h1:UOGuzwb1PwsrDAObMuhUnj0p5ULPj8V/xJ7Kx9qUBdQ= github.com/jonboulle/clockwork v0.2.2/go.mod h1:Pkfl5aHPm1nk2H9h0bjmnJD/BcgbGXUBGnn1kMkgxc8= diff --git a/pkg/util/lock/key_lock.go b/pkg/util/lock/key_lock.go index 97910aed7bd14..f09a0069b884e 100644 --- a/pkg/util/lock/key_lock.go +++ b/pkg/util/lock/key_lock.go @@ -17,13 +17,35 @@ package lock import ( + "context" "sync" + pool "github.com/jolestar/go-commons-pool/v2" "go.uber.org/zap" "github.com/milvus-io/milvus/pkg/log" ) +var ( + ctx = context.Background() + lockPoolFactory = pool.NewPooledObjectFactorySimple(func(ctx2 context.Context) (interface{}, error) { + return newRefLock(), nil + }) + lockerPoolConfig = &pool.ObjectPoolConfig{ + LIFO: pool.DefaultLIFO, + MaxTotal: 64, + MaxIdle: 64, + MinIdle: pool.DefaultMinIdle, + MinEvictableIdleTime: pool.DefaultMinEvictableIdleTime, + SoftMinEvictableIdleTime: pool.DefaultSoftMinEvictableIdleTime, + NumTestsPerEvictionRun: pool.DefaultNumTestsPerEvictionRun, + EvictionPolicyName: pool.DefaultEvictionPolicyName, + EvictionContext: ctx, + BlockWhenExhausted: false, + } + refLockPoolPool = pool.NewObjectPool(ctx, lockPoolFactory, lockerPoolConfig) +) + type RefLock struct { mutex sync.RWMutex refCounter int @@ -33,8 +55,12 @@ func (m *RefLock) ref() { m.refCounter++ } -func (m *RefLock) unref() { - m.refCounter-- +func (m *RefLock) unref() bool { + if m.refCounter > 0 { + m.refCounter-- + return true + } + return false } func newRefLock() *RefLock { @@ -66,7 +92,14 @@ func (k *KeyLock[K]) Lock(key K) { k.keyLocksMutex.Unlock() keyLock.mutex.Lock() } else { - newKLock := newRefLock() + obj, err := refLockPoolPool.BorrowObject(ctx) + if err != nil { + log.Ctx(ctx).Error("BorrowObject failed", zap.Error(err)) + k.keyLocksMutex.Unlock() + return + } + newKLock := obj.(*RefLock) + // newKLock := newRefLock() newKLock.mutex.Lock() k.refLocks[key] = newKLock newKLock.ref() @@ -86,6 +119,7 @@ func (k *KeyLock[K]) Unlock(lockedKey K) { } keyLock.unref() if keyLock.refCounter == 0 { + _ = refLockPoolPool.ReturnObject(ctx, keyLock) delete(k.refLocks, lockedKey) } keyLock.mutex.Unlock() @@ -100,7 +134,14 @@ func (k *KeyLock[K]) RLock(key K) { k.keyLocksMutex.Unlock() keyLock.mutex.RLock() } else { - newKLock := newRefLock() + obj, err := refLockPoolPool.BorrowObject(ctx) + if err != nil { + log.Ctx(ctx).Error("BorrowObject failed", zap.Error(err)) + k.keyLocksMutex.Unlock() + return + } + newKLock := obj.(*RefLock) + // newKLock := newRefLock() newKLock.mutex.RLock() k.refLocks[key] = newKLock newKLock.ref() @@ -120,6 +161,7 @@ func (k *KeyLock[K]) RUnlock(lockedKey K) { } keyLock.unref() if keyLock.refCounter == 0 { + _ = refLockPoolPool.ReturnObject(ctx, keyLock) delete(k.refLocks, lockedKey) } keyLock.mutex.RUnlock() diff --git a/pkg/util/lock/key_lock_test.go b/pkg/util/lock/key_lock_test.go index 46002b9ed4176..755c23a2324ba 100644 --- a/pkg/util/lock/key_lock_test.go +++ b/pkg/util/lock/key_lock_test.go @@ -67,3 +67,18 @@ func TestKeyRLock(t *testing.T) { wg.Wait() assert.Equal(t, keyLock.size(), 0) } + +func TestNewKeyLock(t *testing.T) { + keyLock := NewKeyLock[string]() + keyLock.Lock("a") + keyLock.Lock("b") + + keyLock.Unlock("a") + keyLock.Unlock("b") + + assert.Equal(t, 0, keyLock.size()) + keyLock.keyLocksMutex.Lock() + keyLen := len(keyLock.refLocks) + keyLock.keyLocksMutex.Unlock() + assert.Equal(t, 0, keyLen) +} From a8a65641b5a6b9e45c78e636a28148748f64b693 Mon Sep 17 00:00:00 2001 From: zhuwenxing Date: Sat, 11 Jan 2025 14:21:03 +0800 Subject: [PATCH 16/34] test: update reatful api timeout (#39139) /kind improvement Signed-off-by: zhuwenxing --- tests/restful_client_v2/api/milvus.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/restful_client_v2/api/milvus.py b/tests/restful_client_v2/api/milvus.py index 9cbddf9295ee6..3d3b1c2eae2a3 100644 --- a/tests/restful_client_v2/api/milvus.py +++ b/tests/restful_client_v2/api/milvus.py @@ -10,6 +10,8 @@ from requests.exceptions import ConnectionError import urllib.parse +REQUEST_TIMEOUT = 120 + ENABLE_LOG_SAVE = False @@ -101,7 +103,8 @@ def __init__(self, url=None, api_key=None): self.headers = { 'Content-Type': 'application/json', 'Authorization': f'Bearer {self.api_key}', - 'RequestId': self.uuid + 'RequestId': self.uuid, + "Request-Timeout": REQUEST_TIMEOUT } @classmethod From 032292a4329e1c569f7919fce7e8984d301a3e4a Mon Sep 17 00:00:00 2001 From: Spade A <71589810+SpadeA-Tang@users.noreply.github.com> Date: Sun, 12 Jan 2025 20:24:58 +0800 Subject: [PATCH 17/34] feat: support phrase match query (#38869) The relevant issue: https://github.com/milvus-io/milvus/issues/38930 --------- Signed-off-by: SpadeA-Tang --- internal/core/src/common/EasyAssert.h | 1 + .../core/src/exec/expression/UnaryExpr.cpp | 34 +- internal/core/src/expr/ITypeExpr.h | 26 +- internal/core/src/index/TextMatchIndex.cpp | 21 + internal/core/src/index/TextMatchIndex.h | 3 + internal/core/src/query/PlanProto.cpp | 9 +- .../tantivy-binding/include/tantivy-binding.h | 2 + .../tantivy-binding/src/index_reader_text.rs | 30 +- .../src/index_reader_text_c.rs | 13 + .../core/thirdparty/tantivy/tantivy-wrapper.h | 13 + internal/core/unittest/test_array_expr.cpp | 3 +- internal/core/unittest/test_exec.cpp | 57 +- internal/core/unittest/test_expr.cpp | 72 +- internal/core/unittest/test_retrieve.cpp | 6 +- internal/core/unittest/test_text_match.cpp | 653 +++++++++----- internal/parser/planparserv2/Plan.g4 | 2 + .../parser/planparserv2/generated/Plan.interp | 4 +- .../parser/planparserv2/generated/Plan.tokens | 97 +- .../planparserv2/generated/PlanLexer.interp | 5 +- .../planparserv2/generated/PlanLexer.tokens | 97 +- .../generated/plan_base_visitor.go | 4 + .../planparserv2/generated/plan_lexer.go | 831 +++++++++--------- .../planparserv2/generated/plan_parser.go | 580 +++++++----- .../planparserv2/generated/plan_visitor.go | 3 + .../parser/planparserv2/parser_visitor.go | 35 + .../planparserv2/plan_parser_v2_test.go | 25 + internal/parser/planparserv2/show_visitor.go | 5 + pkg/proto/plan.proto | 2 + pkg/proto/planpb/plan.pb.go | 748 ++++++++-------- 29 files changed, 2001 insertions(+), 1380 deletions(-) diff --git a/internal/core/src/common/EasyAssert.h b/internal/core/src/common/EasyAssert.h index 8258438b28052..93d3cc36d04f1 100644 --- a/internal/core/src/common/EasyAssert.h +++ b/internal/core/src/common/EasyAssert.h @@ -70,6 +70,7 @@ enum ErrorCode { OutOfRange = 2039, GcpNativeError = 2040, TextIndexNotFound = 2041, + InvalidParameter = 2042, KnowhereError = 2099 }; diff --git a/internal/core/src/exec/expression/UnaryExpr.cpp b/internal/core/src/exec/expression/UnaryExpr.cpp index d3453d1f66f69..3983ca759d2da 100644 --- a/internal/core/src/exec/expression/UnaryExpr.cpp +++ b/internal/core/src/exec/expression/UnaryExpr.cpp @@ -796,11 +796,12 @@ PhyUnaryRangeFilterExpr::ExecRangeVisitorImplJson(OffsetVector* input) { template VectorPtr PhyUnaryRangeFilterExpr::ExecRangeVisitorImpl(OffsetVector* input) { - if (expr_->op_type_ == proto::plan::OpType::TextMatch) { + if (expr_->op_type_ == proto::plan::OpType::TextMatch || + expr_->op_type_ == proto::plan::OpType::PhraseMatch) { if (has_offset_input_) { PanicInfo( OpTypeInvalid, - fmt::format("text match does not support iterative filter")); + fmt::format("match query does not support iterative filter")); } return ExecTextMatch(); } @@ -1089,8 +1090,33 @@ VectorPtr PhyUnaryRangeFilterExpr::ExecTextMatch() { using Index = index::TextMatchIndex; auto query = GetValueFromProto(expr_->val_); - auto func = [](Index* index, const std::string& query) -> TargetBitmap { - return index->MatchQuery(query); + int64_t slop = 0; + if (expr_->op_type_ == proto::plan::PhraseMatch) { + // It should be larger than 0 in normal cases. Check it incase of receiving old version proto. + if (expr_->extra_values_.size() > 0) { + slop = GetValueFromProto(expr_->extra_values_[0]); + } + if (slop < 0 || slop > std::numeric_limits::max()) { + throw SegcoreError( + ErrorCode::InvalidParameter, + fmt::format( + "Slop {} is invalid in phrase match query. Should be " + "within [0, UINT32_MAX].", + slop)); + } + } + auto op_type = expr_->op_type_; + auto func = [op_type, slop](Index* index, + const std::string& query) -> TargetBitmap { + if (op_type == proto::plan::OpType::TextMatch) { + return index->MatchQuery(query); + } else if (op_type == proto::plan::OpType::PhraseMatch) { + return index->PhraseMatchQuery(query, slop); + } else { + PanicInfo(OpTypeInvalid, + "unsupported operator type for match query: {}", + op_type); + } }; auto res = ProcessTextMatchIndex(func, query); return res; diff --git a/internal/core/src/expr/ITypeExpr.h b/internal/core/src/expr/ITypeExpr.h index 254975e3ff3d8..fed5a20b3b17f 100644 --- a/internal/core/src/expr/ITypeExpr.h +++ b/internal/core/src/expr/ITypeExpr.h @@ -349,10 +349,16 @@ class ValueExpr : public ITypeExpr { class UnaryRangeFilterExpr : public ITypeFilterExpr { public: - explicit UnaryRangeFilterExpr(const ColumnInfo& column, - proto::plan::OpType op_type, - const proto::plan::GenericValue& val) - : ITypeFilterExpr(), column_(column), op_type_(op_type), val_(val) { + explicit UnaryRangeFilterExpr( + const ColumnInfo& column, + proto::plan::OpType op_type, + const proto::plan::GenericValue& val, + const std::vector& extra_values) + : ITypeFilterExpr(), + column_(column), + op_type_(op_type), + val_(val), + extra_values_(extra_values) { } std::string @@ -360,7 +366,16 @@ class UnaryRangeFilterExpr : public ITypeFilterExpr { std::stringstream ss; ss << "UnaryRangeFilterExpr: {columnInfo:" << column_.ToString() << " op_type:" << milvus::proto::plan::OpType_Name(op_type_) - << " val:" << val_.DebugString() << "}"; + << " val:" << val_.DebugString() << " extra_values: ["; + + for (size_t i = 0; i < extra_values_.size(); i++) { + ss << extra_values_[i].DebugString(); + if (i != extra_values_.size() - 1) { + ss << ", "; + } + } + + ss << "]}"; return ss.str(); } @@ -393,6 +408,7 @@ class UnaryRangeFilterExpr : public ITypeFilterExpr { const ColumnInfo column_; const proto::plan::OpType op_type_; const proto::plan::GenericValue val_; + const std::vector extra_values_; }; class AlwaysTrueExpr : public ITypeFilterExpr { diff --git a/internal/core/src/index/TextMatchIndex.cpp b/internal/core/src/index/TextMatchIndex.cpp index 662295ddb4cd6..0422c3bf2a7fb 100644 --- a/internal/core/src/index/TextMatchIndex.cpp +++ b/internal/core/src/index/TextMatchIndex.cpp @@ -291,4 +291,25 @@ TextMatchIndex::MatchQuery(const std::string& query) { apply_hits(bitset, hits, true); return bitset; } + +TargetBitmap +TextMatchIndex::PhraseMatchQuery(const std::string& query, uint32_t slop) { + if (shouldTriggerCommit()) { + Commit(); + Reload(); + } + + // The count opeartion of tantivy may be get older cnt if the index is committed with new tantivy segment. + // So we cannot use the count operation to get the total count for bitmap. + // Just use the maximum offset of hits to get the total count for bitmap here. + auto hits = wrapper_->phrase_match_query(query, slop); + auto cnt = should_allocate_bitset_size(hits); + TargetBitmap bitset(cnt); + if (bitset.empty()) { + return bitset; + } + apply_hits(bitset, hits, true); + return bitset; +} + } // namespace milvus::index diff --git a/internal/core/src/index/TextMatchIndex.h b/internal/core/src/index/TextMatchIndex.h index 17d14340fc70d..afdfef59ade19 100644 --- a/internal/core/src/index/TextMatchIndex.h +++ b/internal/core/src/index/TextMatchIndex.h @@ -81,6 +81,9 @@ class TextMatchIndex : public InvertedIndexTantivy { TargetBitmap MatchQuery(const std::string& query); + TargetBitmap + PhraseMatchQuery(const std::string& query, uint32_t slop); + private: bool shouldTriggerCommit(); diff --git a/internal/core/src/query/PlanProto.cpp b/internal/core/src/query/PlanProto.cpp index 05c0074016f16..f57daafe4dff4 100644 --- a/internal/core/src/query/PlanProto.cpp +++ b/internal/core/src/query/PlanProto.cpp @@ -313,8 +313,15 @@ ProtoParser::ParseUnaryRangeExprs(const proto::plan::UnaryRangeExpr& expr_pb) { auto field_id = FieldId(column_info.field_id()); auto data_type = schema[field_id].get_data_type(); Assert(data_type == static_cast(column_info.data_type())); + std::vector<::milvus::proto::plan::GenericValue> extra_values; + for (auto val : expr_pb.extra_values()) { + extra_values.emplace_back(val); + } return std::make_shared( - expr::ColumnInfo(column_info), expr_pb.op(), expr_pb.value()); + expr::ColumnInfo(column_info), + expr_pb.op(), + expr_pb.value(), + extra_values); } expr::TypedExprPtr diff --git a/internal/core/thirdparty/tantivy/tantivy-binding/include/tantivy-binding.h b/internal/core/thirdparty/tantivy/tantivy-binding/include/tantivy-binding.h index 61a2088a2d26f..f997151396e59 100644 --- a/internal/core/thirdparty/tantivy/tantivy-binding/include/tantivy-binding.h +++ b/internal/core/thirdparty/tantivy/tantivy-binding/include/tantivy-binding.h @@ -137,6 +137,8 @@ RustResult tantivy_regex_query(void *ptr, const char *pattern); RustResult tantivy_match_query(void *ptr, const char *query); +RustResult tantivy_phrase_match_query(void *ptr, const char *query, uint32_t slop); + RustResult tantivy_register_tokenizer(void *ptr, const char *tokenizer_name, const char *analyzer_params); diff --git a/internal/core/thirdparty/tantivy/tantivy-binding/src/index_reader_text.rs b/internal/core/thirdparty/tantivy/tantivy-binding/src/index_reader_text.rs index f83df709c4098..e696d6bb8a33d 100644 --- a/internal/core/thirdparty/tantivy/tantivy-binding/src/index_reader_text.rs +++ b/internal/core/thirdparty/tantivy/tantivy-binding/src/index_reader_text.rs @@ -1,17 +1,16 @@ use tantivy::{ - query::BooleanQuery, + query::{BooleanQuery, PhraseQuery}, tokenizer::{TextAnalyzer, TokenStream}, Term, }; -use crate::error::Result; +use crate::error::{Result, TantivyBindingError}; use crate::{index_reader::IndexReaderWrapper, tokenizer::standard_analyzer}; impl IndexReaderWrapper { // split the query string into multiple tokens using index's default tokenizer, // and then execute the disconjunction of term query. pub(crate) fn match_query(&self, q: &str) -> Result> { - // clone the tokenizer to make `match_query` thread-safe. let mut tokenizer = self .index .tokenizer_for_field(self.field) @@ -27,6 +26,31 @@ impl IndexReaderWrapper { self.search(&query) } + // split the query string into multiple tokens using index's default tokenizer, + // and then execute the disconjunction of term query. + pub(crate) fn phrase_match_query(&self, q: &str, slop: u32) -> Result> { + // clone the tokenizer to make `match_query` thread-safe. + let mut tokenizer = self + .index + .tokenizer_for_field(self.field) + .unwrap_or(standard_analyzer(vec![])) + .clone(); + let mut token_stream = tokenizer.token_stream(q); + let mut terms: Vec = Vec::new(); + while token_stream.advance() { + let token = token_stream.token(); + terms.push(Term::from_field_text(self.field, &token.text)); + } + if terms.len() <= 1 { + // tantivy will panic when terms.len() <= 1, so we forward to text match instead. + let query = BooleanQuery::new_multiterms_query(terms); + return self.search(&query); + } + let terms = terms.into_iter().enumerate().collect(); + let phrase_query = PhraseQuery::new_with_offset_and_slop(terms, slop); + self.search(&phrase_query) + } + pub(crate) fn register_tokenizer(&self, tokenizer_name: String, tokenizer: TextAnalyzer) { self.index.tokenizers().register(&tokenizer_name, tokenizer) } diff --git a/internal/core/thirdparty/tantivy/tantivy-binding/src/index_reader_text_c.rs b/internal/core/thirdparty/tantivy/tantivy-binding/src/index_reader_text_c.rs index a7f6f12ef56d1..e829bfbb30302 100644 --- a/internal/core/thirdparty/tantivy/tantivy-binding/src/index_reader_text_c.rs +++ b/internal/core/thirdparty/tantivy/tantivy-binding/src/index_reader_text_c.rs @@ -20,6 +20,19 @@ pub extern "C" fn tantivy_match_query(ptr: *mut c_void, query: *const c_char) -> } } +#[no_mangle] +pub extern "C" fn tantivy_phrase_match_query( + ptr: *mut c_void, + query: *const c_char, + slop: u32, +) -> RustResult { + let real = ptr as *mut IndexReaderWrapper; + unsafe { + let query = cstr_to_str!(query); + (*real).phrase_match_query(query, slop).into() + } +} + #[no_mangle] pub extern "C" fn tantivy_register_tokenizer( ptr: *mut c_void, diff --git a/internal/core/thirdparty/tantivy/tantivy-wrapper.h b/internal/core/thirdparty/tantivy/tantivy-wrapper.h index d532b9f100562..71d9d9664f9fa 100644 --- a/internal/core/thirdparty/tantivy/tantivy-wrapper.h +++ b/internal/core/thirdparty/tantivy/tantivy-wrapper.h @@ -577,6 +577,19 @@ struct TantivyIndexWrapper { return RustArrayWrapper(std::move(res.result_->value.rust_array._0)); } + RustArrayWrapper + phrase_match_query(const std::string& query, uint32_t slop) { + auto array = tantivy_phrase_match_query(reader_, query.c_str(), slop); + auto res = RustResultWrapper(array); + AssertInfo(res.result_->success, + "TantivyIndexWrapper.phrase_match_query: {}", + res.result_->error); + AssertInfo( + res.result_->value.tag == Value::Tag::RustArray, + "TantivyIndexWrapper.phrase_match_query: invalid result type"); + return RustArrayWrapper(std::move(res.result_->value.rust_array._0)); + } + public: inline IndexWriter get_writer() { diff --git a/internal/core/unittest/test_array_expr.cpp b/internal/core/unittest/test_array_expr.cpp index 1be5522082432..ac4debfd7b5c4 100644 --- a/internal/core/unittest/test_array_expr.cpp +++ b/internal/core/unittest/test_array_expr.cpp @@ -2461,7 +2461,8 @@ TEST(Expr, TestArrayStringMatch) { milvus::expr::ColumnInfo( string_array_fid, DataType::ARRAY, testcase.nested_path), testcase.op_type, - value); + value, + std::vector{}); BitsetType final; auto plan = std::make_shared(DEFAULT_PLANNODE_ID, expr); diff --git a/internal/core/unittest/test_exec.cpp b/internal/core/unittest/test_exec.cpp index 6e65c1f089dbd..c98ae465c771a 100644 --- a/internal/core/unittest/test_exec.cpp +++ b/internal/core/unittest/test_exec.cpp @@ -180,7 +180,8 @@ TEST_P(TaskTest, UnaryExpr) { auto logical_expr = std::make_shared( expr::ColumnInfo(field_map_["int64"], DataType::INT64), proto::plan::OpType::LessThan, - value); + value, + std::vector{}); std::vector sources; auto filter_node = std::make_shared( "plannode id 1", logical_expr, sources); @@ -217,11 +218,13 @@ TEST_P(TaskTest, LogicalExpr) { auto left = std::make_shared( expr::ColumnInfo(field_map_["int64"], DataType::INT64), proto::plan::OpType::LessThan, - value); + value, + std::vector{}); auto right = std::make_shared( expr::ColumnInfo(field_map_["int64"], DataType::INT64), proto::plan::OpType::LessThan, - value); + value, + std::vector{}); auto top = std::make_shared( expr::LogicalBinaryExpr::OpType::And, left, right); @@ -269,21 +272,25 @@ TEST_P(TaskTest, CompileInputs_and) { auto expr1 = std::make_shared( expr::ColumnInfo(int64_fid, DataType::INT64), proto::plan::OpType::GreaterThan, - val); + val, + std::vector{}); auto expr2 = std::make_shared( expr::ColumnInfo(int64_fid, DataType::INT64), proto::plan::OpType::GreaterThan, - val); + val, + std::vector{}); auto expr3 = std::make_shared( expr::LogicalBinaryExpr::OpType::And, expr1, expr2); auto expr4 = std::make_shared( expr::ColumnInfo(int64_fid, DataType::INT64), proto::plan::OpType::GreaterThan, - val); + val, + std::vector{}); auto expr5 = std::make_shared( expr::ColumnInfo(int64_fid, DataType::INT64), proto::plan::OpType::GreaterThan, - val); + val, + std::vector{}); auto expr6 = std::make_shared( expr::LogicalBinaryExpr::OpType::And, expr1, expr2); auto expr7 = std::make_shared( @@ -313,21 +320,25 @@ TEST_P(TaskTest, CompileInputs_or_with_and) { auto expr1 = std::make_shared( expr::ColumnInfo(int64_fid, DataType::INT64), proto::plan::OpType::GreaterThan, - val); + val, + std::vector{}); auto expr2 = std::make_shared( expr::ColumnInfo(int64_fid, DataType::INT64), proto::plan::OpType::GreaterThan, - val); + val, + std::vector{}); auto expr3 = std::make_shared( expr::LogicalBinaryExpr::OpType::And, expr1, expr2); auto expr4 = std::make_shared( expr::ColumnInfo(int64_fid, DataType::INT64), proto::plan::OpType::GreaterThan, - val); + val, + std::vector{}); auto expr5 = std::make_shared( expr::ColumnInfo(int64_fid, DataType::INT64), proto::plan::OpType::GreaterThan, - val); + val, + std::vector{}); auto expr6 = std::make_shared( expr::LogicalBinaryExpr::OpType::And, expr1, expr2); auto query_context = std::make_shared( @@ -347,21 +358,25 @@ TEST_P(TaskTest, CompileInputs_or_with_and) { auto expr1 = std::make_shared( expr::ColumnInfo(int64_fid, DataType::INT64), proto::plan::OpType::GreaterThan, - val); + val, + std::vector{}); auto expr2 = std::make_shared( expr::ColumnInfo(int64_fid, DataType::INT64), proto::plan::OpType::GreaterThan, - val); + val, + std::vector{}); auto expr3 = std::make_shared( expr::LogicalBinaryExpr::OpType::Or, expr1, expr2); auto expr4 = std::make_shared( expr::ColumnInfo(int64_fid, DataType::INT64), proto::plan::OpType::GreaterThan, - val); + val, + std::vector{}); auto expr5 = std::make_shared( expr::ColumnInfo(int64_fid, DataType::INT64), proto::plan::OpType::GreaterThan, - val); + val, + std::vector{}); auto expr6 = std::make_shared( expr::LogicalBinaryExpr::OpType::And, expr1, expr2); auto query_context = std::make_shared( @@ -384,21 +399,25 @@ TEST_P(TaskTest, CompileInputs_or_with_and) { auto expr1 = std::make_shared( expr::ColumnInfo(int64_fid, DataType::INT64), proto::plan::OpType::GreaterThan, - val); + val, + std::vector{}); auto expr2 = std::make_shared( expr::ColumnInfo(int64_fid, DataType::INT64), proto::plan::OpType::GreaterThan, - val); + val, + std::vector{}); auto expr3 = std::make_shared( expr::LogicalBinaryExpr::OpType::Or, expr1, expr2); auto expr4 = std::make_shared( expr::ColumnInfo(int64_fid, DataType::INT64), proto::plan::OpType::GreaterThan, - val); + val, + std::vector{}); auto expr5 = std::make_shared( expr::ColumnInfo(int64_fid, DataType::INT64), proto::plan::OpType::GreaterThan, - val); + val, + std::vector{}); auto expr6 = std::make_shared( expr::LogicalBinaryExpr::OpType::And, expr1, expr2); auto query_context = std::make_shared( diff --git a/internal/core/unittest/test_expr.cpp b/internal/core/unittest/test_expr.cpp index 4a31ecbd1466a..3bc50c839cf04 100644 --- a/internal/core/unittest/test_expr.cpp +++ b/internal/core/unittest/test_expr.cpp @@ -1315,7 +1315,8 @@ TEST_P(ExprTest, TestUnaryRangeJson) { milvus::expr::ColumnInfo( json_fid, DataType::JSON, testcase.nested_path), op, - value); + value, + std::vector{}); auto plan = std::make_shared( DEFAULT_PLANNODE_ID, expr); auto final = ExecuteQueryExpr( @@ -1399,7 +1400,8 @@ TEST_P(ExprTest, TestUnaryRangeJson) { milvus::expr::ColumnInfo( json_fid, DataType::JSON, testcase.nested_path), op, - testcase.val); + testcase.val, + std::vector{}); BitsetType final; auto plan = std::make_shared( DEFAULT_PLANNODE_ID, expr); @@ -1559,7 +1561,8 @@ TEST_P(ExprTest, TestUnaryRangeJsonNullable) { milvus::expr::ColumnInfo( json_fid, DataType::JSON, testcase.nested_path), op, - value); + value, + std::vector{}); BitsetType final; auto plan = std::make_shared( DEFAULT_PLANNODE_ID, expr); @@ -1647,7 +1650,8 @@ TEST_P(ExprTest, TestUnaryRangeJsonNullable) { milvus::expr::ColumnInfo( json_fid, DataType::JSON, testcase.nested_path), op, - testcase.val); + testcase.val, + std::vector{}); BitsetType final; auto plan = std::make_shared( DEFAULT_PLANNODE_ID, expr); @@ -3758,21 +3762,24 @@ TEST(Expr, TestExprPerformance) { return std::make_shared( expr::ColumnInfo(fids[data_type], data_type), proto::plan::OpType::LessThan, - val); + val, + std::vector{}); } else if (IsFloatDataType(data_type)) { proto::plan::GenericValue val; val.set_float_val(float(value)); return std::make_shared( expr::ColumnInfo(fids[data_type], data_type), proto::plan::OpType::LessThan, - val); + val, + std::vector{}); } else if (IsStringDataType(data_type)) { proto::plan::GenericValue val; val.set_string_val(std::to_string(value)); return std::make_shared( expr::ColumnInfo(fids[data_type], data_type), proto::plan::OpType::LessThan, - val); + val, + std::vector{}); } else { throw std::runtime_error("not supported type"); } @@ -4133,21 +4140,24 @@ TEST(Expr, TestExprNOT) { return std::make_shared( expr::ColumnInfo(fids[data_type], data_type), proto::plan::OpType::LessThan, - val); + val, + std::vector{}); } else if (IsFloatDataType(data_type)) { proto::plan::GenericValue val; val.set_float_val(float(value)); return std::make_shared( expr::ColumnInfo(fids[data_type], data_type), proto::plan::OpType::LessThan, - val); + val, + std::vector{}); } else if (IsStringDataType(data_type)) { proto::plan::GenericValue val; val.set_string_val(std::to_string(value)); return std::make_shared( expr::ColumnInfo(fids[data_type], data_type), proto::plan::OpType::LessThan, - val); + val, + std::vector{}); } else { throw std::runtime_error("not supported type"); } @@ -4544,7 +4554,8 @@ TEST_P(ExprTest, TestGrowingSegmentGetBatchSize) { auto expr = std::make_shared( expr::ColumnInfo(int8_fid, DataType::INT8), proto::plan::OpType::GreaterThan, - val); + val, + std::vector{}); auto plan_node = std::make_shared(DEFAULT_PLANNODE_ID, expr); @@ -4618,12 +4629,14 @@ TEST_P(ExprTest, TestConjuctExpr) { auto left = std::make_shared( expr::ColumnInfo(int64_fid, DataType::INT64), proto::plan::OpType::GreaterThan, - value); + value, + std::vector{}); value.set_int64_val(r); auto right = std::make_shared( expr::ColumnInfo(int64_fid, DataType::INT64), proto::plan::OpType::LessThan, - value); + value, + std::vector{}); return std::make_shared( expr::LogicalBinaryExpr::OpType::And, left, right); @@ -4706,12 +4719,14 @@ TEST_P(ExprTest, TestConjuctExprNullable) { auto left = std::make_shared( expr::ColumnInfo(int64_fid, DataType::INT64), proto::plan::OpType::GreaterThan, - value); + value, + std::vector{}); value.set_int64_val(r); auto right = std::make_shared( expr::ColumnInfo(int64_fid, DataType::INT64), proto::plan::OpType::LessThan, - value); + value, + std::vector{}); return std::make_shared( expr::LogicalBinaryExpr::OpType::And, left, right); @@ -4804,7 +4819,8 @@ TEST_P(ExprTest, TestUnaryBenchTest) { auto expr = std::make_shared( expr::ColumnInfo(pair.first, pair.second), proto::plan::OpType::GreaterThan, - val); + val, + std::vector{}); BitsetType final; auto plan = std::make_shared(DEFAULT_PLANNODE_ID, expr); @@ -4956,7 +4972,8 @@ TEST_P(ExprTest, TestLogicalUnaryBenchTest) { auto child_expr = std::make_shared( expr::ColumnInfo(pair.first, pair.second), proto::plan::OpType::GreaterThan, - val); + val, + std::vector{}); auto expr = std::make_shared( expr::LogicalUnaryExpr::OpType::LogicalNot, child_expr); BitsetType final; @@ -5036,11 +5053,13 @@ TEST_P(ExprTest, TestBinaryLogicalBenchTest) { auto child1_expr = std::make_shared( expr::ColumnInfo(pair.first, pair.second), proto::plan::OpType::LessThan, - val); + val, + std::vector{}); auto child2_expr = std::make_shared( expr::ColumnInfo(pair.first, pair.second), proto::plan::OpType::NotEqual, - val1); + val1, + std::vector{}); auto expr = std::make_shared( expr::LogicalBinaryExpr::OpType::And, child1_expr, child2_expr); BitsetType final; @@ -5492,7 +5511,8 @@ TEST_P(ExprTest, TestRefactorExprs) { return std::make_shared( expr::ColumnInfo(int64_fid, DataType::INT64), proto::plan::OpType::GreaterThan, - val); + val, + std::vector{}); } case TermExprImpl: { std::vector retrieve_ints; @@ -5539,7 +5559,8 @@ TEST_P(ExprTest, TestRefactorExprs) { auto child_expr = std::make_shared( expr::ColumnInfo(int8_fid, DataType::INT8), proto::plan::OpType::GreaterThan, - val); + val, + std::vector{}); return std::make_shared( expr::LogicalUnaryExpr::OpType::LogicalNot, child_expr); } @@ -5549,11 +5570,13 @@ TEST_P(ExprTest, TestRefactorExprs) { auto child1_expr = std::make_shared( expr::ColumnInfo(int8_fid, DataType::INT8), proto::plan::OpType::GreaterThan, - val); + val, + std::vector{}); auto child2_expr = std::make_shared( expr::ColumnInfo(int8_fid, DataType::INT8), proto::plan::OpType::NotEqual, - val); + val, + std::vector{}); ; return std::make_shared( expr::LogicalBinaryExpr::OpType::And, @@ -5578,7 +5601,8 @@ TEST_P(ExprTest, TestRefactorExprs) { return std::make_shared( expr::ColumnInfo(int8_fid, DataType::INT8), proto::plan::OpType::GreaterThan, - val); + val, + std::vector{}); } } }; diff --git a/internal/core/unittest/test_retrieve.cpp b/internal/core/unittest/test_retrieve.cpp index fc354f17c92d0..d570bdb3913ae 100644 --- a/internal/core/unittest/test_retrieve.cpp +++ b/internal/core/unittest/test_retrieve.cpp @@ -300,7 +300,8 @@ TEST_P(RetrieveTest, Limit) { milvus::expr::ColumnInfo( fid_64, DataType::INT64, std::vector()), OpType::GreaterEqual, - unary_val); + unary_val, + std::vector{}); plan->plan_node_ = std::make_unique(); plan->plan_node_->plannodes_ = milvus::test::CreateRetrievePlanByExpr(expr); @@ -347,7 +348,8 @@ TEST_P(RetrieveTest, FillEntry) { milvus::expr::ColumnInfo( fid_64, DataType::INT64, std::vector()), OpType::GreaterEqual, - unary_val); + unary_val, + std::vector{}); plan->plan_node_ = std::make_unique(); plan->plan_node_->plannodes_ = milvus::test::CreateRetrievePlanByExpr(expr); // test query results exceed the limit size diff --git a/internal/core/unittest/test_text_match.cpp b/internal/core/unittest/test_text_match.cpp index f7b6366385e1c..a726cb83e57d0 100644 --- a/internal/core/unittest/test_text_match.cpp +++ b/internal/core/unittest/test_text_match.cpp @@ -60,15 +60,21 @@ GenTestSchema(std::map params = {}, return schema; } std::shared_ptr -GetTextMatchExpr(SchemaPtr schema, const std::string& query) { +GetMatchExpr(SchemaPtr schema, + const std::string& query, + proto::plan::OpType op, + int64_t slop = 0) { const auto& str_meta = schema->operator[](FieldName("str")); auto column_info = test::GenColumnInfo(str_meta.get_id().get(), proto::schema::DataType::VarChar, false, false); - auto unary_range_expr = test::GenUnaryRangeExpr(OpType::TextMatch, query); + auto unary_range_expr = test::GenUnaryRangeExpr(op, query); unary_range_expr->set_allocated_column_info(column_info); + auto generic_for_slop = milvus::test::GenGenericValue(slop); + unary_range_expr->add_extra_values()->CopyFrom(*generic_for_slop); + delete generic_for_slop; auto expr = test::GenExpr(); expr->set_allocated_unary_range_expr(unary_range_expr); @@ -80,14 +86,22 @@ GetTextMatchExpr(SchemaPtr schema, const std::string& query) { }; std::shared_ptr -GetNotTextMatchExpr(SchemaPtr schema, const std::string& query) { +GetNotMatchExpr(SchemaPtr schema, + const std::string& query, + proto::plan::OpType op, + int64_t slop = 0) { const auto& str_meta = schema->operator[](FieldName("str")); proto::plan::GenericValue val; val.set_string_val(query); + std::vector extra_values; + auto generic_for_slop = milvus::test::GenGenericValue(slop); + extra_values.push_back(*generic_for_slop); + delete generic_for_slop; auto child_expr = std::make_shared( milvus::expr::ColumnInfo(str_meta.get_id(), DataType::VARCHAR), - proto::plan::OpType::TextMatch, - val); + op, + val, + extra_values); auto expr = std::make_shared( expr::LogicalUnaryExpr::OpType::LogicalNot, child_expr); auto parsed = @@ -140,21 +154,50 @@ TEST(TextMatch, Index) { index->AddText("swimming, football", true, 2); index->Commit(); index->Reload(); - auto res = index->MatchQuery("football"); - ASSERT_EQ(res.size(), 3); - ASSERT_TRUE(res[0]); - ASSERT_FALSE(res[1]); - ASSERT_TRUE(res[2]); - auto res1 = index->IsNull(); - ASSERT_FALSE(res1[0]); - ASSERT_TRUE(res1[1]); - ASSERT_FALSE(res1[2]); - auto res2 = index->IsNotNull(); - ASSERT_TRUE(res2[0]); - ASSERT_FALSE(res2[1]); - ASSERT_TRUE(res2[2]); - res = index->MatchQuery("nothing"); - ASSERT_EQ(res.size(), 0); + + { + auto res = index->MatchQuery("football"); + ASSERT_EQ(res.size(), 3); + ASSERT_TRUE(res[0]); + ASSERT_FALSE(res[1]); + ASSERT_TRUE(res[2]); + auto res1 = index->IsNull(); + ASSERT_FALSE(res1[0]); + ASSERT_TRUE(res1[1]); + ASSERT_FALSE(res1[2]); + auto res2 = index->IsNotNull(); + ASSERT_TRUE(res2[0]); + ASSERT_FALSE(res2[1]); + ASSERT_TRUE(res2[2]); + res = index->MatchQuery("nothing"); + ASSERT_EQ(res.size(), 0); + } + + { + auto res = index->PhraseMatchQuery("football", 0); + ASSERT_EQ(res.size(), 3); + ASSERT_TRUE(res[0]); + ASSERT_FALSE(res[1]); + ASSERT_TRUE(res[2]); + + auto res1 = index->PhraseMatchQuery("swimming football", 0); + ASSERT_EQ(res1.size(), 3); + ASSERT_FALSE(res1[0]); + ASSERT_FALSE(res1[1]); + ASSERT_TRUE(res1[2]); + + auto res2 = index->PhraseMatchQuery("football swimming", 0); + ASSERT_EQ(res2.size(), 0); + + auto res3 = index->PhraseMatchQuery("football swimming", 1); + ASSERT_EQ(res3.size(), 0); + + auto res4 = index->PhraseMatchQuery("football swimming", 2); + ASSERT_EQ(res4.size(), 3); + ASSERT_FALSE(res4[0]); + ASSERT_FALSE(res4[1]); + ASSERT_TRUE(res4[2]); + } } TEST(TextMatch, GrowingNaive) { @@ -185,45 +228,64 @@ TEST(TextMatch, GrowingNaive) { std::this_thread::sleep_for(std::chrono::milliseconds(200) * 2); { - auto expr = GetTextMatchExpr(schema, "football"); BitsetType final; - final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_TRUE(final[0]); - ASSERT_TRUE(final[1]); - auto expr1 = GetNotTextMatchExpr(schema, "football"); - final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_FALSE(final[0]); - ASSERT_FALSE(final[1]); + for (auto op : {OpType::TextMatch, OpType::PhraseMatch}) { + auto expr = GetMatchExpr(schema, "football", op); + final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_TRUE(final[0]); + ASSERT_TRUE(final[1]); + auto expr1 = GetNotMatchExpr(schema, "football", op); + final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_FALSE(final[0]); + ASSERT_FALSE(final[1]); + } } { - auto expr = GetTextMatchExpr(schema, "swimming"); BitsetType final; - final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_FALSE(final[0]); - ASSERT_TRUE(final[1]); - auto expr1 = GetNotTextMatchExpr(schema, "swimming"); - final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_TRUE(final[0]); - ASSERT_FALSE(final[1]); + for (auto op : {OpType::TextMatch, OpType::PhraseMatch}) { + auto expr = GetMatchExpr(schema, "swimming", op); + final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_FALSE(final[0]); + ASSERT_TRUE(final[1]); + auto expr1 = GetNotMatchExpr(schema, "swimming", op); + final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_TRUE(final[0]); + ASSERT_FALSE(final[1]); + } } { - auto expr = GetTextMatchExpr(schema, "basketball, swimming"); + auto expr = + GetMatchExpr(schema, "basketball, swimming", OpType::TextMatch); BitsetType final; final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); ASSERT_EQ(final.size(), N); ASSERT_TRUE(final[0]); ASSERT_TRUE(final[1]); - auto expr1 = GetNotTextMatchExpr(schema, "basketball, swimming"); + auto expr1 = + GetNotMatchExpr(schema, "basketball, swimming", OpType::TextMatch); final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); ASSERT_EQ(final.size(), N); ASSERT_FALSE(final[0]); ASSERT_FALSE(final[1]); + + auto expr2 = + GetMatchExpr(schema, "football, pingpang", OpType::PhraseMatch); + final = ExecuteQueryExpr(expr2, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_FALSE(final[0]); + ASSERT_FALSE(final[1]); + auto expr3 = + GetMatchExpr(schema, "football, pingpang", OpType::PhraseMatch, 1); + final = ExecuteQueryExpr(expr3, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_TRUE(final[0]); + ASSERT_FALSE(final[1]); } } @@ -260,49 +322,70 @@ TEST(TextMatch, GrowingNaiveNullable) { std::this_thread::sleep_for(std::chrono::milliseconds(200) * 2); { - auto expr = GetTextMatchExpr(schema, "football"); + BitsetType final; + for (auto op : {OpType::TextMatch, OpType::PhraseMatch}) { + auto expr = GetMatchExpr(schema, "football", op); + final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_TRUE(final[0]); + ASSERT_TRUE(final[1]); + ASSERT_FALSE(final[2]); + auto expr1 = GetNotMatchExpr(schema, "football", op); + final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_FALSE(final[0]); + ASSERT_FALSE(final[1]); + ASSERT_FALSE(final[2]); + } + } + + { + BitsetType final; + for (auto op : {OpType::TextMatch, OpType::PhraseMatch}) { + auto expr = GetMatchExpr(schema, "swimming", op); + final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_FALSE(final[0]); + ASSERT_TRUE(final[1]); + ASSERT_FALSE(final[2]); + auto expr1 = GetNotMatchExpr(schema, "swimming", op); + final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_TRUE(final[0]); + ASSERT_FALSE(final[1]); + ASSERT_FALSE(final[2]); + } + } + + { + auto expr = + GetMatchExpr(schema, "basketball, swimming", OpType::TextMatch); BitsetType final; final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); ASSERT_EQ(final.size(), N); ASSERT_TRUE(final[0]); ASSERT_TRUE(final[1]); ASSERT_FALSE(final[2]); - auto expr1 = GetNotTextMatchExpr(schema, "football"); + auto expr1 = + GetNotMatchExpr(schema, "basketball, swimming", OpType::TextMatch); final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); ASSERT_EQ(final.size(), N); ASSERT_FALSE(final[0]); ASSERT_FALSE(final[1]); ASSERT_FALSE(final[2]); - } - { - auto expr = GetTextMatchExpr(schema, "swimming"); - BitsetType final; - final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); + auto expr2 = + GetMatchExpr(schema, "football, pingpang", OpType::PhraseMatch); + final = ExecuteQueryExpr(expr2, seg.get(), N, MAX_TIMESTAMP); ASSERT_EQ(final.size(), N); ASSERT_FALSE(final[0]); - ASSERT_TRUE(final[1]); - ASSERT_FALSE(final[2]); - auto expr1 = GetNotTextMatchExpr(schema, "swimming"); - final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_TRUE(final[0]); ASSERT_FALSE(final[1]); ASSERT_FALSE(final[2]); - } - - { - auto expr = GetTextMatchExpr(schema, "basketball, swimming"); - BitsetType final; - final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); + auto expr3 = + GetMatchExpr(schema, "football, pingpang", OpType::PhraseMatch, 1); + final = ExecuteQueryExpr(expr3, seg.get(), N, MAX_TIMESTAMP); ASSERT_EQ(final.size(), N); ASSERT_TRUE(final[0]); - ASSERT_TRUE(final[1]); - ASSERT_FALSE(final[2]); - auto expr1 = GetNotTextMatchExpr(schema, "basketball, swimming"); - final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_FALSE(final[0]); ASSERT_FALSE(final[1]); ASSERT_FALSE(final[2]); } @@ -330,45 +413,64 @@ TEST(TextMatch, SealedNaive) { seg->CreateTextIndex(FieldId(101)); { - auto expr = GetTextMatchExpr(schema, "football"); BitsetType final; - final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_TRUE(final[0]); - ASSERT_TRUE(final[1]); - auto expr1 = GetNotTextMatchExpr(schema, "football"); - final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_FALSE(final[0]); - ASSERT_FALSE(final[1]); + for (auto op : {OpType::TextMatch, OpType::PhraseMatch}) { + auto expr = GetMatchExpr(schema, "football", op); + final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_TRUE(final[0]); + ASSERT_TRUE(final[1]); + auto expr1 = GetNotMatchExpr(schema, "football", op); + final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_FALSE(final[0]); + ASSERT_FALSE(final[1]); + } } { - auto expr = GetTextMatchExpr(schema, "swimming"); BitsetType final; - final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_FALSE(final[0]); - ASSERT_TRUE(final[1]); - auto expr1 = GetNotTextMatchExpr(schema, "swimming"); - final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_TRUE(final[0]); - ASSERT_FALSE(final[1]); + for (auto op : {OpType::TextMatch, OpType::PhraseMatch}) { + auto expr = GetMatchExpr(schema, "swimming", op); + final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_FALSE(final[0]); + ASSERT_TRUE(final[1]); + auto expr1 = GetNotMatchExpr(schema, "swimming", op); + final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_TRUE(final[0]); + ASSERT_FALSE(final[1]); + } } { - auto expr = GetTextMatchExpr(schema, "basketball, swimming"); + auto expr = + GetMatchExpr(schema, "basketball, swimming", OpType::TextMatch); BitsetType final; final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); ASSERT_EQ(final.size(), N); ASSERT_TRUE(final[0]); ASSERT_TRUE(final[1]); - auto expr1 = GetNotTextMatchExpr(schema, "basketball, swimming"); + auto expr1 = + GetNotMatchExpr(schema, "basketball, swimming", OpType::TextMatch); final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); ASSERT_EQ(final.size(), N); ASSERT_FALSE(final[0]); ASSERT_FALSE(final[1]); + + auto expr2 = + GetMatchExpr(schema, "football, pingpang", OpType::PhraseMatch); + final = ExecuteQueryExpr(expr2, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_FALSE(final[0]); + ASSERT_FALSE(final[1]); + auto expr3 = + GetMatchExpr(schema, "football, pingpang", OpType::PhraseMatch, 1); + final = ExecuteQueryExpr(expr3, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_TRUE(final[0]); + ASSERT_FALSE(final[1]); } } @@ -399,49 +501,70 @@ TEST(TextMatch, SealedNaiveNullable) { SealedLoadFieldData(raw_data, *seg); seg->CreateTextIndex(FieldId(101)); { - auto expr = GetTextMatchExpr(schema, "football"); + BitsetType final; + for (auto op : {OpType::TextMatch, OpType::PhraseMatch}) { + auto expr = GetMatchExpr(schema, "football", op); + final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_TRUE(final[0]); + ASSERT_TRUE(final[1]); + ASSERT_FALSE(final[2]); + auto expr1 = GetNotMatchExpr(schema, "football", op); + final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_FALSE(final[0]); + ASSERT_FALSE(final[1]); + ASSERT_FALSE(final[2]); + } + } + + { + BitsetType final; + for (auto op : {OpType::TextMatch, OpType::PhraseMatch}) { + auto expr = GetMatchExpr(schema, "swimming", op); + final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_FALSE(final[0]); + ASSERT_TRUE(final[1]); + ASSERT_FALSE(final[2]); + auto expr1 = GetNotMatchExpr(schema, "swimming", op); + final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_TRUE(final[0]); + ASSERT_FALSE(final[1]); + ASSERT_FALSE(final[2]); + } + } + + { + auto expr = + GetMatchExpr(schema, "basketball, swimming", OpType::TextMatch); BitsetType final; final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); ASSERT_EQ(final.size(), N); ASSERT_TRUE(final[0]); ASSERT_TRUE(final[1]); ASSERT_FALSE(final[2]); - auto expr1 = GetNotTextMatchExpr(schema, "football"); + auto expr1 = + GetNotMatchExpr(schema, "basketball, swimming", OpType::TextMatch); final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); ASSERT_EQ(final.size(), N); ASSERT_FALSE(final[0]); ASSERT_FALSE(final[1]); ASSERT_FALSE(final[2]); - } - { - auto expr = GetTextMatchExpr(schema, "swimming"); - BitsetType final; - final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); + auto expr2 = + GetMatchExpr(schema, "football, pingpang", OpType::PhraseMatch); + final = ExecuteQueryExpr(expr2, seg.get(), N, MAX_TIMESTAMP); ASSERT_EQ(final.size(), N); ASSERT_FALSE(final[0]); - ASSERT_TRUE(final[1]); - ASSERT_FALSE(final[2]); - auto expr1 = GetNotTextMatchExpr(schema, "swimming"); - final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_TRUE(final[0]); ASSERT_FALSE(final[1]); ASSERT_FALSE(final[2]); - } - - { - auto expr = GetTextMatchExpr(schema, "basketball, swimming"); - BitsetType final; - final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); + auto expr3 = + GetMatchExpr(schema, "football, pingpang", OpType::PhraseMatch, 1); + final = ExecuteQueryExpr(expr3, seg.get(), N, MAX_TIMESTAMP); ASSERT_EQ(final.size(), N); ASSERT_TRUE(final[0]); - ASSERT_TRUE(final[1]); - ASSERT_FALSE(final[2]); - auto expr1 = GetNotTextMatchExpr(schema, "basketball, swimming"); - final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_FALSE(final[0]); ASSERT_FALSE(final[1]); ASSERT_FALSE(final[2]); } @@ -478,45 +601,65 @@ TEST(TextMatch, GrowingJieBa) { std::this_thread::sleep_for(std::chrono::milliseconds(200) * 2); { - auto expr = GetTextMatchExpr(schema, "青铜"); BitsetType final; - final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_TRUE(final[0]); - ASSERT_FALSE(final[1]); - auto expr1 = GetNotTextMatchExpr(schema, "青铜"); - final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_FALSE(final[0]); - ASSERT_TRUE(final[1]); + for (auto op : {OpType::TextMatch, OpType::PhraseMatch}) { + auto expr = GetMatchExpr(schema, "青铜", op); + final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_TRUE(final[0]); + ASSERT_FALSE(final[1]); + auto expr1 = GetNotMatchExpr(schema, "青铜", op); + final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_FALSE(final[0]); + ASSERT_TRUE(final[1]); + } } { - auto expr = GetTextMatchExpr(schema, "黄金"); BitsetType final; - final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_FALSE(final[0]); - ASSERT_TRUE(final[1]); - auto expr1 = GetNotTextMatchExpr(schema, "黄金"); - final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_TRUE(final[0]); - ASSERT_FALSE(final[1]); + for (auto op : {OpType::TextMatch, OpType::PhraseMatch}) { + auto expr = GetMatchExpr(schema, "黄金", op); + final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_FALSE(final[0]); + ASSERT_TRUE(final[1]); + auto expr1 = GetNotMatchExpr(schema, "黄金", op); + final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_TRUE(final[0]); + ASSERT_FALSE(final[1]); + } + } + + { + BitsetType final; + for (auto op : {OpType::TextMatch, OpType::PhraseMatch}) { + auto expr = GetMatchExpr(schema, "时代", op); + final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_TRUE(final[0]); + ASSERT_TRUE(final[1]); + auto expr1 = GetNotMatchExpr(schema, "时代", op); + final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_FALSE(final[0]); + ASSERT_FALSE(final[1]); + } } { - auto expr = GetTextMatchExpr(schema, "时代"); BitsetType final; + auto expr = GetMatchExpr(schema, "黄金时代", OpType::PhraseMatch); final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); ASSERT_EQ(final.size(), N); - ASSERT_TRUE(final[0]); - ASSERT_TRUE(final[1]); - auto expr1 = GetNotTextMatchExpr(schema, "时代"); + ASSERT_FALSE(final[0]); + ASSERT_FALSE(final[1]); + auto expr1 = GetMatchExpr(schema, "黄金时代", OpType::PhraseMatch, 2); final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); ASSERT_EQ(final.size(), N); ASSERT_FALSE(final[0]); - ASSERT_FALSE(final[1]); + ASSERT_TRUE(final[1]); } } @@ -558,50 +701,72 @@ TEST(TextMatch, GrowingJieBaNullable) { std::this_thread::sleep_for(std::chrono::milliseconds(200) * 2); { - auto expr = GetTextMatchExpr(schema, "青铜"); BitsetType final; - final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_TRUE(final[0]); - ASSERT_FALSE(final[1]); - ASSERT_FALSE(final[2]); - auto expr1 = GetNotTextMatchExpr(schema, "青铜"); - final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_FALSE(final[0]); - ASSERT_TRUE(final[1]); - ASSERT_FALSE(final[2]); + for (auto op : {OpType::TextMatch, OpType::PhraseMatch}) { + auto expr = GetMatchExpr(schema, "青铜", op); + final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_TRUE(final[0]); + ASSERT_FALSE(final[1]); + ASSERT_FALSE(final[2]); + auto expr1 = GetNotMatchExpr(schema, "青铜", op); + final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_FALSE(final[0]); + ASSERT_TRUE(final[1]); + ASSERT_FALSE(final[2]); + } } { - auto expr = GetTextMatchExpr(schema, "黄金"); BitsetType final; - final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_FALSE(final[0]); - ASSERT_TRUE(final[1]); - ASSERT_FALSE(final[2]); - auto expr1 = GetNotTextMatchExpr(schema, "黄金"); - final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_TRUE(final[0]); - ASSERT_FALSE(final[1]); - ASSERT_FALSE(final[2]); + for (auto op : {OpType::TextMatch, OpType::PhraseMatch}) { + auto expr = GetMatchExpr(schema, "黄金", op); + final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_FALSE(final[0]); + ASSERT_TRUE(final[1]); + ASSERT_FALSE(final[2]); + auto expr1 = GetNotMatchExpr(schema, "黄金", op); + final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_TRUE(final[0]); + ASSERT_FALSE(final[1]); + ASSERT_FALSE(final[2]); + } + } + + { + BitsetType final; + for (auto op : {OpType::TextMatch, OpType::PhraseMatch}) { + auto expr = GetMatchExpr(schema, "时代", op); + final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_TRUE(final[0]); + ASSERT_TRUE(final[1]); + ASSERT_FALSE(final[2]); + auto expr1 = GetNotMatchExpr(schema, "时代", op); + final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_FALSE(final[0]); + ASSERT_FALSE(final[1]); + ASSERT_FALSE(final[2]); + } } { - auto expr = GetTextMatchExpr(schema, "时代"); BitsetType final; + auto expr = GetMatchExpr(schema, "黄金时代", OpType::PhraseMatch); final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); ASSERT_EQ(final.size(), N); - ASSERT_TRUE(final[0]); - ASSERT_TRUE(final[1]); + ASSERT_FALSE(final[0]); + ASSERT_FALSE(final[1]); ASSERT_FALSE(final[2]); - auto expr1 = GetNotTextMatchExpr(schema, "时代"); + auto expr1 = GetMatchExpr(schema, "黄金时代", OpType::PhraseMatch, 2); final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); ASSERT_EQ(final.size(), N); ASSERT_FALSE(final[0]); - ASSERT_FALSE(final[1]); + ASSERT_TRUE(final[1]); ASSERT_FALSE(final[2]); } } @@ -631,45 +796,65 @@ TEST(TextMatch, SealedJieBa) { seg->CreateTextIndex(FieldId(101)); { - auto expr = GetTextMatchExpr(schema, "青铜"); BitsetType final; - final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_TRUE(final[0]); - ASSERT_FALSE(final[1]); - auto expr1 = GetNotTextMatchExpr(schema, "青铜"); - final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_FALSE(final[0]); - ASSERT_TRUE(final[1]); + for (auto op : {OpType::TextMatch, OpType::PhraseMatch}) { + auto expr = GetMatchExpr(schema, "青铜", op); + final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_TRUE(final[0]); + ASSERT_FALSE(final[1]); + auto expr1 = GetNotMatchExpr(schema, "青铜", op); + final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_FALSE(final[0]); + ASSERT_TRUE(final[1]); + } } { - auto expr = GetTextMatchExpr(schema, "黄金"); BitsetType final; - final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_FALSE(final[0]); - ASSERT_TRUE(final[1]); - auto expr1 = GetNotTextMatchExpr(schema, "黄金"); - final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_TRUE(final[0]); - ASSERT_FALSE(final[1]); + for (auto op : {OpType::TextMatch, OpType::PhraseMatch}) { + auto expr = GetMatchExpr(schema, "黄金", op); + final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_FALSE(final[0]); + ASSERT_TRUE(final[1]); + auto expr1 = GetNotMatchExpr(schema, "黄金", op); + final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_TRUE(final[0]); + ASSERT_FALSE(final[1]); + } } { - auto expr = GetTextMatchExpr(schema, "时代"); BitsetType final; + for (auto op : {OpType::TextMatch, OpType::PhraseMatch}) { + auto expr = GetMatchExpr(schema, "时代", op); + final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_TRUE(final[0]); + ASSERT_TRUE(final[1]); + auto expr1 = GetNotMatchExpr(schema, "时代", op); + final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_FALSE(final[0]); + ASSERT_FALSE(final[1]); + } + } + + { + BitsetType final; + auto expr = GetMatchExpr(schema, "黄金时代", OpType::PhraseMatch); final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); ASSERT_EQ(final.size(), N); - ASSERT_TRUE(final[0]); - ASSERT_TRUE(final[1]); - auto expr1 = GetNotTextMatchExpr(schema, "时代"); + ASSERT_FALSE(final[0]); + ASSERT_FALSE(final[1]); + auto expr1 = GetMatchExpr(schema, "黄金时代", OpType::PhraseMatch, 2); final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); ASSERT_EQ(final.size(), N); ASSERT_FALSE(final[0]); - ASSERT_FALSE(final[1]); + ASSERT_TRUE(final[1]); } } @@ -706,50 +891,72 @@ TEST(TextMatch, SealedJieBaNullable) { seg->CreateTextIndex(FieldId(101)); { - auto expr = GetTextMatchExpr(schema, "青铜"); BitsetType final; - final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_TRUE(final[0]); - ASSERT_FALSE(final[1]); - ASSERT_FALSE(final[2]); - auto expr1 = GetNotTextMatchExpr(schema, "青铜"); - final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_FALSE(final[0]); - ASSERT_TRUE(final[1]); - ASSERT_FALSE(final[2]); + for (auto op : {OpType::TextMatch, OpType::PhraseMatch}) { + auto expr = GetMatchExpr(schema, "青铜", op); + final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_TRUE(final[0]); + ASSERT_FALSE(final[1]); + ASSERT_FALSE(final[2]); + auto expr1 = GetNotMatchExpr(schema, "青铜", op); + final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_FALSE(final[0]); + ASSERT_TRUE(final[1]); + ASSERT_FALSE(final[2]); + } } { - auto expr = GetTextMatchExpr(schema, "黄金"); BitsetType final; - final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_FALSE(final[0]); - ASSERT_TRUE(final[1]); - ASSERT_FALSE(final[2]); - auto expr1 = GetNotTextMatchExpr(schema, "黄金"); - final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); - ASSERT_EQ(final.size(), N); - ASSERT_TRUE(final[0]); - ASSERT_FALSE(final[1]); - ASSERT_FALSE(final[2]); + for (auto op : {OpType::TextMatch, OpType::PhraseMatch}) { + auto expr = GetMatchExpr(schema, "黄金", op); + final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_FALSE(final[0]); + ASSERT_TRUE(final[1]); + ASSERT_FALSE(final[2]); + auto expr1 = GetNotMatchExpr(schema, "黄金", op); + final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_TRUE(final[0]); + ASSERT_FALSE(final[1]); + ASSERT_FALSE(final[2]); + } + } + + { + BitsetType final; + for (auto op : {OpType::TextMatch, OpType::PhraseMatch}) { + auto expr = GetMatchExpr(schema, "时代", op); + final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_TRUE(final[0]); + ASSERT_TRUE(final[1]); + ASSERT_FALSE(final[2]); + auto expr1 = GetNotMatchExpr(schema, "时代", op); + final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); + ASSERT_EQ(final.size(), N); + ASSERT_FALSE(final[0]); + ASSERT_FALSE(final[1]); + ASSERT_FALSE(final[2]); + } } { - auto expr = GetTextMatchExpr(schema, "时代"); BitsetType final; + auto expr = GetMatchExpr(schema, "黄金时代", OpType::PhraseMatch); final = ExecuteQueryExpr(expr, seg.get(), N, MAX_TIMESTAMP); ASSERT_EQ(final.size(), N); - ASSERT_TRUE(final[0]); - ASSERT_TRUE(final[1]); + ASSERT_FALSE(final[0]); + ASSERT_FALSE(final[1]); ASSERT_FALSE(final[2]); - auto expr1 = GetNotTextMatchExpr(schema, "时代"); + auto expr1 = GetMatchExpr(schema, "黄金时代", OpType::PhraseMatch, 2); final = ExecuteQueryExpr(expr1, seg.get(), N, MAX_TIMESTAMP); ASSERT_EQ(final.size(), N); ASSERT_FALSE(final[0]); - ASSERT_FALSE(final[1]); + ASSERT_TRUE(final[1]); ASSERT_FALSE(final[2]); } } @@ -802,7 +1009,7 @@ TEST(TextMatch, GrowingLoadData) { ASSERT_NE(segment->get_field_avg_size(FieldId(101)), 0); // Check whether the text index has been built. - auto expr = GetTextMatchExpr(schema, "football"); + auto expr = GetMatchExpr(schema, "football", OpType::TextMatch); BitsetType final; final = ExecuteQueryExpr(expr, segment.get(), N, MAX_TIMESTAMP); ASSERT_EQ(final.size(), N); diff --git a/internal/parser/planparserv2/Plan.g4 b/internal/parser/planparserv2/Plan.g4 index 5b2eed3992132..819280c83db25 100644 --- a/internal/parser/planparserv2/Plan.g4 +++ b/internal/parser/planparserv2/Plan.g4 @@ -13,6 +13,7 @@ expr: | EmptyArray # EmptyArray | expr LIKE StringLiteral # Like | TEXTMATCH'('Identifier',' StringLiteral')' # TextMatch + | PHRASEMATCH'('Identifier',' StringLiteral (',' IntegerConstant)? ')' # PhraseMatch | expr POW expr # Power | op = (ADD | SUB | BNOT | NOT) expr # Unary // | '(' typeName ')' expr # Cast @@ -60,6 +61,7 @@ NE: '!='; LIKE: 'like' | 'LIKE'; EXISTS: 'exists' | 'EXISTS'; TEXTMATCH: 'text_match'|'TEXT_MATCH'; +PHRASEMATCH: 'phrase_match'|'PHRASE_MATCH'; ADD: '+'; SUB: '-'; diff --git a/internal/parser/planparserv2/generated/Plan.interp b/internal/parser/planparserv2/generated/Plan.interp index 1df02e866d170..d78a9ddacbfa2 100644 --- a/internal/parser/planparserv2/generated/Plan.interp +++ b/internal/parser/planparserv2/generated/Plan.interp @@ -16,6 +16,7 @@ null null null null +null '+' '-' '*' @@ -70,6 +71,7 @@ NE LIKE EXISTS TEXTMATCH +PHRASEMATCH ADD SUB MUL @@ -111,4 +113,4 @@ expr atn: -[4, 1, 51, 146, 2, 0, 7, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 5, 0, 21, 8, 0, 10, 0, 12, 0, 24, 9, 0, 1, 0, 3, 0, 27, 8, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 5, 0, 70, 8, 0, 10, 0, 12, 0, 73, 9, 0, 1, 0, 3, 0, 76, 8, 0, 3, 0, 78, 8, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 3, 0, 87, 8, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 3, 0, 103, 8, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 5, 0, 141, 8, 0, 10, 0, 12, 0, 144, 9, 0, 1, 0, 0, 1, 0, 1, 0, 0, 13, 1, 0, 46, 47, 2, 0, 17, 18, 32, 33, 2, 0, 36, 36, 39, 39, 2, 0, 37, 37, 40, 40, 2, 0, 38, 38, 41, 41, 2, 0, 46, 46, 49, 49, 1, 0, 19, 21, 1, 0, 17, 18, 1, 0, 23, 24, 1, 0, 8, 9, 1, 0, 10, 11, 1, 0, 8, 11, 1, 0, 12, 13, 184, 0, 86, 1, 0, 0, 0, 2, 3, 6, 0, -1, 0, 3, 87, 5, 44, 0, 0, 4, 87, 5, 45, 0, 0, 5, 87, 5, 43, 0, 0, 6, 87, 5, 48, 0, 0, 7, 87, 7, 0, 0, 0, 8, 87, 5, 49, 0, 0, 9, 10, 5, 6, 0, 0, 10, 11, 5, 46, 0, 0, 11, 87, 5, 7, 0, 0, 12, 13, 5, 1, 0, 0, 13, 14, 3, 0, 0, 0, 14, 15, 5, 2, 0, 0, 15, 87, 1, 0, 0, 0, 16, 17, 5, 3, 0, 0, 17, 22, 3, 0, 0, 0, 18, 19, 5, 4, 0, 0, 19, 21, 3, 0, 0, 0, 20, 18, 1, 0, 0, 0, 21, 24, 1, 0, 0, 0, 22, 20, 1, 0, 0, 0, 22, 23, 1, 0, 0, 0, 23, 26, 1, 0, 0, 0, 24, 22, 1, 0, 0, 0, 25, 27, 5, 4, 0, 0, 26, 25, 1, 0, 0, 0, 26, 27, 1, 0, 0, 0, 27, 28, 1, 0, 0, 0, 28, 29, 5, 5, 0, 0, 29, 87, 1, 0, 0, 0, 30, 87, 5, 35, 0, 0, 31, 32, 5, 16, 0, 0, 32, 33, 5, 1, 0, 0, 33, 34, 5, 46, 0, 0, 34, 35, 5, 4, 0, 0, 35, 36, 5, 48, 0, 0, 36, 87, 5, 2, 0, 0, 37, 38, 7, 1, 0, 0, 38, 87, 3, 0, 0, 22, 39, 40, 7, 2, 0, 0, 40, 41, 5, 1, 0, 0, 41, 42, 3, 0, 0, 0, 42, 43, 5, 4, 0, 0, 43, 44, 3, 0, 0, 0, 44, 45, 5, 2, 0, 0, 45, 87, 1, 0, 0, 0, 46, 47, 7, 3, 0, 0, 47, 48, 5, 1, 0, 0, 48, 49, 3, 0, 0, 0, 49, 50, 5, 4, 0, 0, 50, 51, 3, 0, 0, 0, 51, 52, 5, 2, 0, 0, 52, 87, 1, 0, 0, 0, 53, 54, 7, 4, 0, 0, 54, 55, 5, 1, 0, 0, 55, 56, 3, 0, 0, 0, 56, 57, 5, 4, 0, 0, 57, 58, 3, 0, 0, 0, 58, 59, 5, 2, 0, 0, 59, 87, 1, 0, 0, 0, 60, 61, 5, 42, 0, 0, 61, 62, 5, 1, 0, 0, 62, 63, 7, 5, 0, 0, 63, 87, 5, 2, 0, 0, 64, 65, 5, 46, 0, 0, 65, 77, 5, 1, 0, 0, 66, 71, 3, 0, 0, 0, 67, 68, 5, 4, 0, 0, 68, 70, 3, 0, 0, 0, 69, 67, 1, 0, 0, 0, 70, 73, 1, 0, 0, 0, 71, 69, 1, 0, 0, 0, 71, 72, 1, 0, 0, 0, 72, 75, 1, 0, 0, 0, 73, 71, 1, 0, 0, 0, 74, 76, 5, 4, 0, 0, 75, 74, 1, 0, 0, 0, 75, 76, 1, 0, 0, 0, 76, 78, 1, 0, 0, 0, 77, 66, 1, 0, 0, 0, 77, 78, 1, 0, 0, 0, 78, 79, 1, 0, 0, 0, 79, 87, 5, 2, 0, 0, 80, 81, 5, 46, 0, 0, 81, 87, 5, 30, 0, 0, 82, 83, 5, 46, 0, 0, 83, 87, 5, 31, 0, 0, 84, 85, 5, 15, 0, 0, 85, 87, 3, 0, 0, 1, 86, 2, 1, 0, 0, 0, 86, 4, 1, 0, 0, 0, 86, 5, 1, 0, 0, 0, 86, 6, 1, 0, 0, 0, 86, 7, 1, 0, 0, 0, 86, 8, 1, 0, 0, 0, 86, 9, 1, 0, 0, 0, 86, 12, 1, 0, 0, 0, 86, 16, 1, 0, 0, 0, 86, 30, 1, 0, 0, 0, 86, 31, 1, 0, 0, 0, 86, 37, 1, 0, 0, 0, 86, 39, 1, 0, 0, 0, 86, 46, 1, 0, 0, 0, 86, 53, 1, 0, 0, 0, 86, 60, 1, 0, 0, 0, 86, 64, 1, 0, 0, 0, 86, 80, 1, 0, 0, 0, 86, 82, 1, 0, 0, 0, 86, 84, 1, 0, 0, 0, 87, 142, 1, 0, 0, 0, 88, 89, 10, 23, 0, 0, 89, 90, 5, 22, 0, 0, 90, 141, 3, 0, 0, 24, 91, 92, 10, 21, 0, 0, 92, 93, 7, 6, 0, 0, 93, 141, 3, 0, 0, 22, 94, 95, 10, 20, 0, 0, 95, 96, 7, 7, 0, 0, 96, 141, 3, 0, 0, 21, 97, 98, 10, 19, 0, 0, 98, 99, 7, 8, 0, 0, 99, 141, 3, 0, 0, 20, 100, 102, 10, 18, 0, 0, 101, 103, 5, 33, 0, 0, 102, 101, 1, 0, 0, 0, 102, 103, 1, 0, 0, 0, 103, 104, 1, 0, 0, 0, 104, 105, 5, 34, 0, 0, 105, 141, 3, 0, 0, 19, 106, 107, 10, 12, 0, 0, 107, 108, 7, 9, 0, 0, 108, 109, 7, 5, 0, 0, 109, 110, 7, 9, 0, 0, 110, 141, 3, 0, 0, 13, 111, 112, 10, 11, 0, 0, 112, 113, 7, 10, 0, 0, 113, 114, 7, 5, 0, 0, 114, 115, 7, 10, 0, 0, 115, 141, 3, 0, 0, 12, 116, 117, 10, 10, 0, 0, 117, 118, 7, 11, 0, 0, 118, 141, 3, 0, 0, 11, 119, 120, 10, 9, 0, 0, 120, 121, 7, 12, 0, 0, 121, 141, 3, 0, 0, 10, 122, 123, 10, 8, 0, 0, 123, 124, 5, 25, 0, 0, 124, 141, 3, 0, 0, 9, 125, 126, 10, 7, 0, 0, 126, 127, 5, 27, 0, 0, 127, 141, 3, 0, 0, 8, 128, 129, 10, 6, 0, 0, 129, 130, 5, 26, 0, 0, 130, 141, 3, 0, 0, 7, 131, 132, 10, 5, 0, 0, 132, 133, 5, 28, 0, 0, 133, 141, 3, 0, 0, 6, 134, 135, 10, 4, 0, 0, 135, 136, 5, 29, 0, 0, 136, 141, 3, 0, 0, 5, 137, 138, 10, 25, 0, 0, 138, 139, 5, 14, 0, 0, 139, 141, 5, 48, 0, 0, 140, 88, 1, 0, 0, 0, 140, 91, 1, 0, 0, 0, 140, 94, 1, 0, 0, 0, 140, 97, 1, 0, 0, 0, 140, 100, 1, 0, 0, 0, 140, 106, 1, 0, 0, 0, 140, 111, 1, 0, 0, 0, 140, 116, 1, 0, 0, 0, 140, 119, 1, 0, 0, 0, 140, 122, 1, 0, 0, 0, 140, 125, 1, 0, 0, 0, 140, 128, 1, 0, 0, 0, 140, 131, 1, 0, 0, 0, 140, 134, 1, 0, 0, 0, 140, 137, 1, 0, 0, 0, 141, 144, 1, 0, 0, 0, 142, 140, 1, 0, 0, 0, 142, 143, 1, 0, 0, 0, 143, 1, 1, 0, 0, 0, 144, 142, 1, 0, 0, 0, 9, 22, 26, 71, 75, 77, 86, 102, 140, 142] \ No newline at end of file +[4, 1, 52, 156, 2, 0, 7, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 5, 0, 21, 8, 0, 10, 0, 12, 0, 24, 9, 0, 1, 0, 3, 0, 27, 8, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 3, 0, 45, 8, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 5, 0, 80, 8, 0, 10, 0, 12, 0, 83, 9, 0, 1, 0, 3, 0, 86, 8, 0, 3, 0, 88, 8, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 3, 0, 97, 8, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 3, 0, 113, 8, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 5, 0, 151, 8, 0, 10, 0, 12, 0, 154, 9, 0, 1, 0, 0, 1, 0, 1, 0, 0, 13, 1, 0, 47, 48, 2, 0, 18, 19, 33, 34, 2, 0, 37, 37, 40, 40, 2, 0, 38, 38, 41, 41, 2, 0, 39, 39, 42, 42, 2, 0, 47, 47, 50, 50, 1, 0, 20, 22, 1, 0, 18, 19, 1, 0, 24, 25, 1, 0, 8, 9, 1, 0, 10, 11, 1, 0, 8, 11, 1, 0, 12, 13, 196, 0, 96, 1, 0, 0, 0, 2, 3, 6, 0, -1, 0, 3, 97, 5, 45, 0, 0, 4, 97, 5, 46, 0, 0, 5, 97, 5, 44, 0, 0, 6, 97, 5, 49, 0, 0, 7, 97, 7, 0, 0, 0, 8, 97, 5, 50, 0, 0, 9, 10, 5, 6, 0, 0, 10, 11, 5, 47, 0, 0, 11, 97, 5, 7, 0, 0, 12, 13, 5, 1, 0, 0, 13, 14, 3, 0, 0, 0, 14, 15, 5, 2, 0, 0, 15, 97, 1, 0, 0, 0, 16, 17, 5, 3, 0, 0, 17, 22, 3, 0, 0, 0, 18, 19, 5, 4, 0, 0, 19, 21, 3, 0, 0, 0, 20, 18, 1, 0, 0, 0, 21, 24, 1, 0, 0, 0, 22, 20, 1, 0, 0, 0, 22, 23, 1, 0, 0, 0, 23, 26, 1, 0, 0, 0, 24, 22, 1, 0, 0, 0, 25, 27, 5, 4, 0, 0, 26, 25, 1, 0, 0, 0, 26, 27, 1, 0, 0, 0, 27, 28, 1, 0, 0, 0, 28, 29, 5, 5, 0, 0, 29, 97, 1, 0, 0, 0, 30, 97, 5, 36, 0, 0, 31, 32, 5, 16, 0, 0, 32, 33, 5, 1, 0, 0, 33, 34, 5, 47, 0, 0, 34, 35, 5, 4, 0, 0, 35, 36, 5, 49, 0, 0, 36, 97, 5, 2, 0, 0, 37, 38, 5, 17, 0, 0, 38, 39, 5, 1, 0, 0, 39, 40, 5, 47, 0, 0, 40, 41, 5, 4, 0, 0, 41, 44, 5, 49, 0, 0, 42, 43, 5, 4, 0, 0, 43, 45, 5, 45, 0, 0, 44, 42, 1, 0, 0, 0, 44, 45, 1, 0, 0, 0, 45, 46, 1, 0, 0, 0, 46, 97, 5, 2, 0, 0, 47, 48, 7, 1, 0, 0, 48, 97, 3, 0, 0, 22, 49, 50, 7, 2, 0, 0, 50, 51, 5, 1, 0, 0, 51, 52, 3, 0, 0, 0, 52, 53, 5, 4, 0, 0, 53, 54, 3, 0, 0, 0, 54, 55, 5, 2, 0, 0, 55, 97, 1, 0, 0, 0, 56, 57, 7, 3, 0, 0, 57, 58, 5, 1, 0, 0, 58, 59, 3, 0, 0, 0, 59, 60, 5, 4, 0, 0, 60, 61, 3, 0, 0, 0, 61, 62, 5, 2, 0, 0, 62, 97, 1, 0, 0, 0, 63, 64, 7, 4, 0, 0, 64, 65, 5, 1, 0, 0, 65, 66, 3, 0, 0, 0, 66, 67, 5, 4, 0, 0, 67, 68, 3, 0, 0, 0, 68, 69, 5, 2, 0, 0, 69, 97, 1, 0, 0, 0, 70, 71, 5, 43, 0, 0, 71, 72, 5, 1, 0, 0, 72, 73, 7, 5, 0, 0, 73, 97, 5, 2, 0, 0, 74, 75, 5, 47, 0, 0, 75, 87, 5, 1, 0, 0, 76, 81, 3, 0, 0, 0, 77, 78, 5, 4, 0, 0, 78, 80, 3, 0, 0, 0, 79, 77, 1, 0, 0, 0, 80, 83, 1, 0, 0, 0, 81, 79, 1, 0, 0, 0, 81, 82, 1, 0, 0, 0, 82, 85, 1, 0, 0, 0, 83, 81, 1, 0, 0, 0, 84, 86, 5, 4, 0, 0, 85, 84, 1, 0, 0, 0, 85, 86, 1, 0, 0, 0, 86, 88, 1, 0, 0, 0, 87, 76, 1, 0, 0, 0, 87, 88, 1, 0, 0, 0, 88, 89, 1, 0, 0, 0, 89, 97, 5, 2, 0, 0, 90, 91, 5, 47, 0, 0, 91, 97, 5, 31, 0, 0, 92, 93, 5, 47, 0, 0, 93, 97, 5, 32, 0, 0, 94, 95, 5, 15, 0, 0, 95, 97, 3, 0, 0, 1, 96, 2, 1, 0, 0, 0, 96, 4, 1, 0, 0, 0, 96, 5, 1, 0, 0, 0, 96, 6, 1, 0, 0, 0, 96, 7, 1, 0, 0, 0, 96, 8, 1, 0, 0, 0, 96, 9, 1, 0, 0, 0, 96, 12, 1, 0, 0, 0, 96, 16, 1, 0, 0, 0, 96, 30, 1, 0, 0, 0, 96, 31, 1, 0, 0, 0, 96, 37, 1, 0, 0, 0, 96, 47, 1, 0, 0, 0, 96, 49, 1, 0, 0, 0, 96, 56, 1, 0, 0, 0, 96, 63, 1, 0, 0, 0, 96, 70, 1, 0, 0, 0, 96, 74, 1, 0, 0, 0, 96, 90, 1, 0, 0, 0, 96, 92, 1, 0, 0, 0, 96, 94, 1, 0, 0, 0, 97, 152, 1, 0, 0, 0, 98, 99, 10, 23, 0, 0, 99, 100, 5, 23, 0, 0, 100, 151, 3, 0, 0, 24, 101, 102, 10, 21, 0, 0, 102, 103, 7, 6, 0, 0, 103, 151, 3, 0, 0, 22, 104, 105, 10, 20, 0, 0, 105, 106, 7, 7, 0, 0, 106, 151, 3, 0, 0, 21, 107, 108, 10, 19, 0, 0, 108, 109, 7, 8, 0, 0, 109, 151, 3, 0, 0, 20, 110, 112, 10, 18, 0, 0, 111, 113, 5, 34, 0, 0, 112, 111, 1, 0, 0, 0, 112, 113, 1, 0, 0, 0, 113, 114, 1, 0, 0, 0, 114, 115, 5, 35, 0, 0, 115, 151, 3, 0, 0, 19, 116, 117, 10, 12, 0, 0, 117, 118, 7, 9, 0, 0, 118, 119, 7, 5, 0, 0, 119, 120, 7, 9, 0, 0, 120, 151, 3, 0, 0, 13, 121, 122, 10, 11, 0, 0, 122, 123, 7, 10, 0, 0, 123, 124, 7, 5, 0, 0, 124, 125, 7, 10, 0, 0, 125, 151, 3, 0, 0, 12, 126, 127, 10, 10, 0, 0, 127, 128, 7, 11, 0, 0, 128, 151, 3, 0, 0, 11, 129, 130, 10, 9, 0, 0, 130, 131, 7, 12, 0, 0, 131, 151, 3, 0, 0, 10, 132, 133, 10, 8, 0, 0, 133, 134, 5, 26, 0, 0, 134, 151, 3, 0, 0, 9, 135, 136, 10, 7, 0, 0, 136, 137, 5, 28, 0, 0, 137, 151, 3, 0, 0, 8, 138, 139, 10, 6, 0, 0, 139, 140, 5, 27, 0, 0, 140, 151, 3, 0, 0, 7, 141, 142, 10, 5, 0, 0, 142, 143, 5, 29, 0, 0, 143, 151, 3, 0, 0, 6, 144, 145, 10, 4, 0, 0, 145, 146, 5, 30, 0, 0, 146, 151, 3, 0, 0, 5, 147, 148, 10, 26, 0, 0, 148, 149, 5, 14, 0, 0, 149, 151, 5, 49, 0, 0, 150, 98, 1, 0, 0, 0, 150, 101, 1, 0, 0, 0, 150, 104, 1, 0, 0, 0, 150, 107, 1, 0, 0, 0, 150, 110, 1, 0, 0, 0, 150, 116, 1, 0, 0, 0, 150, 121, 1, 0, 0, 0, 150, 126, 1, 0, 0, 0, 150, 129, 1, 0, 0, 0, 150, 132, 1, 0, 0, 0, 150, 135, 1, 0, 0, 0, 150, 138, 1, 0, 0, 0, 150, 141, 1, 0, 0, 0, 150, 144, 1, 0, 0, 0, 150, 147, 1, 0, 0, 0, 151, 154, 1, 0, 0, 0, 152, 150, 1, 0, 0, 0, 152, 153, 1, 0, 0, 0, 153, 1, 1, 0, 0, 0, 154, 152, 1, 0, 0, 0, 10, 22, 26, 44, 81, 85, 87, 96, 112, 150, 152] \ No newline at end of file diff --git a/internal/parser/planparserv2/generated/Plan.tokens b/internal/parser/planparserv2/generated/Plan.tokens index a198ec9a5db41..820be82be1273 100644 --- a/internal/parser/planparserv2/generated/Plan.tokens +++ b/internal/parser/planparserv2/generated/Plan.tokens @@ -14,41 +14,42 @@ NE=13 LIKE=14 EXISTS=15 TEXTMATCH=16 -ADD=17 -SUB=18 -MUL=19 -DIV=20 -MOD=21 -POW=22 -SHL=23 -SHR=24 -BAND=25 -BOR=26 -BXOR=27 -AND=28 -OR=29 -ISNULL=30 -ISNOTNULL=31 -BNOT=32 -NOT=33 -IN=34 -EmptyArray=35 -JSONContains=36 -JSONContainsAll=37 -JSONContainsAny=38 -ArrayContains=39 -ArrayContainsAll=40 -ArrayContainsAny=41 -ArrayLength=42 -BooleanConstant=43 -IntegerConstant=44 -FloatingConstant=45 -Identifier=46 -Meta=47 -StringLiteral=48 -JSONIdentifier=49 -Whitespace=50 -Newline=51 +PHRASEMATCH=17 +ADD=18 +SUB=19 +MUL=20 +DIV=21 +MOD=22 +POW=23 +SHL=24 +SHR=25 +BAND=26 +BOR=27 +BXOR=28 +AND=29 +OR=30 +ISNULL=31 +ISNOTNULL=32 +BNOT=33 +NOT=34 +IN=35 +EmptyArray=36 +JSONContains=37 +JSONContainsAll=38 +JSONContainsAny=39 +ArrayContains=40 +ArrayContainsAll=41 +ArrayContainsAny=42 +ArrayLength=43 +BooleanConstant=44 +IntegerConstant=45 +FloatingConstant=46 +Identifier=47 +Meta=48 +StringLiteral=49 +JSONIdentifier=50 +Whitespace=51 +Newline=52 '('=1 ')'=2 '['=3 @@ -62,16 +63,16 @@ Newline=51 '>='=11 '=='=12 '!='=13 -'+'=17 -'-'=18 -'*'=19 -'/'=20 -'%'=21 -'**'=22 -'<<'=23 -'>>'=24 -'&'=25 -'|'=26 -'^'=27 -'~'=32 -'$meta'=47 +'+'=18 +'-'=19 +'*'=20 +'/'=21 +'%'=22 +'**'=23 +'<<'=24 +'>>'=25 +'&'=26 +'|'=27 +'^'=28 +'~'=33 +'$meta'=48 diff --git a/internal/parser/planparserv2/generated/PlanLexer.interp b/internal/parser/planparserv2/generated/PlanLexer.interp index 7523be5820fe3..e8b2eff451661 100644 --- a/internal/parser/planparserv2/generated/PlanLexer.interp +++ b/internal/parser/planparserv2/generated/PlanLexer.interp @@ -16,6 +16,7 @@ null null null null +null '+' '-' '*' @@ -70,6 +71,7 @@ NE LIKE EXISTS TEXTMATCH +PHRASEMATCH ADD SUB MUL @@ -123,6 +125,7 @@ NE LIKE EXISTS TEXTMATCH +PHRASEMATCH ADD SUB MUL @@ -192,4 +195,4 @@ mode names: DEFAULT_MODE atn: -[4, 0, 51, 834, 6, -1, 2, 0, 7, 0, 2, 1, 7, 1, 2, 2, 7, 2, 2, 3, 7, 3, 2, 4, 7, 4, 2, 5, 7, 5, 2, 6, 7, 6, 2, 7, 7, 7, 2, 8, 7, 8, 2, 9, 7, 9, 2, 10, 7, 10, 2, 11, 7, 11, 2, 12, 7, 12, 2, 13, 7, 13, 2, 14, 7, 14, 2, 15, 7, 15, 2, 16, 7, 16, 2, 17, 7, 17, 2, 18, 7, 18, 2, 19, 7, 19, 2, 20, 7, 20, 2, 21, 7, 21, 2, 22, 7, 22, 2, 23, 7, 23, 2, 24, 7, 24, 2, 25, 7, 25, 2, 26, 7, 26, 2, 27, 7, 27, 2, 28, 7, 28, 2, 29, 7, 29, 2, 30, 7, 30, 2, 31, 7, 31, 2, 32, 7, 32, 2, 33, 7, 33, 2, 34, 7, 34, 2, 35, 7, 35, 2, 36, 7, 36, 2, 37, 7, 37, 2, 38, 7, 38, 2, 39, 7, 39, 2, 40, 7, 40, 2, 41, 7, 41, 2, 42, 7, 42, 2, 43, 7, 43, 2, 44, 7, 44, 2, 45, 7, 45, 2, 46, 7, 46, 2, 47, 7, 47, 2, 48, 7, 48, 2, 49, 7, 49, 2, 50, 7, 50, 2, 51, 7, 51, 2, 52, 7, 52, 2, 53, 7, 53, 2, 54, 7, 54, 2, 55, 7, 55, 2, 56, 7, 56, 2, 57, 7, 57, 2, 58, 7, 58, 2, 59, 7, 59, 2, 60, 7, 60, 2, 61, 7, 61, 2, 62, 7, 62, 2, 63, 7, 63, 2, 64, 7, 64, 2, 65, 7, 65, 2, 66, 7, 66, 2, 67, 7, 67, 2, 68, 7, 68, 2, 69, 7, 69, 2, 70, 7, 70, 2, 71, 7, 71, 2, 72, 7, 72, 2, 73, 7, 73, 2, 74, 7, 74, 2, 75, 7, 75, 1, 0, 1, 0, 1, 1, 1, 1, 1, 2, 1, 2, 1, 3, 1, 3, 1, 4, 1, 4, 1, 5, 1, 5, 1, 6, 1, 6, 1, 7, 1, 7, 1, 8, 1, 8, 1, 8, 1, 9, 1, 9, 1, 10, 1, 10, 1, 10, 1, 11, 1, 11, 1, 11, 1, 12, 1, 12, 1, 12, 1, 13, 1, 13, 1, 13, 1, 13, 1, 13, 1, 13, 1, 13, 1, 13, 3, 13, 192, 8, 13, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 3, 14, 206, 8, 14, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 3, 15, 228, 8, 15, 1, 16, 1, 16, 1, 17, 1, 17, 1, 18, 1, 18, 1, 19, 1, 19, 1, 20, 1, 20, 1, 21, 1, 21, 1, 21, 1, 22, 1, 22, 1, 22, 1, 23, 1, 23, 1, 23, 1, 24, 1, 24, 1, 25, 1, 25, 1, 26, 1, 26, 1, 27, 1, 27, 1, 27, 1, 27, 1, 27, 1, 27, 1, 27, 1, 27, 3, 27, 263, 8, 27, 1, 28, 1, 28, 1, 28, 1, 28, 1, 28, 1, 28, 3, 28, 271, 8, 28, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 3, 29, 287, 8, 29, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 3, 30, 311, 8, 30, 1, 31, 1, 31, 1, 32, 1, 32, 1, 32, 1, 32, 1, 32, 1, 32, 1, 32, 3, 32, 322, 8, 32, 1, 33, 1, 33, 1, 33, 1, 33, 3, 33, 328, 8, 33, 1, 34, 1, 34, 1, 34, 5, 34, 333, 8, 34, 10, 34, 12, 34, 336, 9, 34, 1, 34, 1, 34, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 3, 35, 366, 8, 35, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 3, 36, 402, 8, 36, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 3, 37, 438, 8, 37, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 3, 38, 468, 8, 38, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 3, 39, 506, 8, 39, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 3, 40, 544, 8, 40, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 3, 41, 570, 8, 41, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 3, 42, 599, 8, 42, 1, 43, 1, 43, 1, 43, 1, 43, 3, 43, 605, 8, 43, 1, 44, 1, 44, 3, 44, 609, 8, 44, 1, 45, 1, 45, 1, 45, 5, 45, 614, 8, 45, 10, 45, 12, 45, 617, 9, 45, 1, 46, 1, 46, 1, 46, 1, 46, 1, 46, 1, 46, 1, 47, 3, 47, 626, 8, 47, 1, 47, 1, 47, 3, 47, 630, 8, 47, 1, 47, 1, 47, 1, 47, 3, 47, 635, 8, 47, 1, 47, 3, 47, 638, 8, 47, 1, 48, 1, 48, 3, 48, 642, 8, 48, 1, 48, 1, 48, 1, 48, 3, 48, 647, 8, 48, 1, 48, 1, 48, 4, 48, 651, 8, 48, 11, 48, 12, 48, 652, 1, 49, 1, 49, 1, 49, 3, 49, 658, 8, 49, 1, 50, 4, 50, 661, 8, 50, 11, 50, 12, 50, 662, 1, 51, 4, 51, 666, 8, 51, 11, 51, 12, 51, 667, 1, 52, 1, 52, 1, 52, 1, 52, 1, 52, 1, 52, 1, 52, 3, 52, 677, 8, 52, 1, 53, 1, 53, 1, 53, 1, 53, 1, 53, 1, 53, 1, 53, 3, 53, 686, 8, 53, 1, 54, 1, 54, 1, 55, 1, 55, 1, 56, 1, 56, 1, 56, 4, 56, 695, 8, 56, 11, 56, 12, 56, 696, 1, 57, 1, 57, 5, 57, 701, 8, 57, 10, 57, 12, 57, 704, 9, 57, 1, 57, 3, 57, 707, 8, 57, 1, 58, 1, 58, 5, 58, 711, 8, 58, 10, 58, 12, 58, 714, 9, 58, 1, 59, 1, 59, 1, 59, 1, 59, 1, 60, 1, 60, 1, 61, 1, 61, 1, 62, 1, 62, 1, 63, 1, 63, 1, 63, 1, 63, 1, 63, 1, 64, 1, 64, 1, 64, 1, 64, 1, 64, 1, 64, 1, 64, 1, 64, 1, 64, 1, 64, 3, 64, 741, 8, 64, 1, 65, 1, 65, 3, 65, 745, 8, 65, 1, 65, 1, 65, 1, 65, 3, 65, 750, 8, 65, 1, 66, 1, 66, 1, 66, 1, 66, 3, 66, 756, 8, 66, 1, 66, 1, 66, 1, 67, 3, 67, 761, 8, 67, 1, 67, 1, 67, 1, 67, 1, 67, 1, 67, 3, 67, 768, 8, 67, 1, 68, 1, 68, 3, 68, 772, 8, 68, 1, 68, 1, 68, 1, 69, 4, 69, 777, 8, 69, 11, 69, 12, 69, 778, 1, 70, 3, 70, 782, 8, 70, 1, 70, 1, 70, 1, 70, 1, 70, 1, 70, 3, 70, 789, 8, 70, 1, 71, 4, 71, 792, 8, 71, 11, 71, 12, 71, 793, 1, 72, 1, 72, 3, 72, 798, 8, 72, 1, 72, 1, 72, 1, 73, 1, 73, 1, 73, 1, 73, 1, 73, 3, 73, 807, 8, 73, 1, 73, 3, 73, 810, 8, 73, 1, 73, 1, 73, 1, 73, 1, 73, 1, 73, 3, 73, 817, 8, 73, 1, 74, 4, 74, 820, 8, 74, 11, 74, 12, 74, 821, 1, 74, 1, 74, 1, 75, 1, 75, 3, 75, 828, 8, 75, 1, 75, 3, 75, 831, 8, 75, 1, 75, 1, 75, 0, 0, 76, 1, 1, 3, 2, 5, 3, 7, 4, 9, 5, 11, 6, 13, 7, 15, 8, 17, 9, 19, 10, 21, 11, 23, 12, 25, 13, 27, 14, 29, 15, 31, 16, 33, 17, 35, 18, 37, 19, 39, 20, 41, 21, 43, 22, 45, 23, 47, 24, 49, 25, 51, 26, 53, 27, 55, 28, 57, 29, 59, 30, 61, 31, 63, 32, 65, 33, 67, 34, 69, 35, 71, 36, 73, 37, 75, 38, 77, 39, 79, 40, 81, 41, 83, 42, 85, 43, 87, 44, 89, 45, 91, 46, 93, 47, 95, 48, 97, 49, 99, 0, 101, 0, 103, 0, 105, 0, 107, 0, 109, 0, 111, 0, 113, 0, 115, 0, 117, 0, 119, 0, 121, 0, 123, 0, 125, 0, 127, 0, 129, 0, 131, 0, 133, 0, 135, 0, 137, 0, 139, 0, 141, 0, 143, 0, 145, 0, 147, 0, 149, 50, 151, 51, 1, 0, 16, 3, 0, 76, 76, 85, 85, 117, 117, 4, 0, 10, 10, 13, 13, 34, 34, 92, 92, 4, 0, 10, 10, 13, 13, 39, 39, 92, 92, 3, 0, 65, 90, 95, 95, 97, 122, 1, 0, 48, 57, 2, 0, 66, 66, 98, 98, 1, 0, 48, 49, 2, 0, 88, 88, 120, 120, 1, 0, 49, 57, 1, 0, 48, 55, 3, 0, 48, 57, 65, 70, 97, 102, 2, 0, 69, 69, 101, 101, 2, 0, 43, 43, 45, 45, 2, 0, 80, 80, 112, 112, 10, 0, 34, 34, 39, 39, 63, 63, 92, 92, 97, 98, 102, 102, 110, 110, 114, 114, 116, 116, 118, 118, 2, 0, 9, 9, 32, 32, 880, 0, 1, 1, 0, 0, 0, 0, 3, 1, 0, 0, 0, 0, 5, 1, 0, 0, 0, 0, 7, 1, 0, 0, 0, 0, 9, 1, 0, 0, 0, 0, 11, 1, 0, 0, 0, 0, 13, 1, 0, 0, 0, 0, 15, 1, 0, 0, 0, 0, 17, 1, 0, 0, 0, 0, 19, 1, 0, 0, 0, 0, 21, 1, 0, 0, 0, 0, 23, 1, 0, 0, 0, 0, 25, 1, 0, 0, 0, 0, 27, 1, 0, 0, 0, 0, 29, 1, 0, 0, 0, 0, 31, 1, 0, 0, 0, 0, 33, 1, 0, 0, 0, 0, 35, 1, 0, 0, 0, 0, 37, 1, 0, 0, 0, 0, 39, 1, 0, 0, 0, 0, 41, 1, 0, 0, 0, 0, 43, 1, 0, 0, 0, 0, 45, 1, 0, 0, 0, 0, 47, 1, 0, 0, 0, 0, 49, 1, 0, 0, 0, 0, 51, 1, 0, 0, 0, 0, 53, 1, 0, 0, 0, 0, 55, 1, 0, 0, 0, 0, 57, 1, 0, 0, 0, 0, 59, 1, 0, 0, 0, 0, 61, 1, 0, 0, 0, 0, 63, 1, 0, 0, 0, 0, 65, 1, 0, 0, 0, 0, 67, 1, 0, 0, 0, 0, 69, 1, 0, 0, 0, 0, 71, 1, 0, 0, 0, 0, 73, 1, 0, 0, 0, 0, 75, 1, 0, 0, 0, 0, 77, 1, 0, 0, 0, 0, 79, 1, 0, 0, 0, 0, 81, 1, 0, 0, 0, 0, 83, 1, 0, 0, 0, 0, 85, 1, 0, 0, 0, 0, 87, 1, 0, 0, 0, 0, 89, 1, 0, 0, 0, 0, 91, 1, 0, 0, 0, 0, 93, 1, 0, 0, 0, 0, 95, 1, 0, 0, 0, 0, 97, 1, 0, 0, 0, 0, 149, 1, 0, 0, 0, 0, 151, 1, 0, 0, 0, 1, 153, 1, 0, 0, 0, 3, 155, 1, 0, 0, 0, 5, 157, 1, 0, 0, 0, 7, 159, 1, 0, 0, 0, 9, 161, 1, 0, 0, 0, 11, 163, 1, 0, 0, 0, 13, 165, 1, 0, 0, 0, 15, 167, 1, 0, 0, 0, 17, 169, 1, 0, 0, 0, 19, 172, 1, 0, 0, 0, 21, 174, 1, 0, 0, 0, 23, 177, 1, 0, 0, 0, 25, 180, 1, 0, 0, 0, 27, 191, 1, 0, 0, 0, 29, 205, 1, 0, 0, 0, 31, 227, 1, 0, 0, 0, 33, 229, 1, 0, 0, 0, 35, 231, 1, 0, 0, 0, 37, 233, 1, 0, 0, 0, 39, 235, 1, 0, 0, 0, 41, 237, 1, 0, 0, 0, 43, 239, 1, 0, 0, 0, 45, 242, 1, 0, 0, 0, 47, 245, 1, 0, 0, 0, 49, 248, 1, 0, 0, 0, 51, 250, 1, 0, 0, 0, 53, 252, 1, 0, 0, 0, 55, 262, 1, 0, 0, 0, 57, 270, 1, 0, 0, 0, 59, 286, 1, 0, 0, 0, 61, 310, 1, 0, 0, 0, 63, 312, 1, 0, 0, 0, 65, 321, 1, 0, 0, 0, 67, 327, 1, 0, 0, 0, 69, 329, 1, 0, 0, 0, 71, 365, 1, 0, 0, 0, 73, 401, 1, 0, 0, 0, 75, 437, 1, 0, 0, 0, 77, 467, 1, 0, 0, 0, 79, 505, 1, 0, 0, 0, 81, 543, 1, 0, 0, 0, 83, 569, 1, 0, 0, 0, 85, 598, 1, 0, 0, 0, 87, 604, 1, 0, 0, 0, 89, 608, 1, 0, 0, 0, 91, 610, 1, 0, 0, 0, 93, 618, 1, 0, 0, 0, 95, 625, 1, 0, 0, 0, 97, 641, 1, 0, 0, 0, 99, 657, 1, 0, 0, 0, 101, 660, 1, 0, 0, 0, 103, 665, 1, 0, 0, 0, 105, 676, 1, 0, 0, 0, 107, 685, 1, 0, 0, 0, 109, 687, 1, 0, 0, 0, 111, 689, 1, 0, 0, 0, 113, 691, 1, 0, 0, 0, 115, 706, 1, 0, 0, 0, 117, 708, 1, 0, 0, 0, 119, 715, 1, 0, 0, 0, 121, 719, 1, 0, 0, 0, 123, 721, 1, 0, 0, 0, 125, 723, 1, 0, 0, 0, 127, 725, 1, 0, 0, 0, 129, 740, 1, 0, 0, 0, 131, 749, 1, 0, 0, 0, 133, 751, 1, 0, 0, 0, 135, 767, 1, 0, 0, 0, 137, 769, 1, 0, 0, 0, 139, 776, 1, 0, 0, 0, 141, 788, 1, 0, 0, 0, 143, 791, 1, 0, 0, 0, 145, 795, 1, 0, 0, 0, 147, 816, 1, 0, 0, 0, 149, 819, 1, 0, 0, 0, 151, 830, 1, 0, 0, 0, 153, 154, 5, 40, 0, 0, 154, 2, 1, 0, 0, 0, 155, 156, 5, 41, 0, 0, 156, 4, 1, 0, 0, 0, 157, 158, 5, 91, 0, 0, 158, 6, 1, 0, 0, 0, 159, 160, 5, 44, 0, 0, 160, 8, 1, 0, 0, 0, 161, 162, 5, 93, 0, 0, 162, 10, 1, 0, 0, 0, 163, 164, 5, 123, 0, 0, 164, 12, 1, 0, 0, 0, 165, 166, 5, 125, 0, 0, 166, 14, 1, 0, 0, 0, 167, 168, 5, 60, 0, 0, 168, 16, 1, 0, 0, 0, 169, 170, 5, 60, 0, 0, 170, 171, 5, 61, 0, 0, 171, 18, 1, 0, 0, 0, 172, 173, 5, 62, 0, 0, 173, 20, 1, 0, 0, 0, 174, 175, 5, 62, 0, 0, 175, 176, 5, 61, 0, 0, 176, 22, 1, 0, 0, 0, 177, 178, 5, 61, 0, 0, 178, 179, 5, 61, 0, 0, 179, 24, 1, 0, 0, 0, 180, 181, 5, 33, 0, 0, 181, 182, 5, 61, 0, 0, 182, 26, 1, 0, 0, 0, 183, 184, 5, 108, 0, 0, 184, 185, 5, 105, 0, 0, 185, 186, 5, 107, 0, 0, 186, 192, 5, 101, 0, 0, 187, 188, 5, 76, 0, 0, 188, 189, 5, 73, 0, 0, 189, 190, 5, 75, 0, 0, 190, 192, 5, 69, 0, 0, 191, 183, 1, 0, 0, 0, 191, 187, 1, 0, 0, 0, 192, 28, 1, 0, 0, 0, 193, 194, 5, 101, 0, 0, 194, 195, 5, 120, 0, 0, 195, 196, 5, 105, 0, 0, 196, 197, 5, 115, 0, 0, 197, 198, 5, 116, 0, 0, 198, 206, 5, 115, 0, 0, 199, 200, 5, 69, 0, 0, 200, 201, 5, 88, 0, 0, 201, 202, 5, 73, 0, 0, 202, 203, 5, 83, 0, 0, 203, 204, 5, 84, 0, 0, 204, 206, 5, 83, 0, 0, 205, 193, 1, 0, 0, 0, 205, 199, 1, 0, 0, 0, 206, 30, 1, 0, 0, 0, 207, 208, 5, 116, 0, 0, 208, 209, 5, 101, 0, 0, 209, 210, 5, 120, 0, 0, 210, 211, 5, 116, 0, 0, 211, 212, 5, 95, 0, 0, 212, 213, 5, 109, 0, 0, 213, 214, 5, 97, 0, 0, 214, 215, 5, 116, 0, 0, 215, 216, 5, 99, 0, 0, 216, 228, 5, 104, 0, 0, 217, 218, 5, 84, 0, 0, 218, 219, 5, 69, 0, 0, 219, 220, 5, 88, 0, 0, 220, 221, 5, 84, 0, 0, 221, 222, 5, 95, 0, 0, 222, 223, 5, 77, 0, 0, 223, 224, 5, 65, 0, 0, 224, 225, 5, 84, 0, 0, 225, 226, 5, 67, 0, 0, 226, 228, 5, 72, 0, 0, 227, 207, 1, 0, 0, 0, 227, 217, 1, 0, 0, 0, 228, 32, 1, 0, 0, 0, 229, 230, 5, 43, 0, 0, 230, 34, 1, 0, 0, 0, 231, 232, 5, 45, 0, 0, 232, 36, 1, 0, 0, 0, 233, 234, 5, 42, 0, 0, 234, 38, 1, 0, 0, 0, 235, 236, 5, 47, 0, 0, 236, 40, 1, 0, 0, 0, 237, 238, 5, 37, 0, 0, 238, 42, 1, 0, 0, 0, 239, 240, 5, 42, 0, 0, 240, 241, 5, 42, 0, 0, 241, 44, 1, 0, 0, 0, 242, 243, 5, 60, 0, 0, 243, 244, 5, 60, 0, 0, 244, 46, 1, 0, 0, 0, 245, 246, 5, 62, 0, 0, 246, 247, 5, 62, 0, 0, 247, 48, 1, 0, 0, 0, 248, 249, 5, 38, 0, 0, 249, 50, 1, 0, 0, 0, 250, 251, 5, 124, 0, 0, 251, 52, 1, 0, 0, 0, 252, 253, 5, 94, 0, 0, 253, 54, 1, 0, 0, 0, 254, 255, 5, 38, 0, 0, 255, 263, 5, 38, 0, 0, 256, 257, 5, 97, 0, 0, 257, 258, 5, 110, 0, 0, 258, 263, 5, 100, 0, 0, 259, 260, 5, 65, 0, 0, 260, 261, 5, 78, 0, 0, 261, 263, 5, 68, 0, 0, 262, 254, 1, 0, 0, 0, 262, 256, 1, 0, 0, 0, 262, 259, 1, 0, 0, 0, 263, 56, 1, 0, 0, 0, 264, 265, 5, 124, 0, 0, 265, 271, 5, 124, 0, 0, 266, 267, 5, 111, 0, 0, 267, 271, 5, 114, 0, 0, 268, 269, 5, 79, 0, 0, 269, 271, 5, 82, 0, 0, 270, 264, 1, 0, 0, 0, 270, 266, 1, 0, 0, 0, 270, 268, 1, 0, 0, 0, 271, 58, 1, 0, 0, 0, 272, 273, 5, 105, 0, 0, 273, 274, 5, 115, 0, 0, 274, 275, 5, 32, 0, 0, 275, 276, 5, 110, 0, 0, 276, 277, 5, 117, 0, 0, 277, 278, 5, 108, 0, 0, 278, 287, 5, 108, 0, 0, 279, 280, 5, 73, 0, 0, 280, 281, 5, 83, 0, 0, 281, 282, 5, 32, 0, 0, 282, 283, 5, 78, 0, 0, 283, 284, 5, 85, 0, 0, 284, 285, 5, 76, 0, 0, 285, 287, 5, 76, 0, 0, 286, 272, 1, 0, 0, 0, 286, 279, 1, 0, 0, 0, 287, 60, 1, 0, 0, 0, 288, 289, 5, 105, 0, 0, 289, 290, 5, 115, 0, 0, 290, 291, 5, 32, 0, 0, 291, 292, 5, 110, 0, 0, 292, 293, 5, 111, 0, 0, 293, 294, 5, 116, 0, 0, 294, 295, 5, 32, 0, 0, 295, 296, 5, 110, 0, 0, 296, 297, 5, 117, 0, 0, 297, 298, 5, 108, 0, 0, 298, 311, 5, 108, 0, 0, 299, 300, 5, 73, 0, 0, 300, 301, 5, 83, 0, 0, 301, 302, 5, 32, 0, 0, 302, 303, 5, 78, 0, 0, 303, 304, 5, 79, 0, 0, 304, 305, 5, 84, 0, 0, 305, 306, 5, 32, 0, 0, 306, 307, 5, 78, 0, 0, 307, 308, 5, 85, 0, 0, 308, 309, 5, 76, 0, 0, 309, 311, 5, 76, 0, 0, 310, 288, 1, 0, 0, 0, 310, 299, 1, 0, 0, 0, 311, 62, 1, 0, 0, 0, 312, 313, 5, 126, 0, 0, 313, 64, 1, 0, 0, 0, 314, 322, 5, 33, 0, 0, 315, 316, 5, 110, 0, 0, 316, 317, 5, 111, 0, 0, 317, 322, 5, 116, 0, 0, 318, 319, 5, 78, 0, 0, 319, 320, 5, 79, 0, 0, 320, 322, 5, 84, 0, 0, 321, 314, 1, 0, 0, 0, 321, 315, 1, 0, 0, 0, 321, 318, 1, 0, 0, 0, 322, 66, 1, 0, 0, 0, 323, 324, 5, 105, 0, 0, 324, 328, 5, 110, 0, 0, 325, 326, 5, 73, 0, 0, 326, 328, 5, 78, 0, 0, 327, 323, 1, 0, 0, 0, 327, 325, 1, 0, 0, 0, 328, 68, 1, 0, 0, 0, 329, 334, 5, 91, 0, 0, 330, 333, 3, 149, 74, 0, 331, 333, 3, 151, 75, 0, 332, 330, 1, 0, 0, 0, 332, 331, 1, 0, 0, 0, 333, 336, 1, 0, 0, 0, 334, 332, 1, 0, 0, 0, 334, 335, 1, 0, 0, 0, 335, 337, 1, 0, 0, 0, 336, 334, 1, 0, 0, 0, 337, 338, 5, 93, 0, 0, 338, 70, 1, 0, 0, 0, 339, 340, 5, 106, 0, 0, 340, 341, 5, 115, 0, 0, 341, 342, 5, 111, 0, 0, 342, 343, 5, 110, 0, 0, 343, 344, 5, 95, 0, 0, 344, 345, 5, 99, 0, 0, 345, 346, 5, 111, 0, 0, 346, 347, 5, 110, 0, 0, 347, 348, 5, 116, 0, 0, 348, 349, 5, 97, 0, 0, 349, 350, 5, 105, 0, 0, 350, 351, 5, 110, 0, 0, 351, 366, 5, 115, 0, 0, 352, 353, 5, 74, 0, 0, 353, 354, 5, 83, 0, 0, 354, 355, 5, 79, 0, 0, 355, 356, 5, 78, 0, 0, 356, 357, 5, 95, 0, 0, 357, 358, 5, 67, 0, 0, 358, 359, 5, 79, 0, 0, 359, 360, 5, 78, 0, 0, 360, 361, 5, 84, 0, 0, 361, 362, 5, 65, 0, 0, 362, 363, 5, 73, 0, 0, 363, 364, 5, 78, 0, 0, 364, 366, 5, 83, 0, 0, 365, 339, 1, 0, 0, 0, 365, 352, 1, 0, 0, 0, 366, 72, 1, 0, 0, 0, 367, 368, 5, 106, 0, 0, 368, 369, 5, 115, 0, 0, 369, 370, 5, 111, 0, 0, 370, 371, 5, 110, 0, 0, 371, 372, 5, 95, 0, 0, 372, 373, 5, 99, 0, 0, 373, 374, 5, 111, 0, 0, 374, 375, 5, 110, 0, 0, 375, 376, 5, 116, 0, 0, 376, 377, 5, 97, 0, 0, 377, 378, 5, 105, 0, 0, 378, 379, 5, 110, 0, 0, 379, 380, 5, 115, 0, 0, 380, 381, 5, 95, 0, 0, 381, 382, 5, 97, 0, 0, 382, 383, 5, 108, 0, 0, 383, 402, 5, 108, 0, 0, 384, 385, 5, 74, 0, 0, 385, 386, 5, 83, 0, 0, 386, 387, 5, 79, 0, 0, 387, 388, 5, 78, 0, 0, 388, 389, 5, 95, 0, 0, 389, 390, 5, 67, 0, 0, 390, 391, 5, 79, 0, 0, 391, 392, 5, 78, 0, 0, 392, 393, 5, 84, 0, 0, 393, 394, 5, 65, 0, 0, 394, 395, 5, 73, 0, 0, 395, 396, 5, 78, 0, 0, 396, 397, 5, 83, 0, 0, 397, 398, 5, 95, 0, 0, 398, 399, 5, 65, 0, 0, 399, 400, 5, 76, 0, 0, 400, 402, 5, 76, 0, 0, 401, 367, 1, 0, 0, 0, 401, 384, 1, 0, 0, 0, 402, 74, 1, 0, 0, 0, 403, 404, 5, 106, 0, 0, 404, 405, 5, 115, 0, 0, 405, 406, 5, 111, 0, 0, 406, 407, 5, 110, 0, 0, 407, 408, 5, 95, 0, 0, 408, 409, 5, 99, 0, 0, 409, 410, 5, 111, 0, 0, 410, 411, 5, 110, 0, 0, 411, 412, 5, 116, 0, 0, 412, 413, 5, 97, 0, 0, 413, 414, 5, 105, 0, 0, 414, 415, 5, 110, 0, 0, 415, 416, 5, 115, 0, 0, 416, 417, 5, 95, 0, 0, 417, 418, 5, 97, 0, 0, 418, 419, 5, 110, 0, 0, 419, 438, 5, 121, 0, 0, 420, 421, 5, 74, 0, 0, 421, 422, 5, 83, 0, 0, 422, 423, 5, 79, 0, 0, 423, 424, 5, 78, 0, 0, 424, 425, 5, 95, 0, 0, 425, 426, 5, 67, 0, 0, 426, 427, 5, 79, 0, 0, 427, 428, 5, 78, 0, 0, 428, 429, 5, 84, 0, 0, 429, 430, 5, 65, 0, 0, 430, 431, 5, 73, 0, 0, 431, 432, 5, 78, 0, 0, 432, 433, 5, 83, 0, 0, 433, 434, 5, 95, 0, 0, 434, 435, 5, 65, 0, 0, 435, 436, 5, 78, 0, 0, 436, 438, 5, 89, 0, 0, 437, 403, 1, 0, 0, 0, 437, 420, 1, 0, 0, 0, 438, 76, 1, 0, 0, 0, 439, 440, 5, 97, 0, 0, 440, 441, 5, 114, 0, 0, 441, 442, 5, 114, 0, 0, 442, 443, 5, 97, 0, 0, 443, 444, 5, 121, 0, 0, 444, 445, 5, 95, 0, 0, 445, 446, 5, 99, 0, 0, 446, 447, 5, 111, 0, 0, 447, 448, 5, 110, 0, 0, 448, 449, 5, 116, 0, 0, 449, 450, 5, 97, 0, 0, 450, 451, 5, 105, 0, 0, 451, 452, 5, 110, 0, 0, 452, 468, 5, 115, 0, 0, 453, 454, 5, 65, 0, 0, 454, 455, 5, 82, 0, 0, 455, 456, 5, 82, 0, 0, 456, 457, 5, 65, 0, 0, 457, 458, 5, 89, 0, 0, 458, 459, 5, 95, 0, 0, 459, 460, 5, 67, 0, 0, 460, 461, 5, 79, 0, 0, 461, 462, 5, 78, 0, 0, 462, 463, 5, 84, 0, 0, 463, 464, 5, 65, 0, 0, 464, 465, 5, 73, 0, 0, 465, 466, 5, 78, 0, 0, 466, 468, 5, 83, 0, 0, 467, 439, 1, 0, 0, 0, 467, 453, 1, 0, 0, 0, 468, 78, 1, 0, 0, 0, 469, 470, 5, 97, 0, 0, 470, 471, 5, 114, 0, 0, 471, 472, 5, 114, 0, 0, 472, 473, 5, 97, 0, 0, 473, 474, 5, 121, 0, 0, 474, 475, 5, 95, 0, 0, 475, 476, 5, 99, 0, 0, 476, 477, 5, 111, 0, 0, 477, 478, 5, 110, 0, 0, 478, 479, 5, 116, 0, 0, 479, 480, 5, 97, 0, 0, 480, 481, 5, 105, 0, 0, 481, 482, 5, 110, 0, 0, 482, 483, 5, 115, 0, 0, 483, 484, 5, 95, 0, 0, 484, 485, 5, 97, 0, 0, 485, 486, 5, 108, 0, 0, 486, 506, 5, 108, 0, 0, 487, 488, 5, 65, 0, 0, 488, 489, 5, 82, 0, 0, 489, 490, 5, 82, 0, 0, 490, 491, 5, 65, 0, 0, 491, 492, 5, 89, 0, 0, 492, 493, 5, 95, 0, 0, 493, 494, 5, 67, 0, 0, 494, 495, 5, 79, 0, 0, 495, 496, 5, 78, 0, 0, 496, 497, 5, 84, 0, 0, 497, 498, 5, 65, 0, 0, 498, 499, 5, 73, 0, 0, 499, 500, 5, 78, 0, 0, 500, 501, 5, 83, 0, 0, 501, 502, 5, 95, 0, 0, 502, 503, 5, 65, 0, 0, 503, 504, 5, 76, 0, 0, 504, 506, 5, 76, 0, 0, 505, 469, 1, 0, 0, 0, 505, 487, 1, 0, 0, 0, 506, 80, 1, 0, 0, 0, 507, 508, 5, 97, 0, 0, 508, 509, 5, 114, 0, 0, 509, 510, 5, 114, 0, 0, 510, 511, 5, 97, 0, 0, 511, 512, 5, 121, 0, 0, 512, 513, 5, 95, 0, 0, 513, 514, 5, 99, 0, 0, 514, 515, 5, 111, 0, 0, 515, 516, 5, 110, 0, 0, 516, 517, 5, 116, 0, 0, 517, 518, 5, 97, 0, 0, 518, 519, 5, 105, 0, 0, 519, 520, 5, 110, 0, 0, 520, 521, 5, 115, 0, 0, 521, 522, 5, 95, 0, 0, 522, 523, 5, 97, 0, 0, 523, 524, 5, 110, 0, 0, 524, 544, 5, 121, 0, 0, 525, 526, 5, 65, 0, 0, 526, 527, 5, 82, 0, 0, 527, 528, 5, 82, 0, 0, 528, 529, 5, 65, 0, 0, 529, 530, 5, 89, 0, 0, 530, 531, 5, 95, 0, 0, 531, 532, 5, 67, 0, 0, 532, 533, 5, 79, 0, 0, 533, 534, 5, 78, 0, 0, 534, 535, 5, 84, 0, 0, 535, 536, 5, 65, 0, 0, 536, 537, 5, 73, 0, 0, 537, 538, 5, 78, 0, 0, 538, 539, 5, 83, 0, 0, 539, 540, 5, 95, 0, 0, 540, 541, 5, 65, 0, 0, 541, 542, 5, 78, 0, 0, 542, 544, 5, 89, 0, 0, 543, 507, 1, 0, 0, 0, 543, 525, 1, 0, 0, 0, 544, 82, 1, 0, 0, 0, 545, 546, 5, 97, 0, 0, 546, 547, 5, 114, 0, 0, 547, 548, 5, 114, 0, 0, 548, 549, 5, 97, 0, 0, 549, 550, 5, 121, 0, 0, 550, 551, 5, 95, 0, 0, 551, 552, 5, 108, 0, 0, 552, 553, 5, 101, 0, 0, 553, 554, 5, 110, 0, 0, 554, 555, 5, 103, 0, 0, 555, 556, 5, 116, 0, 0, 556, 570, 5, 104, 0, 0, 557, 558, 5, 65, 0, 0, 558, 559, 5, 82, 0, 0, 559, 560, 5, 82, 0, 0, 560, 561, 5, 65, 0, 0, 561, 562, 5, 89, 0, 0, 562, 563, 5, 95, 0, 0, 563, 564, 5, 76, 0, 0, 564, 565, 5, 69, 0, 0, 565, 566, 5, 78, 0, 0, 566, 567, 5, 71, 0, 0, 567, 568, 5, 84, 0, 0, 568, 570, 5, 72, 0, 0, 569, 545, 1, 0, 0, 0, 569, 557, 1, 0, 0, 0, 570, 84, 1, 0, 0, 0, 571, 572, 5, 116, 0, 0, 572, 573, 5, 114, 0, 0, 573, 574, 5, 117, 0, 0, 574, 599, 5, 101, 0, 0, 575, 576, 5, 84, 0, 0, 576, 577, 5, 114, 0, 0, 577, 578, 5, 117, 0, 0, 578, 599, 5, 101, 0, 0, 579, 580, 5, 84, 0, 0, 580, 581, 5, 82, 0, 0, 581, 582, 5, 85, 0, 0, 582, 599, 5, 69, 0, 0, 583, 584, 5, 102, 0, 0, 584, 585, 5, 97, 0, 0, 585, 586, 5, 108, 0, 0, 586, 587, 5, 115, 0, 0, 587, 599, 5, 101, 0, 0, 588, 589, 5, 70, 0, 0, 589, 590, 5, 97, 0, 0, 590, 591, 5, 108, 0, 0, 591, 592, 5, 115, 0, 0, 592, 599, 5, 101, 0, 0, 593, 594, 5, 70, 0, 0, 594, 595, 5, 65, 0, 0, 595, 596, 5, 76, 0, 0, 596, 597, 5, 83, 0, 0, 597, 599, 5, 69, 0, 0, 598, 571, 1, 0, 0, 0, 598, 575, 1, 0, 0, 0, 598, 579, 1, 0, 0, 0, 598, 583, 1, 0, 0, 0, 598, 588, 1, 0, 0, 0, 598, 593, 1, 0, 0, 0, 599, 86, 1, 0, 0, 0, 600, 605, 3, 115, 57, 0, 601, 605, 3, 117, 58, 0, 602, 605, 3, 119, 59, 0, 603, 605, 3, 113, 56, 0, 604, 600, 1, 0, 0, 0, 604, 601, 1, 0, 0, 0, 604, 602, 1, 0, 0, 0, 604, 603, 1, 0, 0, 0, 605, 88, 1, 0, 0, 0, 606, 609, 3, 131, 65, 0, 607, 609, 3, 133, 66, 0, 608, 606, 1, 0, 0, 0, 608, 607, 1, 0, 0, 0, 609, 90, 1, 0, 0, 0, 610, 615, 3, 109, 54, 0, 611, 614, 3, 109, 54, 0, 612, 614, 3, 111, 55, 0, 613, 611, 1, 0, 0, 0, 613, 612, 1, 0, 0, 0, 614, 617, 1, 0, 0, 0, 615, 613, 1, 0, 0, 0, 615, 616, 1, 0, 0, 0, 616, 92, 1, 0, 0, 0, 617, 615, 1, 0, 0, 0, 618, 619, 5, 36, 0, 0, 619, 620, 5, 109, 0, 0, 620, 621, 5, 101, 0, 0, 621, 622, 5, 116, 0, 0, 622, 623, 5, 97, 0, 0, 623, 94, 1, 0, 0, 0, 624, 626, 3, 99, 49, 0, 625, 624, 1, 0, 0, 0, 625, 626, 1, 0, 0, 0, 626, 637, 1, 0, 0, 0, 627, 629, 5, 34, 0, 0, 628, 630, 3, 101, 50, 0, 629, 628, 1, 0, 0, 0, 629, 630, 1, 0, 0, 0, 630, 631, 1, 0, 0, 0, 631, 638, 5, 34, 0, 0, 632, 634, 5, 39, 0, 0, 633, 635, 3, 103, 51, 0, 634, 633, 1, 0, 0, 0, 634, 635, 1, 0, 0, 0, 635, 636, 1, 0, 0, 0, 636, 638, 5, 39, 0, 0, 637, 627, 1, 0, 0, 0, 637, 632, 1, 0, 0, 0, 638, 96, 1, 0, 0, 0, 639, 642, 3, 91, 45, 0, 640, 642, 3, 93, 46, 0, 641, 639, 1, 0, 0, 0, 641, 640, 1, 0, 0, 0, 642, 650, 1, 0, 0, 0, 643, 646, 5, 91, 0, 0, 644, 647, 3, 95, 47, 0, 645, 647, 3, 115, 57, 0, 646, 644, 1, 0, 0, 0, 646, 645, 1, 0, 0, 0, 647, 648, 1, 0, 0, 0, 648, 649, 5, 93, 0, 0, 649, 651, 1, 0, 0, 0, 650, 643, 1, 0, 0, 0, 651, 652, 1, 0, 0, 0, 652, 650, 1, 0, 0, 0, 652, 653, 1, 0, 0, 0, 653, 98, 1, 0, 0, 0, 654, 655, 5, 117, 0, 0, 655, 658, 5, 56, 0, 0, 656, 658, 7, 0, 0, 0, 657, 654, 1, 0, 0, 0, 657, 656, 1, 0, 0, 0, 658, 100, 1, 0, 0, 0, 659, 661, 3, 105, 52, 0, 660, 659, 1, 0, 0, 0, 661, 662, 1, 0, 0, 0, 662, 660, 1, 0, 0, 0, 662, 663, 1, 0, 0, 0, 663, 102, 1, 0, 0, 0, 664, 666, 3, 107, 53, 0, 665, 664, 1, 0, 0, 0, 666, 667, 1, 0, 0, 0, 667, 665, 1, 0, 0, 0, 667, 668, 1, 0, 0, 0, 668, 104, 1, 0, 0, 0, 669, 677, 8, 1, 0, 0, 670, 677, 3, 147, 73, 0, 671, 672, 5, 92, 0, 0, 672, 677, 5, 10, 0, 0, 673, 674, 5, 92, 0, 0, 674, 675, 5, 13, 0, 0, 675, 677, 5, 10, 0, 0, 676, 669, 1, 0, 0, 0, 676, 670, 1, 0, 0, 0, 676, 671, 1, 0, 0, 0, 676, 673, 1, 0, 0, 0, 677, 106, 1, 0, 0, 0, 678, 686, 8, 2, 0, 0, 679, 686, 3, 147, 73, 0, 680, 681, 5, 92, 0, 0, 681, 686, 5, 10, 0, 0, 682, 683, 5, 92, 0, 0, 683, 684, 5, 13, 0, 0, 684, 686, 5, 10, 0, 0, 685, 678, 1, 0, 0, 0, 685, 679, 1, 0, 0, 0, 685, 680, 1, 0, 0, 0, 685, 682, 1, 0, 0, 0, 686, 108, 1, 0, 0, 0, 687, 688, 7, 3, 0, 0, 688, 110, 1, 0, 0, 0, 689, 690, 7, 4, 0, 0, 690, 112, 1, 0, 0, 0, 691, 692, 5, 48, 0, 0, 692, 694, 7, 5, 0, 0, 693, 695, 7, 6, 0, 0, 694, 693, 1, 0, 0, 0, 695, 696, 1, 0, 0, 0, 696, 694, 1, 0, 0, 0, 696, 697, 1, 0, 0, 0, 697, 114, 1, 0, 0, 0, 698, 702, 3, 121, 60, 0, 699, 701, 3, 111, 55, 0, 700, 699, 1, 0, 0, 0, 701, 704, 1, 0, 0, 0, 702, 700, 1, 0, 0, 0, 702, 703, 1, 0, 0, 0, 703, 707, 1, 0, 0, 0, 704, 702, 1, 0, 0, 0, 705, 707, 5, 48, 0, 0, 706, 698, 1, 0, 0, 0, 706, 705, 1, 0, 0, 0, 707, 116, 1, 0, 0, 0, 708, 712, 5, 48, 0, 0, 709, 711, 3, 123, 61, 0, 710, 709, 1, 0, 0, 0, 711, 714, 1, 0, 0, 0, 712, 710, 1, 0, 0, 0, 712, 713, 1, 0, 0, 0, 713, 118, 1, 0, 0, 0, 714, 712, 1, 0, 0, 0, 715, 716, 5, 48, 0, 0, 716, 717, 7, 7, 0, 0, 717, 718, 3, 143, 71, 0, 718, 120, 1, 0, 0, 0, 719, 720, 7, 8, 0, 0, 720, 122, 1, 0, 0, 0, 721, 722, 7, 9, 0, 0, 722, 124, 1, 0, 0, 0, 723, 724, 7, 10, 0, 0, 724, 126, 1, 0, 0, 0, 725, 726, 3, 125, 62, 0, 726, 727, 3, 125, 62, 0, 727, 728, 3, 125, 62, 0, 728, 729, 3, 125, 62, 0, 729, 128, 1, 0, 0, 0, 730, 731, 5, 92, 0, 0, 731, 732, 5, 117, 0, 0, 732, 733, 1, 0, 0, 0, 733, 741, 3, 127, 63, 0, 734, 735, 5, 92, 0, 0, 735, 736, 5, 85, 0, 0, 736, 737, 1, 0, 0, 0, 737, 738, 3, 127, 63, 0, 738, 739, 3, 127, 63, 0, 739, 741, 1, 0, 0, 0, 740, 730, 1, 0, 0, 0, 740, 734, 1, 0, 0, 0, 741, 130, 1, 0, 0, 0, 742, 744, 3, 135, 67, 0, 743, 745, 3, 137, 68, 0, 744, 743, 1, 0, 0, 0, 744, 745, 1, 0, 0, 0, 745, 750, 1, 0, 0, 0, 746, 747, 3, 139, 69, 0, 747, 748, 3, 137, 68, 0, 748, 750, 1, 0, 0, 0, 749, 742, 1, 0, 0, 0, 749, 746, 1, 0, 0, 0, 750, 132, 1, 0, 0, 0, 751, 752, 5, 48, 0, 0, 752, 755, 7, 7, 0, 0, 753, 756, 3, 141, 70, 0, 754, 756, 3, 143, 71, 0, 755, 753, 1, 0, 0, 0, 755, 754, 1, 0, 0, 0, 756, 757, 1, 0, 0, 0, 757, 758, 3, 145, 72, 0, 758, 134, 1, 0, 0, 0, 759, 761, 3, 139, 69, 0, 760, 759, 1, 0, 0, 0, 760, 761, 1, 0, 0, 0, 761, 762, 1, 0, 0, 0, 762, 763, 5, 46, 0, 0, 763, 768, 3, 139, 69, 0, 764, 765, 3, 139, 69, 0, 765, 766, 5, 46, 0, 0, 766, 768, 1, 0, 0, 0, 767, 760, 1, 0, 0, 0, 767, 764, 1, 0, 0, 0, 768, 136, 1, 0, 0, 0, 769, 771, 7, 11, 0, 0, 770, 772, 7, 12, 0, 0, 771, 770, 1, 0, 0, 0, 771, 772, 1, 0, 0, 0, 772, 773, 1, 0, 0, 0, 773, 774, 3, 139, 69, 0, 774, 138, 1, 0, 0, 0, 775, 777, 3, 111, 55, 0, 776, 775, 1, 0, 0, 0, 777, 778, 1, 0, 0, 0, 778, 776, 1, 0, 0, 0, 778, 779, 1, 0, 0, 0, 779, 140, 1, 0, 0, 0, 780, 782, 3, 143, 71, 0, 781, 780, 1, 0, 0, 0, 781, 782, 1, 0, 0, 0, 782, 783, 1, 0, 0, 0, 783, 784, 5, 46, 0, 0, 784, 789, 3, 143, 71, 0, 785, 786, 3, 143, 71, 0, 786, 787, 5, 46, 0, 0, 787, 789, 1, 0, 0, 0, 788, 781, 1, 0, 0, 0, 788, 785, 1, 0, 0, 0, 789, 142, 1, 0, 0, 0, 790, 792, 3, 125, 62, 0, 791, 790, 1, 0, 0, 0, 792, 793, 1, 0, 0, 0, 793, 791, 1, 0, 0, 0, 793, 794, 1, 0, 0, 0, 794, 144, 1, 0, 0, 0, 795, 797, 7, 13, 0, 0, 796, 798, 7, 12, 0, 0, 797, 796, 1, 0, 0, 0, 797, 798, 1, 0, 0, 0, 798, 799, 1, 0, 0, 0, 799, 800, 3, 139, 69, 0, 800, 146, 1, 0, 0, 0, 801, 802, 5, 92, 0, 0, 802, 817, 7, 14, 0, 0, 803, 804, 5, 92, 0, 0, 804, 806, 3, 123, 61, 0, 805, 807, 3, 123, 61, 0, 806, 805, 1, 0, 0, 0, 806, 807, 1, 0, 0, 0, 807, 809, 1, 0, 0, 0, 808, 810, 3, 123, 61, 0, 809, 808, 1, 0, 0, 0, 809, 810, 1, 0, 0, 0, 810, 817, 1, 0, 0, 0, 811, 812, 5, 92, 0, 0, 812, 813, 5, 120, 0, 0, 813, 814, 1, 0, 0, 0, 814, 817, 3, 143, 71, 0, 815, 817, 3, 129, 64, 0, 816, 801, 1, 0, 0, 0, 816, 803, 1, 0, 0, 0, 816, 811, 1, 0, 0, 0, 816, 815, 1, 0, 0, 0, 817, 148, 1, 0, 0, 0, 818, 820, 7, 15, 0, 0, 819, 818, 1, 0, 0, 0, 820, 821, 1, 0, 0, 0, 821, 819, 1, 0, 0, 0, 821, 822, 1, 0, 0, 0, 822, 823, 1, 0, 0, 0, 823, 824, 6, 74, 0, 0, 824, 150, 1, 0, 0, 0, 825, 827, 5, 13, 0, 0, 826, 828, 5, 10, 0, 0, 827, 826, 1, 0, 0, 0, 827, 828, 1, 0, 0, 0, 828, 831, 1, 0, 0, 0, 829, 831, 5, 10, 0, 0, 830, 825, 1, 0, 0, 0, 830, 829, 1, 0, 0, 0, 831, 832, 1, 0, 0, 0, 832, 833, 6, 75, 0, 0, 833, 152, 1, 0, 0, 0, 58, 0, 191, 205, 227, 262, 270, 286, 310, 321, 327, 332, 334, 365, 401, 437, 467, 505, 543, 569, 598, 604, 608, 613, 615, 625, 629, 634, 637, 641, 646, 652, 657, 662, 667, 676, 685, 696, 702, 706, 712, 740, 744, 749, 755, 760, 767, 771, 778, 781, 788, 793, 797, 806, 809, 816, 821, 827, 830, 1, 6, 0, 0] \ No newline at end of file +[4, 0, 52, 862, 6, -1, 2, 0, 7, 0, 2, 1, 7, 1, 2, 2, 7, 2, 2, 3, 7, 3, 2, 4, 7, 4, 2, 5, 7, 5, 2, 6, 7, 6, 2, 7, 7, 7, 2, 8, 7, 8, 2, 9, 7, 9, 2, 10, 7, 10, 2, 11, 7, 11, 2, 12, 7, 12, 2, 13, 7, 13, 2, 14, 7, 14, 2, 15, 7, 15, 2, 16, 7, 16, 2, 17, 7, 17, 2, 18, 7, 18, 2, 19, 7, 19, 2, 20, 7, 20, 2, 21, 7, 21, 2, 22, 7, 22, 2, 23, 7, 23, 2, 24, 7, 24, 2, 25, 7, 25, 2, 26, 7, 26, 2, 27, 7, 27, 2, 28, 7, 28, 2, 29, 7, 29, 2, 30, 7, 30, 2, 31, 7, 31, 2, 32, 7, 32, 2, 33, 7, 33, 2, 34, 7, 34, 2, 35, 7, 35, 2, 36, 7, 36, 2, 37, 7, 37, 2, 38, 7, 38, 2, 39, 7, 39, 2, 40, 7, 40, 2, 41, 7, 41, 2, 42, 7, 42, 2, 43, 7, 43, 2, 44, 7, 44, 2, 45, 7, 45, 2, 46, 7, 46, 2, 47, 7, 47, 2, 48, 7, 48, 2, 49, 7, 49, 2, 50, 7, 50, 2, 51, 7, 51, 2, 52, 7, 52, 2, 53, 7, 53, 2, 54, 7, 54, 2, 55, 7, 55, 2, 56, 7, 56, 2, 57, 7, 57, 2, 58, 7, 58, 2, 59, 7, 59, 2, 60, 7, 60, 2, 61, 7, 61, 2, 62, 7, 62, 2, 63, 7, 63, 2, 64, 7, 64, 2, 65, 7, 65, 2, 66, 7, 66, 2, 67, 7, 67, 2, 68, 7, 68, 2, 69, 7, 69, 2, 70, 7, 70, 2, 71, 7, 71, 2, 72, 7, 72, 2, 73, 7, 73, 2, 74, 7, 74, 2, 75, 7, 75, 2, 76, 7, 76, 1, 0, 1, 0, 1, 1, 1, 1, 1, 2, 1, 2, 1, 3, 1, 3, 1, 4, 1, 4, 1, 5, 1, 5, 1, 6, 1, 6, 1, 7, 1, 7, 1, 8, 1, 8, 1, 8, 1, 9, 1, 9, 1, 10, 1, 10, 1, 10, 1, 11, 1, 11, 1, 11, 1, 12, 1, 12, 1, 12, 1, 13, 1, 13, 1, 13, 1, 13, 1, 13, 1, 13, 1, 13, 1, 13, 3, 13, 194, 8, 13, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 3, 14, 208, 8, 14, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 3, 15, 230, 8, 15, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 3, 16, 256, 8, 16, 1, 17, 1, 17, 1, 18, 1, 18, 1, 19, 1, 19, 1, 20, 1, 20, 1, 21, 1, 21, 1, 22, 1, 22, 1, 22, 1, 23, 1, 23, 1, 23, 1, 24, 1, 24, 1, 24, 1, 25, 1, 25, 1, 26, 1, 26, 1, 27, 1, 27, 1, 28, 1, 28, 1, 28, 1, 28, 1, 28, 1, 28, 1, 28, 1, 28, 3, 28, 291, 8, 28, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 3, 29, 299, 8, 29, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 3, 30, 315, 8, 30, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 3, 31, 339, 8, 31, 1, 32, 1, 32, 1, 33, 1, 33, 1, 33, 1, 33, 1, 33, 1, 33, 1, 33, 3, 33, 350, 8, 33, 1, 34, 1, 34, 1, 34, 1, 34, 3, 34, 356, 8, 34, 1, 35, 1, 35, 1, 35, 5, 35, 361, 8, 35, 10, 35, 12, 35, 364, 9, 35, 1, 35, 1, 35, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 3, 36, 394, 8, 36, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 3, 37, 430, 8, 37, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 3, 38, 466, 8, 38, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 3, 39, 496, 8, 39, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 3, 40, 534, 8, 40, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 3, 41, 572, 8, 41, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 3, 42, 598, 8, 42, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 3, 43, 627, 8, 43, 1, 44, 1, 44, 1, 44, 1, 44, 3, 44, 633, 8, 44, 1, 45, 1, 45, 3, 45, 637, 8, 45, 1, 46, 1, 46, 1, 46, 5, 46, 642, 8, 46, 10, 46, 12, 46, 645, 9, 46, 1, 47, 1, 47, 1, 47, 1, 47, 1, 47, 1, 47, 1, 48, 3, 48, 654, 8, 48, 1, 48, 1, 48, 3, 48, 658, 8, 48, 1, 48, 1, 48, 1, 48, 3, 48, 663, 8, 48, 1, 48, 3, 48, 666, 8, 48, 1, 49, 1, 49, 3, 49, 670, 8, 49, 1, 49, 1, 49, 1, 49, 3, 49, 675, 8, 49, 1, 49, 1, 49, 4, 49, 679, 8, 49, 11, 49, 12, 49, 680, 1, 50, 1, 50, 1, 50, 3, 50, 686, 8, 50, 1, 51, 4, 51, 689, 8, 51, 11, 51, 12, 51, 690, 1, 52, 4, 52, 694, 8, 52, 11, 52, 12, 52, 695, 1, 53, 1, 53, 1, 53, 1, 53, 1, 53, 1, 53, 1, 53, 3, 53, 705, 8, 53, 1, 54, 1, 54, 1, 54, 1, 54, 1, 54, 1, 54, 1, 54, 3, 54, 714, 8, 54, 1, 55, 1, 55, 1, 56, 1, 56, 1, 57, 1, 57, 1, 57, 4, 57, 723, 8, 57, 11, 57, 12, 57, 724, 1, 58, 1, 58, 5, 58, 729, 8, 58, 10, 58, 12, 58, 732, 9, 58, 1, 58, 3, 58, 735, 8, 58, 1, 59, 1, 59, 5, 59, 739, 8, 59, 10, 59, 12, 59, 742, 9, 59, 1, 60, 1, 60, 1, 60, 1, 60, 1, 61, 1, 61, 1, 62, 1, 62, 1, 63, 1, 63, 1, 64, 1, 64, 1, 64, 1, 64, 1, 64, 1, 65, 1, 65, 1, 65, 1, 65, 1, 65, 1, 65, 1, 65, 1, 65, 1, 65, 1, 65, 3, 65, 769, 8, 65, 1, 66, 1, 66, 3, 66, 773, 8, 66, 1, 66, 1, 66, 1, 66, 3, 66, 778, 8, 66, 1, 67, 1, 67, 1, 67, 1, 67, 3, 67, 784, 8, 67, 1, 67, 1, 67, 1, 68, 3, 68, 789, 8, 68, 1, 68, 1, 68, 1, 68, 1, 68, 1, 68, 3, 68, 796, 8, 68, 1, 69, 1, 69, 3, 69, 800, 8, 69, 1, 69, 1, 69, 1, 70, 4, 70, 805, 8, 70, 11, 70, 12, 70, 806, 1, 71, 3, 71, 810, 8, 71, 1, 71, 1, 71, 1, 71, 1, 71, 1, 71, 3, 71, 817, 8, 71, 1, 72, 4, 72, 820, 8, 72, 11, 72, 12, 72, 821, 1, 73, 1, 73, 3, 73, 826, 8, 73, 1, 73, 1, 73, 1, 74, 1, 74, 1, 74, 1, 74, 1, 74, 3, 74, 835, 8, 74, 1, 74, 3, 74, 838, 8, 74, 1, 74, 1, 74, 1, 74, 1, 74, 1, 74, 3, 74, 845, 8, 74, 1, 75, 4, 75, 848, 8, 75, 11, 75, 12, 75, 849, 1, 75, 1, 75, 1, 76, 1, 76, 3, 76, 856, 8, 76, 1, 76, 3, 76, 859, 8, 76, 1, 76, 1, 76, 0, 0, 77, 1, 1, 3, 2, 5, 3, 7, 4, 9, 5, 11, 6, 13, 7, 15, 8, 17, 9, 19, 10, 21, 11, 23, 12, 25, 13, 27, 14, 29, 15, 31, 16, 33, 17, 35, 18, 37, 19, 39, 20, 41, 21, 43, 22, 45, 23, 47, 24, 49, 25, 51, 26, 53, 27, 55, 28, 57, 29, 59, 30, 61, 31, 63, 32, 65, 33, 67, 34, 69, 35, 71, 36, 73, 37, 75, 38, 77, 39, 79, 40, 81, 41, 83, 42, 85, 43, 87, 44, 89, 45, 91, 46, 93, 47, 95, 48, 97, 49, 99, 50, 101, 0, 103, 0, 105, 0, 107, 0, 109, 0, 111, 0, 113, 0, 115, 0, 117, 0, 119, 0, 121, 0, 123, 0, 125, 0, 127, 0, 129, 0, 131, 0, 133, 0, 135, 0, 137, 0, 139, 0, 141, 0, 143, 0, 145, 0, 147, 0, 149, 0, 151, 51, 153, 52, 1, 0, 16, 3, 0, 76, 76, 85, 85, 117, 117, 4, 0, 10, 10, 13, 13, 34, 34, 92, 92, 4, 0, 10, 10, 13, 13, 39, 39, 92, 92, 3, 0, 65, 90, 95, 95, 97, 122, 1, 0, 48, 57, 2, 0, 66, 66, 98, 98, 1, 0, 48, 49, 2, 0, 88, 88, 120, 120, 1, 0, 49, 57, 1, 0, 48, 55, 3, 0, 48, 57, 65, 70, 97, 102, 2, 0, 69, 69, 101, 101, 2, 0, 43, 43, 45, 45, 2, 0, 80, 80, 112, 112, 10, 0, 34, 34, 39, 39, 63, 63, 92, 92, 97, 98, 102, 102, 110, 110, 114, 114, 116, 116, 118, 118, 2, 0, 9, 9, 32, 32, 909, 0, 1, 1, 0, 0, 0, 0, 3, 1, 0, 0, 0, 0, 5, 1, 0, 0, 0, 0, 7, 1, 0, 0, 0, 0, 9, 1, 0, 0, 0, 0, 11, 1, 0, 0, 0, 0, 13, 1, 0, 0, 0, 0, 15, 1, 0, 0, 0, 0, 17, 1, 0, 0, 0, 0, 19, 1, 0, 0, 0, 0, 21, 1, 0, 0, 0, 0, 23, 1, 0, 0, 0, 0, 25, 1, 0, 0, 0, 0, 27, 1, 0, 0, 0, 0, 29, 1, 0, 0, 0, 0, 31, 1, 0, 0, 0, 0, 33, 1, 0, 0, 0, 0, 35, 1, 0, 0, 0, 0, 37, 1, 0, 0, 0, 0, 39, 1, 0, 0, 0, 0, 41, 1, 0, 0, 0, 0, 43, 1, 0, 0, 0, 0, 45, 1, 0, 0, 0, 0, 47, 1, 0, 0, 0, 0, 49, 1, 0, 0, 0, 0, 51, 1, 0, 0, 0, 0, 53, 1, 0, 0, 0, 0, 55, 1, 0, 0, 0, 0, 57, 1, 0, 0, 0, 0, 59, 1, 0, 0, 0, 0, 61, 1, 0, 0, 0, 0, 63, 1, 0, 0, 0, 0, 65, 1, 0, 0, 0, 0, 67, 1, 0, 0, 0, 0, 69, 1, 0, 0, 0, 0, 71, 1, 0, 0, 0, 0, 73, 1, 0, 0, 0, 0, 75, 1, 0, 0, 0, 0, 77, 1, 0, 0, 0, 0, 79, 1, 0, 0, 0, 0, 81, 1, 0, 0, 0, 0, 83, 1, 0, 0, 0, 0, 85, 1, 0, 0, 0, 0, 87, 1, 0, 0, 0, 0, 89, 1, 0, 0, 0, 0, 91, 1, 0, 0, 0, 0, 93, 1, 0, 0, 0, 0, 95, 1, 0, 0, 0, 0, 97, 1, 0, 0, 0, 0, 99, 1, 0, 0, 0, 0, 151, 1, 0, 0, 0, 0, 153, 1, 0, 0, 0, 1, 155, 1, 0, 0, 0, 3, 157, 1, 0, 0, 0, 5, 159, 1, 0, 0, 0, 7, 161, 1, 0, 0, 0, 9, 163, 1, 0, 0, 0, 11, 165, 1, 0, 0, 0, 13, 167, 1, 0, 0, 0, 15, 169, 1, 0, 0, 0, 17, 171, 1, 0, 0, 0, 19, 174, 1, 0, 0, 0, 21, 176, 1, 0, 0, 0, 23, 179, 1, 0, 0, 0, 25, 182, 1, 0, 0, 0, 27, 193, 1, 0, 0, 0, 29, 207, 1, 0, 0, 0, 31, 229, 1, 0, 0, 0, 33, 255, 1, 0, 0, 0, 35, 257, 1, 0, 0, 0, 37, 259, 1, 0, 0, 0, 39, 261, 1, 0, 0, 0, 41, 263, 1, 0, 0, 0, 43, 265, 1, 0, 0, 0, 45, 267, 1, 0, 0, 0, 47, 270, 1, 0, 0, 0, 49, 273, 1, 0, 0, 0, 51, 276, 1, 0, 0, 0, 53, 278, 1, 0, 0, 0, 55, 280, 1, 0, 0, 0, 57, 290, 1, 0, 0, 0, 59, 298, 1, 0, 0, 0, 61, 314, 1, 0, 0, 0, 63, 338, 1, 0, 0, 0, 65, 340, 1, 0, 0, 0, 67, 349, 1, 0, 0, 0, 69, 355, 1, 0, 0, 0, 71, 357, 1, 0, 0, 0, 73, 393, 1, 0, 0, 0, 75, 429, 1, 0, 0, 0, 77, 465, 1, 0, 0, 0, 79, 495, 1, 0, 0, 0, 81, 533, 1, 0, 0, 0, 83, 571, 1, 0, 0, 0, 85, 597, 1, 0, 0, 0, 87, 626, 1, 0, 0, 0, 89, 632, 1, 0, 0, 0, 91, 636, 1, 0, 0, 0, 93, 638, 1, 0, 0, 0, 95, 646, 1, 0, 0, 0, 97, 653, 1, 0, 0, 0, 99, 669, 1, 0, 0, 0, 101, 685, 1, 0, 0, 0, 103, 688, 1, 0, 0, 0, 105, 693, 1, 0, 0, 0, 107, 704, 1, 0, 0, 0, 109, 713, 1, 0, 0, 0, 111, 715, 1, 0, 0, 0, 113, 717, 1, 0, 0, 0, 115, 719, 1, 0, 0, 0, 117, 734, 1, 0, 0, 0, 119, 736, 1, 0, 0, 0, 121, 743, 1, 0, 0, 0, 123, 747, 1, 0, 0, 0, 125, 749, 1, 0, 0, 0, 127, 751, 1, 0, 0, 0, 129, 753, 1, 0, 0, 0, 131, 768, 1, 0, 0, 0, 133, 777, 1, 0, 0, 0, 135, 779, 1, 0, 0, 0, 137, 795, 1, 0, 0, 0, 139, 797, 1, 0, 0, 0, 141, 804, 1, 0, 0, 0, 143, 816, 1, 0, 0, 0, 145, 819, 1, 0, 0, 0, 147, 823, 1, 0, 0, 0, 149, 844, 1, 0, 0, 0, 151, 847, 1, 0, 0, 0, 153, 858, 1, 0, 0, 0, 155, 156, 5, 40, 0, 0, 156, 2, 1, 0, 0, 0, 157, 158, 5, 41, 0, 0, 158, 4, 1, 0, 0, 0, 159, 160, 5, 91, 0, 0, 160, 6, 1, 0, 0, 0, 161, 162, 5, 44, 0, 0, 162, 8, 1, 0, 0, 0, 163, 164, 5, 93, 0, 0, 164, 10, 1, 0, 0, 0, 165, 166, 5, 123, 0, 0, 166, 12, 1, 0, 0, 0, 167, 168, 5, 125, 0, 0, 168, 14, 1, 0, 0, 0, 169, 170, 5, 60, 0, 0, 170, 16, 1, 0, 0, 0, 171, 172, 5, 60, 0, 0, 172, 173, 5, 61, 0, 0, 173, 18, 1, 0, 0, 0, 174, 175, 5, 62, 0, 0, 175, 20, 1, 0, 0, 0, 176, 177, 5, 62, 0, 0, 177, 178, 5, 61, 0, 0, 178, 22, 1, 0, 0, 0, 179, 180, 5, 61, 0, 0, 180, 181, 5, 61, 0, 0, 181, 24, 1, 0, 0, 0, 182, 183, 5, 33, 0, 0, 183, 184, 5, 61, 0, 0, 184, 26, 1, 0, 0, 0, 185, 186, 5, 108, 0, 0, 186, 187, 5, 105, 0, 0, 187, 188, 5, 107, 0, 0, 188, 194, 5, 101, 0, 0, 189, 190, 5, 76, 0, 0, 190, 191, 5, 73, 0, 0, 191, 192, 5, 75, 0, 0, 192, 194, 5, 69, 0, 0, 193, 185, 1, 0, 0, 0, 193, 189, 1, 0, 0, 0, 194, 28, 1, 0, 0, 0, 195, 196, 5, 101, 0, 0, 196, 197, 5, 120, 0, 0, 197, 198, 5, 105, 0, 0, 198, 199, 5, 115, 0, 0, 199, 200, 5, 116, 0, 0, 200, 208, 5, 115, 0, 0, 201, 202, 5, 69, 0, 0, 202, 203, 5, 88, 0, 0, 203, 204, 5, 73, 0, 0, 204, 205, 5, 83, 0, 0, 205, 206, 5, 84, 0, 0, 206, 208, 5, 83, 0, 0, 207, 195, 1, 0, 0, 0, 207, 201, 1, 0, 0, 0, 208, 30, 1, 0, 0, 0, 209, 210, 5, 116, 0, 0, 210, 211, 5, 101, 0, 0, 211, 212, 5, 120, 0, 0, 212, 213, 5, 116, 0, 0, 213, 214, 5, 95, 0, 0, 214, 215, 5, 109, 0, 0, 215, 216, 5, 97, 0, 0, 216, 217, 5, 116, 0, 0, 217, 218, 5, 99, 0, 0, 218, 230, 5, 104, 0, 0, 219, 220, 5, 84, 0, 0, 220, 221, 5, 69, 0, 0, 221, 222, 5, 88, 0, 0, 222, 223, 5, 84, 0, 0, 223, 224, 5, 95, 0, 0, 224, 225, 5, 77, 0, 0, 225, 226, 5, 65, 0, 0, 226, 227, 5, 84, 0, 0, 227, 228, 5, 67, 0, 0, 228, 230, 5, 72, 0, 0, 229, 209, 1, 0, 0, 0, 229, 219, 1, 0, 0, 0, 230, 32, 1, 0, 0, 0, 231, 232, 5, 112, 0, 0, 232, 233, 5, 104, 0, 0, 233, 234, 5, 114, 0, 0, 234, 235, 5, 97, 0, 0, 235, 236, 5, 115, 0, 0, 236, 237, 5, 101, 0, 0, 237, 238, 5, 95, 0, 0, 238, 239, 5, 109, 0, 0, 239, 240, 5, 97, 0, 0, 240, 241, 5, 116, 0, 0, 241, 242, 5, 99, 0, 0, 242, 256, 5, 104, 0, 0, 243, 244, 5, 80, 0, 0, 244, 245, 5, 72, 0, 0, 245, 246, 5, 82, 0, 0, 246, 247, 5, 65, 0, 0, 247, 248, 5, 83, 0, 0, 248, 249, 5, 69, 0, 0, 249, 250, 5, 95, 0, 0, 250, 251, 5, 77, 0, 0, 251, 252, 5, 65, 0, 0, 252, 253, 5, 84, 0, 0, 253, 254, 5, 67, 0, 0, 254, 256, 5, 72, 0, 0, 255, 231, 1, 0, 0, 0, 255, 243, 1, 0, 0, 0, 256, 34, 1, 0, 0, 0, 257, 258, 5, 43, 0, 0, 258, 36, 1, 0, 0, 0, 259, 260, 5, 45, 0, 0, 260, 38, 1, 0, 0, 0, 261, 262, 5, 42, 0, 0, 262, 40, 1, 0, 0, 0, 263, 264, 5, 47, 0, 0, 264, 42, 1, 0, 0, 0, 265, 266, 5, 37, 0, 0, 266, 44, 1, 0, 0, 0, 267, 268, 5, 42, 0, 0, 268, 269, 5, 42, 0, 0, 269, 46, 1, 0, 0, 0, 270, 271, 5, 60, 0, 0, 271, 272, 5, 60, 0, 0, 272, 48, 1, 0, 0, 0, 273, 274, 5, 62, 0, 0, 274, 275, 5, 62, 0, 0, 275, 50, 1, 0, 0, 0, 276, 277, 5, 38, 0, 0, 277, 52, 1, 0, 0, 0, 278, 279, 5, 124, 0, 0, 279, 54, 1, 0, 0, 0, 280, 281, 5, 94, 0, 0, 281, 56, 1, 0, 0, 0, 282, 283, 5, 38, 0, 0, 283, 291, 5, 38, 0, 0, 284, 285, 5, 97, 0, 0, 285, 286, 5, 110, 0, 0, 286, 291, 5, 100, 0, 0, 287, 288, 5, 65, 0, 0, 288, 289, 5, 78, 0, 0, 289, 291, 5, 68, 0, 0, 290, 282, 1, 0, 0, 0, 290, 284, 1, 0, 0, 0, 290, 287, 1, 0, 0, 0, 291, 58, 1, 0, 0, 0, 292, 293, 5, 124, 0, 0, 293, 299, 5, 124, 0, 0, 294, 295, 5, 111, 0, 0, 295, 299, 5, 114, 0, 0, 296, 297, 5, 79, 0, 0, 297, 299, 5, 82, 0, 0, 298, 292, 1, 0, 0, 0, 298, 294, 1, 0, 0, 0, 298, 296, 1, 0, 0, 0, 299, 60, 1, 0, 0, 0, 300, 301, 5, 105, 0, 0, 301, 302, 5, 115, 0, 0, 302, 303, 5, 32, 0, 0, 303, 304, 5, 110, 0, 0, 304, 305, 5, 117, 0, 0, 305, 306, 5, 108, 0, 0, 306, 315, 5, 108, 0, 0, 307, 308, 5, 73, 0, 0, 308, 309, 5, 83, 0, 0, 309, 310, 5, 32, 0, 0, 310, 311, 5, 78, 0, 0, 311, 312, 5, 85, 0, 0, 312, 313, 5, 76, 0, 0, 313, 315, 5, 76, 0, 0, 314, 300, 1, 0, 0, 0, 314, 307, 1, 0, 0, 0, 315, 62, 1, 0, 0, 0, 316, 317, 5, 105, 0, 0, 317, 318, 5, 115, 0, 0, 318, 319, 5, 32, 0, 0, 319, 320, 5, 110, 0, 0, 320, 321, 5, 111, 0, 0, 321, 322, 5, 116, 0, 0, 322, 323, 5, 32, 0, 0, 323, 324, 5, 110, 0, 0, 324, 325, 5, 117, 0, 0, 325, 326, 5, 108, 0, 0, 326, 339, 5, 108, 0, 0, 327, 328, 5, 73, 0, 0, 328, 329, 5, 83, 0, 0, 329, 330, 5, 32, 0, 0, 330, 331, 5, 78, 0, 0, 331, 332, 5, 79, 0, 0, 332, 333, 5, 84, 0, 0, 333, 334, 5, 32, 0, 0, 334, 335, 5, 78, 0, 0, 335, 336, 5, 85, 0, 0, 336, 337, 5, 76, 0, 0, 337, 339, 5, 76, 0, 0, 338, 316, 1, 0, 0, 0, 338, 327, 1, 0, 0, 0, 339, 64, 1, 0, 0, 0, 340, 341, 5, 126, 0, 0, 341, 66, 1, 0, 0, 0, 342, 350, 5, 33, 0, 0, 343, 344, 5, 110, 0, 0, 344, 345, 5, 111, 0, 0, 345, 350, 5, 116, 0, 0, 346, 347, 5, 78, 0, 0, 347, 348, 5, 79, 0, 0, 348, 350, 5, 84, 0, 0, 349, 342, 1, 0, 0, 0, 349, 343, 1, 0, 0, 0, 349, 346, 1, 0, 0, 0, 350, 68, 1, 0, 0, 0, 351, 352, 5, 105, 0, 0, 352, 356, 5, 110, 0, 0, 353, 354, 5, 73, 0, 0, 354, 356, 5, 78, 0, 0, 355, 351, 1, 0, 0, 0, 355, 353, 1, 0, 0, 0, 356, 70, 1, 0, 0, 0, 357, 362, 5, 91, 0, 0, 358, 361, 3, 151, 75, 0, 359, 361, 3, 153, 76, 0, 360, 358, 1, 0, 0, 0, 360, 359, 1, 0, 0, 0, 361, 364, 1, 0, 0, 0, 362, 360, 1, 0, 0, 0, 362, 363, 1, 0, 0, 0, 363, 365, 1, 0, 0, 0, 364, 362, 1, 0, 0, 0, 365, 366, 5, 93, 0, 0, 366, 72, 1, 0, 0, 0, 367, 368, 5, 106, 0, 0, 368, 369, 5, 115, 0, 0, 369, 370, 5, 111, 0, 0, 370, 371, 5, 110, 0, 0, 371, 372, 5, 95, 0, 0, 372, 373, 5, 99, 0, 0, 373, 374, 5, 111, 0, 0, 374, 375, 5, 110, 0, 0, 375, 376, 5, 116, 0, 0, 376, 377, 5, 97, 0, 0, 377, 378, 5, 105, 0, 0, 378, 379, 5, 110, 0, 0, 379, 394, 5, 115, 0, 0, 380, 381, 5, 74, 0, 0, 381, 382, 5, 83, 0, 0, 382, 383, 5, 79, 0, 0, 383, 384, 5, 78, 0, 0, 384, 385, 5, 95, 0, 0, 385, 386, 5, 67, 0, 0, 386, 387, 5, 79, 0, 0, 387, 388, 5, 78, 0, 0, 388, 389, 5, 84, 0, 0, 389, 390, 5, 65, 0, 0, 390, 391, 5, 73, 0, 0, 391, 392, 5, 78, 0, 0, 392, 394, 5, 83, 0, 0, 393, 367, 1, 0, 0, 0, 393, 380, 1, 0, 0, 0, 394, 74, 1, 0, 0, 0, 395, 396, 5, 106, 0, 0, 396, 397, 5, 115, 0, 0, 397, 398, 5, 111, 0, 0, 398, 399, 5, 110, 0, 0, 399, 400, 5, 95, 0, 0, 400, 401, 5, 99, 0, 0, 401, 402, 5, 111, 0, 0, 402, 403, 5, 110, 0, 0, 403, 404, 5, 116, 0, 0, 404, 405, 5, 97, 0, 0, 405, 406, 5, 105, 0, 0, 406, 407, 5, 110, 0, 0, 407, 408, 5, 115, 0, 0, 408, 409, 5, 95, 0, 0, 409, 410, 5, 97, 0, 0, 410, 411, 5, 108, 0, 0, 411, 430, 5, 108, 0, 0, 412, 413, 5, 74, 0, 0, 413, 414, 5, 83, 0, 0, 414, 415, 5, 79, 0, 0, 415, 416, 5, 78, 0, 0, 416, 417, 5, 95, 0, 0, 417, 418, 5, 67, 0, 0, 418, 419, 5, 79, 0, 0, 419, 420, 5, 78, 0, 0, 420, 421, 5, 84, 0, 0, 421, 422, 5, 65, 0, 0, 422, 423, 5, 73, 0, 0, 423, 424, 5, 78, 0, 0, 424, 425, 5, 83, 0, 0, 425, 426, 5, 95, 0, 0, 426, 427, 5, 65, 0, 0, 427, 428, 5, 76, 0, 0, 428, 430, 5, 76, 0, 0, 429, 395, 1, 0, 0, 0, 429, 412, 1, 0, 0, 0, 430, 76, 1, 0, 0, 0, 431, 432, 5, 106, 0, 0, 432, 433, 5, 115, 0, 0, 433, 434, 5, 111, 0, 0, 434, 435, 5, 110, 0, 0, 435, 436, 5, 95, 0, 0, 436, 437, 5, 99, 0, 0, 437, 438, 5, 111, 0, 0, 438, 439, 5, 110, 0, 0, 439, 440, 5, 116, 0, 0, 440, 441, 5, 97, 0, 0, 441, 442, 5, 105, 0, 0, 442, 443, 5, 110, 0, 0, 443, 444, 5, 115, 0, 0, 444, 445, 5, 95, 0, 0, 445, 446, 5, 97, 0, 0, 446, 447, 5, 110, 0, 0, 447, 466, 5, 121, 0, 0, 448, 449, 5, 74, 0, 0, 449, 450, 5, 83, 0, 0, 450, 451, 5, 79, 0, 0, 451, 452, 5, 78, 0, 0, 452, 453, 5, 95, 0, 0, 453, 454, 5, 67, 0, 0, 454, 455, 5, 79, 0, 0, 455, 456, 5, 78, 0, 0, 456, 457, 5, 84, 0, 0, 457, 458, 5, 65, 0, 0, 458, 459, 5, 73, 0, 0, 459, 460, 5, 78, 0, 0, 460, 461, 5, 83, 0, 0, 461, 462, 5, 95, 0, 0, 462, 463, 5, 65, 0, 0, 463, 464, 5, 78, 0, 0, 464, 466, 5, 89, 0, 0, 465, 431, 1, 0, 0, 0, 465, 448, 1, 0, 0, 0, 466, 78, 1, 0, 0, 0, 467, 468, 5, 97, 0, 0, 468, 469, 5, 114, 0, 0, 469, 470, 5, 114, 0, 0, 470, 471, 5, 97, 0, 0, 471, 472, 5, 121, 0, 0, 472, 473, 5, 95, 0, 0, 473, 474, 5, 99, 0, 0, 474, 475, 5, 111, 0, 0, 475, 476, 5, 110, 0, 0, 476, 477, 5, 116, 0, 0, 477, 478, 5, 97, 0, 0, 478, 479, 5, 105, 0, 0, 479, 480, 5, 110, 0, 0, 480, 496, 5, 115, 0, 0, 481, 482, 5, 65, 0, 0, 482, 483, 5, 82, 0, 0, 483, 484, 5, 82, 0, 0, 484, 485, 5, 65, 0, 0, 485, 486, 5, 89, 0, 0, 486, 487, 5, 95, 0, 0, 487, 488, 5, 67, 0, 0, 488, 489, 5, 79, 0, 0, 489, 490, 5, 78, 0, 0, 490, 491, 5, 84, 0, 0, 491, 492, 5, 65, 0, 0, 492, 493, 5, 73, 0, 0, 493, 494, 5, 78, 0, 0, 494, 496, 5, 83, 0, 0, 495, 467, 1, 0, 0, 0, 495, 481, 1, 0, 0, 0, 496, 80, 1, 0, 0, 0, 497, 498, 5, 97, 0, 0, 498, 499, 5, 114, 0, 0, 499, 500, 5, 114, 0, 0, 500, 501, 5, 97, 0, 0, 501, 502, 5, 121, 0, 0, 502, 503, 5, 95, 0, 0, 503, 504, 5, 99, 0, 0, 504, 505, 5, 111, 0, 0, 505, 506, 5, 110, 0, 0, 506, 507, 5, 116, 0, 0, 507, 508, 5, 97, 0, 0, 508, 509, 5, 105, 0, 0, 509, 510, 5, 110, 0, 0, 510, 511, 5, 115, 0, 0, 511, 512, 5, 95, 0, 0, 512, 513, 5, 97, 0, 0, 513, 514, 5, 108, 0, 0, 514, 534, 5, 108, 0, 0, 515, 516, 5, 65, 0, 0, 516, 517, 5, 82, 0, 0, 517, 518, 5, 82, 0, 0, 518, 519, 5, 65, 0, 0, 519, 520, 5, 89, 0, 0, 520, 521, 5, 95, 0, 0, 521, 522, 5, 67, 0, 0, 522, 523, 5, 79, 0, 0, 523, 524, 5, 78, 0, 0, 524, 525, 5, 84, 0, 0, 525, 526, 5, 65, 0, 0, 526, 527, 5, 73, 0, 0, 527, 528, 5, 78, 0, 0, 528, 529, 5, 83, 0, 0, 529, 530, 5, 95, 0, 0, 530, 531, 5, 65, 0, 0, 531, 532, 5, 76, 0, 0, 532, 534, 5, 76, 0, 0, 533, 497, 1, 0, 0, 0, 533, 515, 1, 0, 0, 0, 534, 82, 1, 0, 0, 0, 535, 536, 5, 97, 0, 0, 536, 537, 5, 114, 0, 0, 537, 538, 5, 114, 0, 0, 538, 539, 5, 97, 0, 0, 539, 540, 5, 121, 0, 0, 540, 541, 5, 95, 0, 0, 541, 542, 5, 99, 0, 0, 542, 543, 5, 111, 0, 0, 543, 544, 5, 110, 0, 0, 544, 545, 5, 116, 0, 0, 545, 546, 5, 97, 0, 0, 546, 547, 5, 105, 0, 0, 547, 548, 5, 110, 0, 0, 548, 549, 5, 115, 0, 0, 549, 550, 5, 95, 0, 0, 550, 551, 5, 97, 0, 0, 551, 552, 5, 110, 0, 0, 552, 572, 5, 121, 0, 0, 553, 554, 5, 65, 0, 0, 554, 555, 5, 82, 0, 0, 555, 556, 5, 82, 0, 0, 556, 557, 5, 65, 0, 0, 557, 558, 5, 89, 0, 0, 558, 559, 5, 95, 0, 0, 559, 560, 5, 67, 0, 0, 560, 561, 5, 79, 0, 0, 561, 562, 5, 78, 0, 0, 562, 563, 5, 84, 0, 0, 563, 564, 5, 65, 0, 0, 564, 565, 5, 73, 0, 0, 565, 566, 5, 78, 0, 0, 566, 567, 5, 83, 0, 0, 567, 568, 5, 95, 0, 0, 568, 569, 5, 65, 0, 0, 569, 570, 5, 78, 0, 0, 570, 572, 5, 89, 0, 0, 571, 535, 1, 0, 0, 0, 571, 553, 1, 0, 0, 0, 572, 84, 1, 0, 0, 0, 573, 574, 5, 97, 0, 0, 574, 575, 5, 114, 0, 0, 575, 576, 5, 114, 0, 0, 576, 577, 5, 97, 0, 0, 577, 578, 5, 121, 0, 0, 578, 579, 5, 95, 0, 0, 579, 580, 5, 108, 0, 0, 580, 581, 5, 101, 0, 0, 581, 582, 5, 110, 0, 0, 582, 583, 5, 103, 0, 0, 583, 584, 5, 116, 0, 0, 584, 598, 5, 104, 0, 0, 585, 586, 5, 65, 0, 0, 586, 587, 5, 82, 0, 0, 587, 588, 5, 82, 0, 0, 588, 589, 5, 65, 0, 0, 589, 590, 5, 89, 0, 0, 590, 591, 5, 95, 0, 0, 591, 592, 5, 76, 0, 0, 592, 593, 5, 69, 0, 0, 593, 594, 5, 78, 0, 0, 594, 595, 5, 71, 0, 0, 595, 596, 5, 84, 0, 0, 596, 598, 5, 72, 0, 0, 597, 573, 1, 0, 0, 0, 597, 585, 1, 0, 0, 0, 598, 86, 1, 0, 0, 0, 599, 600, 5, 116, 0, 0, 600, 601, 5, 114, 0, 0, 601, 602, 5, 117, 0, 0, 602, 627, 5, 101, 0, 0, 603, 604, 5, 84, 0, 0, 604, 605, 5, 114, 0, 0, 605, 606, 5, 117, 0, 0, 606, 627, 5, 101, 0, 0, 607, 608, 5, 84, 0, 0, 608, 609, 5, 82, 0, 0, 609, 610, 5, 85, 0, 0, 610, 627, 5, 69, 0, 0, 611, 612, 5, 102, 0, 0, 612, 613, 5, 97, 0, 0, 613, 614, 5, 108, 0, 0, 614, 615, 5, 115, 0, 0, 615, 627, 5, 101, 0, 0, 616, 617, 5, 70, 0, 0, 617, 618, 5, 97, 0, 0, 618, 619, 5, 108, 0, 0, 619, 620, 5, 115, 0, 0, 620, 627, 5, 101, 0, 0, 621, 622, 5, 70, 0, 0, 622, 623, 5, 65, 0, 0, 623, 624, 5, 76, 0, 0, 624, 625, 5, 83, 0, 0, 625, 627, 5, 69, 0, 0, 626, 599, 1, 0, 0, 0, 626, 603, 1, 0, 0, 0, 626, 607, 1, 0, 0, 0, 626, 611, 1, 0, 0, 0, 626, 616, 1, 0, 0, 0, 626, 621, 1, 0, 0, 0, 627, 88, 1, 0, 0, 0, 628, 633, 3, 117, 58, 0, 629, 633, 3, 119, 59, 0, 630, 633, 3, 121, 60, 0, 631, 633, 3, 115, 57, 0, 632, 628, 1, 0, 0, 0, 632, 629, 1, 0, 0, 0, 632, 630, 1, 0, 0, 0, 632, 631, 1, 0, 0, 0, 633, 90, 1, 0, 0, 0, 634, 637, 3, 133, 66, 0, 635, 637, 3, 135, 67, 0, 636, 634, 1, 0, 0, 0, 636, 635, 1, 0, 0, 0, 637, 92, 1, 0, 0, 0, 638, 643, 3, 111, 55, 0, 639, 642, 3, 111, 55, 0, 640, 642, 3, 113, 56, 0, 641, 639, 1, 0, 0, 0, 641, 640, 1, 0, 0, 0, 642, 645, 1, 0, 0, 0, 643, 641, 1, 0, 0, 0, 643, 644, 1, 0, 0, 0, 644, 94, 1, 0, 0, 0, 645, 643, 1, 0, 0, 0, 646, 647, 5, 36, 0, 0, 647, 648, 5, 109, 0, 0, 648, 649, 5, 101, 0, 0, 649, 650, 5, 116, 0, 0, 650, 651, 5, 97, 0, 0, 651, 96, 1, 0, 0, 0, 652, 654, 3, 101, 50, 0, 653, 652, 1, 0, 0, 0, 653, 654, 1, 0, 0, 0, 654, 665, 1, 0, 0, 0, 655, 657, 5, 34, 0, 0, 656, 658, 3, 103, 51, 0, 657, 656, 1, 0, 0, 0, 657, 658, 1, 0, 0, 0, 658, 659, 1, 0, 0, 0, 659, 666, 5, 34, 0, 0, 660, 662, 5, 39, 0, 0, 661, 663, 3, 105, 52, 0, 662, 661, 1, 0, 0, 0, 662, 663, 1, 0, 0, 0, 663, 664, 1, 0, 0, 0, 664, 666, 5, 39, 0, 0, 665, 655, 1, 0, 0, 0, 665, 660, 1, 0, 0, 0, 666, 98, 1, 0, 0, 0, 667, 670, 3, 93, 46, 0, 668, 670, 3, 95, 47, 0, 669, 667, 1, 0, 0, 0, 669, 668, 1, 0, 0, 0, 670, 678, 1, 0, 0, 0, 671, 674, 5, 91, 0, 0, 672, 675, 3, 97, 48, 0, 673, 675, 3, 117, 58, 0, 674, 672, 1, 0, 0, 0, 674, 673, 1, 0, 0, 0, 675, 676, 1, 0, 0, 0, 676, 677, 5, 93, 0, 0, 677, 679, 1, 0, 0, 0, 678, 671, 1, 0, 0, 0, 679, 680, 1, 0, 0, 0, 680, 678, 1, 0, 0, 0, 680, 681, 1, 0, 0, 0, 681, 100, 1, 0, 0, 0, 682, 683, 5, 117, 0, 0, 683, 686, 5, 56, 0, 0, 684, 686, 7, 0, 0, 0, 685, 682, 1, 0, 0, 0, 685, 684, 1, 0, 0, 0, 686, 102, 1, 0, 0, 0, 687, 689, 3, 107, 53, 0, 688, 687, 1, 0, 0, 0, 689, 690, 1, 0, 0, 0, 690, 688, 1, 0, 0, 0, 690, 691, 1, 0, 0, 0, 691, 104, 1, 0, 0, 0, 692, 694, 3, 109, 54, 0, 693, 692, 1, 0, 0, 0, 694, 695, 1, 0, 0, 0, 695, 693, 1, 0, 0, 0, 695, 696, 1, 0, 0, 0, 696, 106, 1, 0, 0, 0, 697, 705, 8, 1, 0, 0, 698, 705, 3, 149, 74, 0, 699, 700, 5, 92, 0, 0, 700, 705, 5, 10, 0, 0, 701, 702, 5, 92, 0, 0, 702, 703, 5, 13, 0, 0, 703, 705, 5, 10, 0, 0, 704, 697, 1, 0, 0, 0, 704, 698, 1, 0, 0, 0, 704, 699, 1, 0, 0, 0, 704, 701, 1, 0, 0, 0, 705, 108, 1, 0, 0, 0, 706, 714, 8, 2, 0, 0, 707, 714, 3, 149, 74, 0, 708, 709, 5, 92, 0, 0, 709, 714, 5, 10, 0, 0, 710, 711, 5, 92, 0, 0, 711, 712, 5, 13, 0, 0, 712, 714, 5, 10, 0, 0, 713, 706, 1, 0, 0, 0, 713, 707, 1, 0, 0, 0, 713, 708, 1, 0, 0, 0, 713, 710, 1, 0, 0, 0, 714, 110, 1, 0, 0, 0, 715, 716, 7, 3, 0, 0, 716, 112, 1, 0, 0, 0, 717, 718, 7, 4, 0, 0, 718, 114, 1, 0, 0, 0, 719, 720, 5, 48, 0, 0, 720, 722, 7, 5, 0, 0, 721, 723, 7, 6, 0, 0, 722, 721, 1, 0, 0, 0, 723, 724, 1, 0, 0, 0, 724, 722, 1, 0, 0, 0, 724, 725, 1, 0, 0, 0, 725, 116, 1, 0, 0, 0, 726, 730, 3, 123, 61, 0, 727, 729, 3, 113, 56, 0, 728, 727, 1, 0, 0, 0, 729, 732, 1, 0, 0, 0, 730, 728, 1, 0, 0, 0, 730, 731, 1, 0, 0, 0, 731, 735, 1, 0, 0, 0, 732, 730, 1, 0, 0, 0, 733, 735, 5, 48, 0, 0, 734, 726, 1, 0, 0, 0, 734, 733, 1, 0, 0, 0, 735, 118, 1, 0, 0, 0, 736, 740, 5, 48, 0, 0, 737, 739, 3, 125, 62, 0, 738, 737, 1, 0, 0, 0, 739, 742, 1, 0, 0, 0, 740, 738, 1, 0, 0, 0, 740, 741, 1, 0, 0, 0, 741, 120, 1, 0, 0, 0, 742, 740, 1, 0, 0, 0, 743, 744, 5, 48, 0, 0, 744, 745, 7, 7, 0, 0, 745, 746, 3, 145, 72, 0, 746, 122, 1, 0, 0, 0, 747, 748, 7, 8, 0, 0, 748, 124, 1, 0, 0, 0, 749, 750, 7, 9, 0, 0, 750, 126, 1, 0, 0, 0, 751, 752, 7, 10, 0, 0, 752, 128, 1, 0, 0, 0, 753, 754, 3, 127, 63, 0, 754, 755, 3, 127, 63, 0, 755, 756, 3, 127, 63, 0, 756, 757, 3, 127, 63, 0, 757, 130, 1, 0, 0, 0, 758, 759, 5, 92, 0, 0, 759, 760, 5, 117, 0, 0, 760, 761, 1, 0, 0, 0, 761, 769, 3, 129, 64, 0, 762, 763, 5, 92, 0, 0, 763, 764, 5, 85, 0, 0, 764, 765, 1, 0, 0, 0, 765, 766, 3, 129, 64, 0, 766, 767, 3, 129, 64, 0, 767, 769, 1, 0, 0, 0, 768, 758, 1, 0, 0, 0, 768, 762, 1, 0, 0, 0, 769, 132, 1, 0, 0, 0, 770, 772, 3, 137, 68, 0, 771, 773, 3, 139, 69, 0, 772, 771, 1, 0, 0, 0, 772, 773, 1, 0, 0, 0, 773, 778, 1, 0, 0, 0, 774, 775, 3, 141, 70, 0, 775, 776, 3, 139, 69, 0, 776, 778, 1, 0, 0, 0, 777, 770, 1, 0, 0, 0, 777, 774, 1, 0, 0, 0, 778, 134, 1, 0, 0, 0, 779, 780, 5, 48, 0, 0, 780, 783, 7, 7, 0, 0, 781, 784, 3, 143, 71, 0, 782, 784, 3, 145, 72, 0, 783, 781, 1, 0, 0, 0, 783, 782, 1, 0, 0, 0, 784, 785, 1, 0, 0, 0, 785, 786, 3, 147, 73, 0, 786, 136, 1, 0, 0, 0, 787, 789, 3, 141, 70, 0, 788, 787, 1, 0, 0, 0, 788, 789, 1, 0, 0, 0, 789, 790, 1, 0, 0, 0, 790, 791, 5, 46, 0, 0, 791, 796, 3, 141, 70, 0, 792, 793, 3, 141, 70, 0, 793, 794, 5, 46, 0, 0, 794, 796, 1, 0, 0, 0, 795, 788, 1, 0, 0, 0, 795, 792, 1, 0, 0, 0, 796, 138, 1, 0, 0, 0, 797, 799, 7, 11, 0, 0, 798, 800, 7, 12, 0, 0, 799, 798, 1, 0, 0, 0, 799, 800, 1, 0, 0, 0, 800, 801, 1, 0, 0, 0, 801, 802, 3, 141, 70, 0, 802, 140, 1, 0, 0, 0, 803, 805, 3, 113, 56, 0, 804, 803, 1, 0, 0, 0, 805, 806, 1, 0, 0, 0, 806, 804, 1, 0, 0, 0, 806, 807, 1, 0, 0, 0, 807, 142, 1, 0, 0, 0, 808, 810, 3, 145, 72, 0, 809, 808, 1, 0, 0, 0, 809, 810, 1, 0, 0, 0, 810, 811, 1, 0, 0, 0, 811, 812, 5, 46, 0, 0, 812, 817, 3, 145, 72, 0, 813, 814, 3, 145, 72, 0, 814, 815, 5, 46, 0, 0, 815, 817, 1, 0, 0, 0, 816, 809, 1, 0, 0, 0, 816, 813, 1, 0, 0, 0, 817, 144, 1, 0, 0, 0, 818, 820, 3, 127, 63, 0, 819, 818, 1, 0, 0, 0, 820, 821, 1, 0, 0, 0, 821, 819, 1, 0, 0, 0, 821, 822, 1, 0, 0, 0, 822, 146, 1, 0, 0, 0, 823, 825, 7, 13, 0, 0, 824, 826, 7, 12, 0, 0, 825, 824, 1, 0, 0, 0, 825, 826, 1, 0, 0, 0, 826, 827, 1, 0, 0, 0, 827, 828, 3, 141, 70, 0, 828, 148, 1, 0, 0, 0, 829, 830, 5, 92, 0, 0, 830, 845, 7, 14, 0, 0, 831, 832, 5, 92, 0, 0, 832, 834, 3, 125, 62, 0, 833, 835, 3, 125, 62, 0, 834, 833, 1, 0, 0, 0, 834, 835, 1, 0, 0, 0, 835, 837, 1, 0, 0, 0, 836, 838, 3, 125, 62, 0, 837, 836, 1, 0, 0, 0, 837, 838, 1, 0, 0, 0, 838, 845, 1, 0, 0, 0, 839, 840, 5, 92, 0, 0, 840, 841, 5, 120, 0, 0, 841, 842, 1, 0, 0, 0, 842, 845, 3, 145, 72, 0, 843, 845, 3, 131, 65, 0, 844, 829, 1, 0, 0, 0, 844, 831, 1, 0, 0, 0, 844, 839, 1, 0, 0, 0, 844, 843, 1, 0, 0, 0, 845, 150, 1, 0, 0, 0, 846, 848, 7, 15, 0, 0, 847, 846, 1, 0, 0, 0, 848, 849, 1, 0, 0, 0, 849, 847, 1, 0, 0, 0, 849, 850, 1, 0, 0, 0, 850, 851, 1, 0, 0, 0, 851, 852, 6, 75, 0, 0, 852, 152, 1, 0, 0, 0, 853, 855, 5, 13, 0, 0, 854, 856, 5, 10, 0, 0, 855, 854, 1, 0, 0, 0, 855, 856, 1, 0, 0, 0, 856, 859, 1, 0, 0, 0, 857, 859, 5, 10, 0, 0, 858, 853, 1, 0, 0, 0, 858, 857, 1, 0, 0, 0, 859, 860, 1, 0, 0, 0, 860, 861, 6, 76, 0, 0, 861, 154, 1, 0, 0, 0, 59, 0, 193, 207, 229, 255, 290, 298, 314, 338, 349, 355, 360, 362, 393, 429, 465, 495, 533, 571, 597, 626, 632, 636, 641, 643, 653, 657, 662, 665, 669, 674, 680, 685, 690, 695, 704, 713, 724, 730, 734, 740, 768, 772, 777, 783, 788, 795, 799, 806, 809, 816, 821, 825, 834, 837, 844, 849, 855, 858, 1, 6, 0, 0] \ No newline at end of file diff --git a/internal/parser/planparserv2/generated/PlanLexer.tokens b/internal/parser/planparserv2/generated/PlanLexer.tokens index a198ec9a5db41..820be82be1273 100644 --- a/internal/parser/planparserv2/generated/PlanLexer.tokens +++ b/internal/parser/planparserv2/generated/PlanLexer.tokens @@ -14,41 +14,42 @@ NE=13 LIKE=14 EXISTS=15 TEXTMATCH=16 -ADD=17 -SUB=18 -MUL=19 -DIV=20 -MOD=21 -POW=22 -SHL=23 -SHR=24 -BAND=25 -BOR=26 -BXOR=27 -AND=28 -OR=29 -ISNULL=30 -ISNOTNULL=31 -BNOT=32 -NOT=33 -IN=34 -EmptyArray=35 -JSONContains=36 -JSONContainsAll=37 -JSONContainsAny=38 -ArrayContains=39 -ArrayContainsAll=40 -ArrayContainsAny=41 -ArrayLength=42 -BooleanConstant=43 -IntegerConstant=44 -FloatingConstant=45 -Identifier=46 -Meta=47 -StringLiteral=48 -JSONIdentifier=49 -Whitespace=50 -Newline=51 +PHRASEMATCH=17 +ADD=18 +SUB=19 +MUL=20 +DIV=21 +MOD=22 +POW=23 +SHL=24 +SHR=25 +BAND=26 +BOR=27 +BXOR=28 +AND=29 +OR=30 +ISNULL=31 +ISNOTNULL=32 +BNOT=33 +NOT=34 +IN=35 +EmptyArray=36 +JSONContains=37 +JSONContainsAll=38 +JSONContainsAny=39 +ArrayContains=40 +ArrayContainsAll=41 +ArrayContainsAny=42 +ArrayLength=43 +BooleanConstant=44 +IntegerConstant=45 +FloatingConstant=46 +Identifier=47 +Meta=48 +StringLiteral=49 +JSONIdentifier=50 +Whitespace=51 +Newline=52 '('=1 ')'=2 '['=3 @@ -62,16 +63,16 @@ Newline=51 '>='=11 '=='=12 '!='=13 -'+'=17 -'-'=18 -'*'=19 -'/'=20 -'%'=21 -'**'=22 -'<<'=23 -'>>'=24 -'&'=25 -'|'=26 -'^'=27 -'~'=32 -'$meta'=47 +'+'=18 +'-'=19 +'*'=20 +'/'=21 +'%'=22 +'**'=23 +'<<'=24 +'>>'=25 +'&'=26 +'|'=27 +'^'=28 +'~'=33 +'$meta'=48 diff --git a/internal/parser/planparserv2/generated/plan_base_visitor.go b/internal/parser/planparserv2/generated/plan_base_visitor.go index 4e7e3cb6b47c3..2c7fa5aaa2619 100644 --- a/internal/parser/planparserv2/generated/plan_base_visitor.go +++ b/internal/parser/planparserv2/generated/plan_base_visitor.go @@ -87,6 +87,10 @@ func (v *BasePlanVisitor) VisitAddSub(ctx *AddSubContext) interface{} { return v.VisitChildren(ctx) } +func (v *BasePlanVisitor) VisitPhraseMatch(ctx *PhraseMatchContext) interface{} { + return v.VisitChildren(ctx) +} + func (v *BasePlanVisitor) VisitRelational(ctx *RelationalContext) interface{} { return v.VisitChildren(ctx) } diff --git a/internal/parser/planparserv2/generated/plan_lexer.go b/internal/parser/planparserv2/generated/plan_lexer.go index 55ec744471d6a..9c916d8c75476 100644 --- a/internal/parser/planparserv2/generated/plan_lexer.go +++ b/internal/parser/planparserv2/generated/plan_lexer.go @@ -44,39 +44,39 @@ func planlexerLexerInit() { } staticData.LiteralNames = []string{ "", "'('", "')'", "'['", "','", "']'", "'{'", "'}'", "'<'", "'<='", - "'>'", "'>='", "'=='", "'!='", "", "", "", "'+'", "'-'", "'*'", "'/'", - "'%'", "'**'", "'<<'", "'>>'", "'&'", "'|'", "'^'", "", "", "", "", - "'~'", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "'$meta'", + "'>'", "'>='", "'=='", "'!='", "", "", "", "", "'+'", "'-'", "'*'", + "'/'", "'%'", "'**'", "'<<'", "'>>'", "'&'", "'|'", "'^'", "", "", "", + "", "'~'", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "'$meta'", } staticData.SymbolicNames = []string{ "", "", "", "", "", "", "LBRACE", "RBRACE", "LT", "LE", "GT", "GE", - "EQ", "NE", "LIKE", "EXISTS", "TEXTMATCH", "ADD", "SUB", "MUL", "DIV", - "MOD", "POW", "SHL", "SHR", "BAND", "BOR", "BXOR", "AND", "OR", "ISNULL", - "ISNOTNULL", "BNOT", "NOT", "IN", "EmptyArray", "JSONContains", "JSONContainsAll", - "JSONContainsAny", "ArrayContains", "ArrayContainsAll", "ArrayContainsAny", - "ArrayLength", "BooleanConstant", "IntegerConstant", "FloatingConstant", - "Identifier", "Meta", "StringLiteral", "JSONIdentifier", "Whitespace", - "Newline", - } - staticData.RuleNames = []string{ - "T__0", "T__1", "T__2", "T__3", "T__4", "LBRACE", "RBRACE", "LT", "LE", - "GT", "GE", "EQ", "NE", "LIKE", "EXISTS", "TEXTMATCH", "ADD", "SUB", + "EQ", "NE", "LIKE", "EXISTS", "TEXTMATCH", "PHRASEMATCH", "ADD", "SUB", "MUL", "DIV", "MOD", "POW", "SHL", "SHR", "BAND", "BOR", "BXOR", "AND", "OR", "ISNULL", "ISNOTNULL", "BNOT", "NOT", "IN", "EmptyArray", "JSONContains", "JSONContainsAll", "JSONContainsAny", "ArrayContains", "ArrayContainsAll", "ArrayContainsAny", "ArrayLength", "BooleanConstant", "IntegerConstant", "FloatingConstant", "Identifier", "Meta", "StringLiteral", "JSONIdentifier", - "EncodingPrefix", "DoubleSCharSequence", "SingleSCharSequence", "DoubleSChar", - "SingleSChar", "Nondigit", "Digit", "BinaryConstant", "DecimalConstant", - "OctalConstant", "HexadecimalConstant", "NonzeroDigit", "OctalDigit", - "HexadecimalDigit", "HexQuad", "UniversalCharacterName", "DecimalFloatingConstant", - "HexadecimalFloatingConstant", "FractionalConstant", "ExponentPart", - "DigitSequence", "HexadecimalFractionalConstant", "HexadecimalDigitSequence", + "Whitespace", "Newline", + } + staticData.RuleNames = []string{ + "T__0", "T__1", "T__2", "T__3", "T__4", "LBRACE", "RBRACE", "LT", "LE", + "GT", "GE", "EQ", "NE", "LIKE", "EXISTS", "TEXTMATCH", "PHRASEMATCH", + "ADD", "SUB", "MUL", "DIV", "MOD", "POW", "SHL", "SHR", "BAND", "BOR", + "BXOR", "AND", "OR", "ISNULL", "ISNOTNULL", "BNOT", "NOT", "IN", "EmptyArray", + "JSONContains", "JSONContainsAll", "JSONContainsAny", "ArrayContains", + "ArrayContainsAll", "ArrayContainsAny", "ArrayLength", "BooleanConstant", + "IntegerConstant", "FloatingConstant", "Identifier", "Meta", "StringLiteral", + "JSONIdentifier", "EncodingPrefix", "DoubleSCharSequence", "SingleSCharSequence", + "DoubleSChar", "SingleSChar", "Nondigit", "Digit", "BinaryConstant", + "DecimalConstant", "OctalConstant", "HexadecimalConstant", "NonzeroDigit", + "OctalDigit", "HexadecimalDigit", "HexQuad", "UniversalCharacterName", + "DecimalFloatingConstant", "HexadecimalFloatingConstant", "FractionalConstant", + "ExponentPart", "DigitSequence", "HexadecimalFractionalConstant", "HexadecimalDigitSequence", "BinaryExponentPart", "EscapeSequence", "Whitespace", "Newline", } staticData.PredictionContextCache = antlr.NewPredictionContextCache() staticData.serializedATN = []int32{ - 4, 0, 51, 834, 6, -1, 2, 0, 7, 0, 2, 1, 7, 1, 2, 2, 7, 2, 2, 3, 7, 3, 2, + 4, 0, 52, 862, 6, -1, 2, 0, 7, 0, 2, 1, 7, 1, 2, 2, 7, 2, 2, 3, 7, 3, 2, 4, 7, 4, 2, 5, 7, 5, 2, 6, 7, 6, 2, 7, 7, 7, 2, 8, 7, 8, 2, 9, 7, 9, 2, 10, 7, 10, 2, 11, 7, 11, 2, 12, 7, 12, 2, 13, 7, 13, 2, 14, 7, 14, 2, 15, 7, 15, 2, 16, 7, 16, 2, 17, 7, 17, 2, 18, 7, 18, 2, 19, 7, 19, 2, 20, 7, @@ -90,370 +90,382 @@ func planlexerLexerInit() { 7, 57, 2, 58, 7, 58, 2, 59, 7, 59, 2, 60, 7, 60, 2, 61, 7, 61, 2, 62, 7, 62, 2, 63, 7, 63, 2, 64, 7, 64, 2, 65, 7, 65, 2, 66, 7, 66, 2, 67, 7, 67, 2, 68, 7, 68, 2, 69, 7, 69, 2, 70, 7, 70, 2, 71, 7, 71, 2, 72, 7, 72, 2, - 73, 7, 73, 2, 74, 7, 74, 2, 75, 7, 75, 1, 0, 1, 0, 1, 1, 1, 1, 1, 2, 1, - 2, 1, 3, 1, 3, 1, 4, 1, 4, 1, 5, 1, 5, 1, 6, 1, 6, 1, 7, 1, 7, 1, 8, 1, - 8, 1, 8, 1, 9, 1, 9, 1, 10, 1, 10, 1, 10, 1, 11, 1, 11, 1, 11, 1, 12, 1, - 12, 1, 12, 1, 13, 1, 13, 1, 13, 1, 13, 1, 13, 1, 13, 1, 13, 1, 13, 3, 13, - 192, 8, 13, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, - 14, 1, 14, 1, 14, 1, 14, 3, 14, 206, 8, 14, 1, 15, 1, 15, 1, 15, 1, 15, - 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, - 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 3, 15, 228, 8, 15, 1, 16, 1, 16, - 1, 17, 1, 17, 1, 18, 1, 18, 1, 19, 1, 19, 1, 20, 1, 20, 1, 21, 1, 21, 1, - 21, 1, 22, 1, 22, 1, 22, 1, 23, 1, 23, 1, 23, 1, 24, 1, 24, 1, 25, 1, 25, - 1, 26, 1, 26, 1, 27, 1, 27, 1, 27, 1, 27, 1, 27, 1, 27, 1, 27, 1, 27, 3, - 27, 263, 8, 27, 1, 28, 1, 28, 1, 28, 1, 28, 1, 28, 1, 28, 3, 28, 271, 8, - 28, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, - 1, 29, 1, 29, 1, 29, 1, 29, 3, 29, 287, 8, 29, 1, 30, 1, 30, 1, 30, 1, - 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, - 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 3, 30, 311, 8, - 30, 1, 31, 1, 31, 1, 32, 1, 32, 1, 32, 1, 32, 1, 32, 1, 32, 1, 32, 3, 32, - 322, 8, 32, 1, 33, 1, 33, 1, 33, 1, 33, 3, 33, 328, 8, 33, 1, 34, 1, 34, - 1, 34, 5, 34, 333, 8, 34, 10, 34, 12, 34, 336, 9, 34, 1, 34, 1, 34, 1, - 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, - 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, - 35, 1, 35, 1, 35, 1, 35, 1, 35, 3, 35, 366, 8, 35, 1, 36, 1, 36, 1, 36, - 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, + 73, 7, 73, 2, 74, 7, 74, 2, 75, 7, 75, 2, 76, 7, 76, 1, 0, 1, 0, 1, 1, + 1, 1, 1, 2, 1, 2, 1, 3, 1, 3, 1, 4, 1, 4, 1, 5, 1, 5, 1, 6, 1, 6, 1, 7, + 1, 7, 1, 8, 1, 8, 1, 8, 1, 9, 1, 9, 1, 10, 1, 10, 1, 10, 1, 11, 1, 11, + 1, 11, 1, 12, 1, 12, 1, 12, 1, 13, 1, 13, 1, 13, 1, 13, 1, 13, 1, 13, 1, + 13, 1, 13, 3, 13, 194, 8, 13, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, + 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 3, 14, 208, 8, 14, 1, 15, 1, + 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, + 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 3, 15, 230, 8, + 15, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, + 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, + 16, 1, 16, 1, 16, 1, 16, 3, 16, 256, 8, 16, 1, 17, 1, 17, 1, 18, 1, 18, + 1, 19, 1, 19, 1, 20, 1, 20, 1, 21, 1, 21, 1, 22, 1, 22, 1, 22, 1, 23, 1, + 23, 1, 23, 1, 24, 1, 24, 1, 24, 1, 25, 1, 25, 1, 26, 1, 26, 1, 27, 1, 27, + 1, 28, 1, 28, 1, 28, 1, 28, 1, 28, 1, 28, 1, 28, 1, 28, 3, 28, 291, 8, + 28, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 3, 29, 299, 8, 29, 1, 30, + 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, + 30, 1, 30, 1, 30, 3, 30, 315, 8, 30, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, + 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, + 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 3, 31, 339, 8, 31, 1, 32, + 1, 32, 1, 33, 1, 33, 1, 33, 1, 33, 1, 33, 1, 33, 1, 33, 3, 33, 350, 8, + 33, 1, 34, 1, 34, 1, 34, 1, 34, 3, 34, 356, 8, 34, 1, 35, 1, 35, 1, 35, + 5, 35, 361, 8, 35, 10, 35, 12, 35, 364, 9, 35, 1, 35, 1, 35, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, - 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 3, - 36, 402, 8, 36, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, + 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, + 36, 1, 36, 1, 36, 1, 36, 3, 36, 394, 8, 36, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, - 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 3, 37, 438, 8, 37, 1, 38, 1, 38, 1, + 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 3, 37, 430, + 8, 37, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, - 38, 1, 38, 1, 38, 1, 38, 1, 38, 3, 38, 468, 8, 38, 1, 39, 1, 39, 1, 39, + 38, 1, 38, 1, 38, 1, 38, 3, 38, 466, 8, 38, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, - 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, - 39, 1, 39, 3, 39, 506, 8, 39, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, + 1, 39, 1, 39, 1, 39, 3, 39, 496, 8, 39, 1, 40, 1, 40, 1, 40, 1, 40, 1, + 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, - 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 3, 40, 544, - 8, 40, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, + 3, 40, 534, 8, 40, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, - 1, 41, 1, 41, 1, 41, 1, 41, 3, 41, 570, 8, 41, 1, 42, 1, 42, 1, 42, 1, - 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, + 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, + 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 3, 41, 572, 8, 41, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, - 42, 1, 42, 1, 42, 3, 42, 599, 8, 42, 1, 43, 1, 43, 1, 43, 1, 43, 3, 43, - 605, 8, 43, 1, 44, 1, 44, 3, 44, 609, 8, 44, 1, 45, 1, 45, 1, 45, 5, 45, - 614, 8, 45, 10, 45, 12, 45, 617, 9, 45, 1, 46, 1, 46, 1, 46, 1, 46, 1, - 46, 1, 46, 1, 47, 3, 47, 626, 8, 47, 1, 47, 1, 47, 3, 47, 630, 8, 47, 1, - 47, 1, 47, 1, 47, 3, 47, 635, 8, 47, 1, 47, 3, 47, 638, 8, 47, 1, 48, 1, - 48, 3, 48, 642, 8, 48, 1, 48, 1, 48, 1, 48, 3, 48, 647, 8, 48, 1, 48, 1, - 48, 4, 48, 651, 8, 48, 11, 48, 12, 48, 652, 1, 49, 1, 49, 1, 49, 3, 49, - 658, 8, 49, 1, 50, 4, 50, 661, 8, 50, 11, 50, 12, 50, 662, 1, 51, 4, 51, - 666, 8, 51, 11, 51, 12, 51, 667, 1, 52, 1, 52, 1, 52, 1, 52, 1, 52, 1, - 52, 1, 52, 3, 52, 677, 8, 52, 1, 53, 1, 53, 1, 53, 1, 53, 1, 53, 1, 53, - 1, 53, 3, 53, 686, 8, 53, 1, 54, 1, 54, 1, 55, 1, 55, 1, 56, 1, 56, 1, - 56, 4, 56, 695, 8, 56, 11, 56, 12, 56, 696, 1, 57, 1, 57, 5, 57, 701, 8, - 57, 10, 57, 12, 57, 704, 9, 57, 1, 57, 3, 57, 707, 8, 57, 1, 58, 1, 58, - 5, 58, 711, 8, 58, 10, 58, 12, 58, 714, 9, 58, 1, 59, 1, 59, 1, 59, 1, - 59, 1, 60, 1, 60, 1, 61, 1, 61, 1, 62, 1, 62, 1, 63, 1, 63, 1, 63, 1, 63, - 1, 63, 1, 64, 1, 64, 1, 64, 1, 64, 1, 64, 1, 64, 1, 64, 1, 64, 1, 64, 1, - 64, 3, 64, 741, 8, 64, 1, 65, 1, 65, 3, 65, 745, 8, 65, 1, 65, 1, 65, 1, - 65, 3, 65, 750, 8, 65, 1, 66, 1, 66, 1, 66, 1, 66, 3, 66, 756, 8, 66, 1, - 66, 1, 66, 1, 67, 3, 67, 761, 8, 67, 1, 67, 1, 67, 1, 67, 1, 67, 1, 67, - 3, 67, 768, 8, 67, 1, 68, 1, 68, 3, 68, 772, 8, 68, 1, 68, 1, 68, 1, 69, - 4, 69, 777, 8, 69, 11, 69, 12, 69, 778, 1, 70, 3, 70, 782, 8, 70, 1, 70, - 1, 70, 1, 70, 1, 70, 1, 70, 3, 70, 789, 8, 70, 1, 71, 4, 71, 792, 8, 71, - 11, 71, 12, 71, 793, 1, 72, 1, 72, 3, 72, 798, 8, 72, 1, 72, 1, 72, 1, - 73, 1, 73, 1, 73, 1, 73, 1, 73, 3, 73, 807, 8, 73, 1, 73, 3, 73, 810, 8, - 73, 1, 73, 1, 73, 1, 73, 1, 73, 1, 73, 3, 73, 817, 8, 73, 1, 74, 4, 74, - 820, 8, 74, 11, 74, 12, 74, 821, 1, 74, 1, 74, 1, 75, 1, 75, 3, 75, 828, - 8, 75, 1, 75, 3, 75, 831, 8, 75, 1, 75, 1, 75, 0, 0, 76, 1, 1, 3, 2, 5, - 3, 7, 4, 9, 5, 11, 6, 13, 7, 15, 8, 17, 9, 19, 10, 21, 11, 23, 12, 25, - 13, 27, 14, 29, 15, 31, 16, 33, 17, 35, 18, 37, 19, 39, 20, 41, 21, 43, - 22, 45, 23, 47, 24, 49, 25, 51, 26, 53, 27, 55, 28, 57, 29, 59, 30, 61, - 31, 63, 32, 65, 33, 67, 34, 69, 35, 71, 36, 73, 37, 75, 38, 77, 39, 79, - 40, 81, 41, 83, 42, 85, 43, 87, 44, 89, 45, 91, 46, 93, 47, 95, 48, 97, - 49, 99, 0, 101, 0, 103, 0, 105, 0, 107, 0, 109, 0, 111, 0, 113, 0, 115, - 0, 117, 0, 119, 0, 121, 0, 123, 0, 125, 0, 127, 0, 129, 0, 131, 0, 133, - 0, 135, 0, 137, 0, 139, 0, 141, 0, 143, 0, 145, 0, 147, 0, 149, 50, 151, - 51, 1, 0, 16, 3, 0, 76, 76, 85, 85, 117, 117, 4, 0, 10, 10, 13, 13, 34, - 34, 92, 92, 4, 0, 10, 10, 13, 13, 39, 39, 92, 92, 3, 0, 65, 90, 95, 95, - 97, 122, 1, 0, 48, 57, 2, 0, 66, 66, 98, 98, 1, 0, 48, 49, 2, 0, 88, 88, - 120, 120, 1, 0, 49, 57, 1, 0, 48, 55, 3, 0, 48, 57, 65, 70, 97, 102, 2, - 0, 69, 69, 101, 101, 2, 0, 43, 43, 45, 45, 2, 0, 80, 80, 112, 112, 10, - 0, 34, 34, 39, 39, 63, 63, 92, 92, 97, 98, 102, 102, 110, 110, 114, 114, - 116, 116, 118, 118, 2, 0, 9, 9, 32, 32, 880, 0, 1, 1, 0, 0, 0, 0, 3, 1, - 0, 0, 0, 0, 5, 1, 0, 0, 0, 0, 7, 1, 0, 0, 0, 0, 9, 1, 0, 0, 0, 0, 11, 1, - 0, 0, 0, 0, 13, 1, 0, 0, 0, 0, 15, 1, 0, 0, 0, 0, 17, 1, 0, 0, 0, 0, 19, - 1, 0, 0, 0, 0, 21, 1, 0, 0, 0, 0, 23, 1, 0, 0, 0, 0, 25, 1, 0, 0, 0, 0, - 27, 1, 0, 0, 0, 0, 29, 1, 0, 0, 0, 0, 31, 1, 0, 0, 0, 0, 33, 1, 0, 0, 0, - 0, 35, 1, 0, 0, 0, 0, 37, 1, 0, 0, 0, 0, 39, 1, 0, 0, 0, 0, 41, 1, 0, 0, - 0, 0, 43, 1, 0, 0, 0, 0, 45, 1, 0, 0, 0, 0, 47, 1, 0, 0, 0, 0, 49, 1, 0, - 0, 0, 0, 51, 1, 0, 0, 0, 0, 53, 1, 0, 0, 0, 0, 55, 1, 0, 0, 0, 0, 57, 1, - 0, 0, 0, 0, 59, 1, 0, 0, 0, 0, 61, 1, 0, 0, 0, 0, 63, 1, 0, 0, 0, 0, 65, - 1, 0, 0, 0, 0, 67, 1, 0, 0, 0, 0, 69, 1, 0, 0, 0, 0, 71, 1, 0, 0, 0, 0, - 73, 1, 0, 0, 0, 0, 75, 1, 0, 0, 0, 0, 77, 1, 0, 0, 0, 0, 79, 1, 0, 0, 0, - 0, 81, 1, 0, 0, 0, 0, 83, 1, 0, 0, 0, 0, 85, 1, 0, 0, 0, 0, 87, 1, 0, 0, - 0, 0, 89, 1, 0, 0, 0, 0, 91, 1, 0, 0, 0, 0, 93, 1, 0, 0, 0, 0, 95, 1, 0, - 0, 0, 0, 97, 1, 0, 0, 0, 0, 149, 1, 0, 0, 0, 0, 151, 1, 0, 0, 0, 1, 153, - 1, 0, 0, 0, 3, 155, 1, 0, 0, 0, 5, 157, 1, 0, 0, 0, 7, 159, 1, 0, 0, 0, - 9, 161, 1, 0, 0, 0, 11, 163, 1, 0, 0, 0, 13, 165, 1, 0, 0, 0, 15, 167, - 1, 0, 0, 0, 17, 169, 1, 0, 0, 0, 19, 172, 1, 0, 0, 0, 21, 174, 1, 0, 0, - 0, 23, 177, 1, 0, 0, 0, 25, 180, 1, 0, 0, 0, 27, 191, 1, 0, 0, 0, 29, 205, - 1, 0, 0, 0, 31, 227, 1, 0, 0, 0, 33, 229, 1, 0, 0, 0, 35, 231, 1, 0, 0, - 0, 37, 233, 1, 0, 0, 0, 39, 235, 1, 0, 0, 0, 41, 237, 1, 0, 0, 0, 43, 239, - 1, 0, 0, 0, 45, 242, 1, 0, 0, 0, 47, 245, 1, 0, 0, 0, 49, 248, 1, 0, 0, - 0, 51, 250, 1, 0, 0, 0, 53, 252, 1, 0, 0, 0, 55, 262, 1, 0, 0, 0, 57, 270, - 1, 0, 0, 0, 59, 286, 1, 0, 0, 0, 61, 310, 1, 0, 0, 0, 63, 312, 1, 0, 0, - 0, 65, 321, 1, 0, 0, 0, 67, 327, 1, 0, 0, 0, 69, 329, 1, 0, 0, 0, 71, 365, - 1, 0, 0, 0, 73, 401, 1, 0, 0, 0, 75, 437, 1, 0, 0, 0, 77, 467, 1, 0, 0, - 0, 79, 505, 1, 0, 0, 0, 81, 543, 1, 0, 0, 0, 83, 569, 1, 0, 0, 0, 85, 598, - 1, 0, 0, 0, 87, 604, 1, 0, 0, 0, 89, 608, 1, 0, 0, 0, 91, 610, 1, 0, 0, - 0, 93, 618, 1, 0, 0, 0, 95, 625, 1, 0, 0, 0, 97, 641, 1, 0, 0, 0, 99, 657, - 1, 0, 0, 0, 101, 660, 1, 0, 0, 0, 103, 665, 1, 0, 0, 0, 105, 676, 1, 0, - 0, 0, 107, 685, 1, 0, 0, 0, 109, 687, 1, 0, 0, 0, 111, 689, 1, 0, 0, 0, - 113, 691, 1, 0, 0, 0, 115, 706, 1, 0, 0, 0, 117, 708, 1, 0, 0, 0, 119, - 715, 1, 0, 0, 0, 121, 719, 1, 0, 0, 0, 123, 721, 1, 0, 0, 0, 125, 723, - 1, 0, 0, 0, 127, 725, 1, 0, 0, 0, 129, 740, 1, 0, 0, 0, 131, 749, 1, 0, - 0, 0, 133, 751, 1, 0, 0, 0, 135, 767, 1, 0, 0, 0, 137, 769, 1, 0, 0, 0, - 139, 776, 1, 0, 0, 0, 141, 788, 1, 0, 0, 0, 143, 791, 1, 0, 0, 0, 145, - 795, 1, 0, 0, 0, 147, 816, 1, 0, 0, 0, 149, 819, 1, 0, 0, 0, 151, 830, - 1, 0, 0, 0, 153, 154, 5, 40, 0, 0, 154, 2, 1, 0, 0, 0, 155, 156, 5, 41, - 0, 0, 156, 4, 1, 0, 0, 0, 157, 158, 5, 91, 0, 0, 158, 6, 1, 0, 0, 0, 159, - 160, 5, 44, 0, 0, 160, 8, 1, 0, 0, 0, 161, 162, 5, 93, 0, 0, 162, 10, 1, - 0, 0, 0, 163, 164, 5, 123, 0, 0, 164, 12, 1, 0, 0, 0, 165, 166, 5, 125, - 0, 0, 166, 14, 1, 0, 0, 0, 167, 168, 5, 60, 0, 0, 168, 16, 1, 0, 0, 0, - 169, 170, 5, 60, 0, 0, 170, 171, 5, 61, 0, 0, 171, 18, 1, 0, 0, 0, 172, - 173, 5, 62, 0, 0, 173, 20, 1, 0, 0, 0, 174, 175, 5, 62, 0, 0, 175, 176, - 5, 61, 0, 0, 176, 22, 1, 0, 0, 0, 177, 178, 5, 61, 0, 0, 178, 179, 5, 61, - 0, 0, 179, 24, 1, 0, 0, 0, 180, 181, 5, 33, 0, 0, 181, 182, 5, 61, 0, 0, - 182, 26, 1, 0, 0, 0, 183, 184, 5, 108, 0, 0, 184, 185, 5, 105, 0, 0, 185, - 186, 5, 107, 0, 0, 186, 192, 5, 101, 0, 0, 187, 188, 5, 76, 0, 0, 188, - 189, 5, 73, 0, 0, 189, 190, 5, 75, 0, 0, 190, 192, 5, 69, 0, 0, 191, 183, - 1, 0, 0, 0, 191, 187, 1, 0, 0, 0, 192, 28, 1, 0, 0, 0, 193, 194, 5, 101, - 0, 0, 194, 195, 5, 120, 0, 0, 195, 196, 5, 105, 0, 0, 196, 197, 5, 115, - 0, 0, 197, 198, 5, 116, 0, 0, 198, 206, 5, 115, 0, 0, 199, 200, 5, 69, - 0, 0, 200, 201, 5, 88, 0, 0, 201, 202, 5, 73, 0, 0, 202, 203, 5, 83, 0, - 0, 203, 204, 5, 84, 0, 0, 204, 206, 5, 83, 0, 0, 205, 193, 1, 0, 0, 0, - 205, 199, 1, 0, 0, 0, 206, 30, 1, 0, 0, 0, 207, 208, 5, 116, 0, 0, 208, - 209, 5, 101, 0, 0, 209, 210, 5, 120, 0, 0, 210, 211, 5, 116, 0, 0, 211, - 212, 5, 95, 0, 0, 212, 213, 5, 109, 0, 0, 213, 214, 5, 97, 0, 0, 214, 215, - 5, 116, 0, 0, 215, 216, 5, 99, 0, 0, 216, 228, 5, 104, 0, 0, 217, 218, - 5, 84, 0, 0, 218, 219, 5, 69, 0, 0, 219, 220, 5, 88, 0, 0, 220, 221, 5, - 84, 0, 0, 221, 222, 5, 95, 0, 0, 222, 223, 5, 77, 0, 0, 223, 224, 5, 65, - 0, 0, 224, 225, 5, 84, 0, 0, 225, 226, 5, 67, 0, 0, 226, 228, 5, 72, 0, - 0, 227, 207, 1, 0, 0, 0, 227, 217, 1, 0, 0, 0, 228, 32, 1, 0, 0, 0, 229, - 230, 5, 43, 0, 0, 230, 34, 1, 0, 0, 0, 231, 232, 5, 45, 0, 0, 232, 36, - 1, 0, 0, 0, 233, 234, 5, 42, 0, 0, 234, 38, 1, 0, 0, 0, 235, 236, 5, 47, - 0, 0, 236, 40, 1, 0, 0, 0, 237, 238, 5, 37, 0, 0, 238, 42, 1, 0, 0, 0, - 239, 240, 5, 42, 0, 0, 240, 241, 5, 42, 0, 0, 241, 44, 1, 0, 0, 0, 242, - 243, 5, 60, 0, 0, 243, 244, 5, 60, 0, 0, 244, 46, 1, 0, 0, 0, 245, 246, - 5, 62, 0, 0, 246, 247, 5, 62, 0, 0, 247, 48, 1, 0, 0, 0, 248, 249, 5, 38, - 0, 0, 249, 50, 1, 0, 0, 0, 250, 251, 5, 124, 0, 0, 251, 52, 1, 0, 0, 0, - 252, 253, 5, 94, 0, 0, 253, 54, 1, 0, 0, 0, 254, 255, 5, 38, 0, 0, 255, - 263, 5, 38, 0, 0, 256, 257, 5, 97, 0, 0, 257, 258, 5, 110, 0, 0, 258, 263, - 5, 100, 0, 0, 259, 260, 5, 65, 0, 0, 260, 261, 5, 78, 0, 0, 261, 263, 5, - 68, 0, 0, 262, 254, 1, 0, 0, 0, 262, 256, 1, 0, 0, 0, 262, 259, 1, 0, 0, - 0, 263, 56, 1, 0, 0, 0, 264, 265, 5, 124, 0, 0, 265, 271, 5, 124, 0, 0, - 266, 267, 5, 111, 0, 0, 267, 271, 5, 114, 0, 0, 268, 269, 5, 79, 0, 0, - 269, 271, 5, 82, 0, 0, 270, 264, 1, 0, 0, 0, 270, 266, 1, 0, 0, 0, 270, - 268, 1, 0, 0, 0, 271, 58, 1, 0, 0, 0, 272, 273, 5, 105, 0, 0, 273, 274, - 5, 115, 0, 0, 274, 275, 5, 32, 0, 0, 275, 276, 5, 110, 0, 0, 276, 277, - 5, 117, 0, 0, 277, 278, 5, 108, 0, 0, 278, 287, 5, 108, 0, 0, 279, 280, - 5, 73, 0, 0, 280, 281, 5, 83, 0, 0, 281, 282, 5, 32, 0, 0, 282, 283, 5, - 78, 0, 0, 283, 284, 5, 85, 0, 0, 284, 285, 5, 76, 0, 0, 285, 287, 5, 76, - 0, 0, 286, 272, 1, 0, 0, 0, 286, 279, 1, 0, 0, 0, 287, 60, 1, 0, 0, 0, - 288, 289, 5, 105, 0, 0, 289, 290, 5, 115, 0, 0, 290, 291, 5, 32, 0, 0, - 291, 292, 5, 110, 0, 0, 292, 293, 5, 111, 0, 0, 293, 294, 5, 116, 0, 0, - 294, 295, 5, 32, 0, 0, 295, 296, 5, 110, 0, 0, 296, 297, 5, 117, 0, 0, - 297, 298, 5, 108, 0, 0, 298, 311, 5, 108, 0, 0, 299, 300, 5, 73, 0, 0, - 300, 301, 5, 83, 0, 0, 301, 302, 5, 32, 0, 0, 302, 303, 5, 78, 0, 0, 303, - 304, 5, 79, 0, 0, 304, 305, 5, 84, 0, 0, 305, 306, 5, 32, 0, 0, 306, 307, - 5, 78, 0, 0, 307, 308, 5, 85, 0, 0, 308, 309, 5, 76, 0, 0, 309, 311, 5, - 76, 0, 0, 310, 288, 1, 0, 0, 0, 310, 299, 1, 0, 0, 0, 311, 62, 1, 0, 0, - 0, 312, 313, 5, 126, 0, 0, 313, 64, 1, 0, 0, 0, 314, 322, 5, 33, 0, 0, - 315, 316, 5, 110, 0, 0, 316, 317, 5, 111, 0, 0, 317, 322, 5, 116, 0, 0, - 318, 319, 5, 78, 0, 0, 319, 320, 5, 79, 0, 0, 320, 322, 5, 84, 0, 0, 321, - 314, 1, 0, 0, 0, 321, 315, 1, 0, 0, 0, 321, 318, 1, 0, 0, 0, 322, 66, 1, - 0, 0, 0, 323, 324, 5, 105, 0, 0, 324, 328, 5, 110, 0, 0, 325, 326, 5, 73, - 0, 0, 326, 328, 5, 78, 0, 0, 327, 323, 1, 0, 0, 0, 327, 325, 1, 0, 0, 0, - 328, 68, 1, 0, 0, 0, 329, 334, 5, 91, 0, 0, 330, 333, 3, 149, 74, 0, 331, - 333, 3, 151, 75, 0, 332, 330, 1, 0, 0, 0, 332, 331, 1, 0, 0, 0, 333, 336, - 1, 0, 0, 0, 334, 332, 1, 0, 0, 0, 334, 335, 1, 0, 0, 0, 335, 337, 1, 0, - 0, 0, 336, 334, 1, 0, 0, 0, 337, 338, 5, 93, 0, 0, 338, 70, 1, 0, 0, 0, - 339, 340, 5, 106, 0, 0, 340, 341, 5, 115, 0, 0, 341, 342, 5, 111, 0, 0, - 342, 343, 5, 110, 0, 0, 343, 344, 5, 95, 0, 0, 344, 345, 5, 99, 0, 0, 345, - 346, 5, 111, 0, 0, 346, 347, 5, 110, 0, 0, 347, 348, 5, 116, 0, 0, 348, - 349, 5, 97, 0, 0, 349, 350, 5, 105, 0, 0, 350, 351, 5, 110, 0, 0, 351, - 366, 5, 115, 0, 0, 352, 353, 5, 74, 0, 0, 353, 354, 5, 83, 0, 0, 354, 355, - 5, 79, 0, 0, 355, 356, 5, 78, 0, 0, 356, 357, 5, 95, 0, 0, 357, 358, 5, - 67, 0, 0, 358, 359, 5, 79, 0, 0, 359, 360, 5, 78, 0, 0, 360, 361, 5, 84, - 0, 0, 361, 362, 5, 65, 0, 0, 362, 363, 5, 73, 0, 0, 363, 364, 5, 78, 0, - 0, 364, 366, 5, 83, 0, 0, 365, 339, 1, 0, 0, 0, 365, 352, 1, 0, 0, 0, 366, - 72, 1, 0, 0, 0, 367, 368, 5, 106, 0, 0, 368, 369, 5, 115, 0, 0, 369, 370, - 5, 111, 0, 0, 370, 371, 5, 110, 0, 0, 371, 372, 5, 95, 0, 0, 372, 373, - 5, 99, 0, 0, 373, 374, 5, 111, 0, 0, 374, 375, 5, 110, 0, 0, 375, 376, - 5, 116, 0, 0, 376, 377, 5, 97, 0, 0, 377, 378, 5, 105, 0, 0, 378, 379, - 5, 110, 0, 0, 379, 380, 5, 115, 0, 0, 380, 381, 5, 95, 0, 0, 381, 382, - 5, 97, 0, 0, 382, 383, 5, 108, 0, 0, 383, 402, 5, 108, 0, 0, 384, 385, - 5, 74, 0, 0, 385, 386, 5, 83, 0, 0, 386, 387, 5, 79, 0, 0, 387, 388, 5, - 78, 0, 0, 388, 389, 5, 95, 0, 0, 389, 390, 5, 67, 0, 0, 390, 391, 5, 79, - 0, 0, 391, 392, 5, 78, 0, 0, 392, 393, 5, 84, 0, 0, 393, 394, 5, 65, 0, - 0, 394, 395, 5, 73, 0, 0, 395, 396, 5, 78, 0, 0, 396, 397, 5, 83, 0, 0, - 397, 398, 5, 95, 0, 0, 398, 399, 5, 65, 0, 0, 399, 400, 5, 76, 0, 0, 400, - 402, 5, 76, 0, 0, 401, 367, 1, 0, 0, 0, 401, 384, 1, 0, 0, 0, 402, 74, - 1, 0, 0, 0, 403, 404, 5, 106, 0, 0, 404, 405, 5, 115, 0, 0, 405, 406, 5, - 111, 0, 0, 406, 407, 5, 110, 0, 0, 407, 408, 5, 95, 0, 0, 408, 409, 5, - 99, 0, 0, 409, 410, 5, 111, 0, 0, 410, 411, 5, 110, 0, 0, 411, 412, 5, - 116, 0, 0, 412, 413, 5, 97, 0, 0, 413, 414, 5, 105, 0, 0, 414, 415, 5, - 110, 0, 0, 415, 416, 5, 115, 0, 0, 416, 417, 5, 95, 0, 0, 417, 418, 5, - 97, 0, 0, 418, 419, 5, 110, 0, 0, 419, 438, 5, 121, 0, 0, 420, 421, 5, - 74, 0, 0, 421, 422, 5, 83, 0, 0, 422, 423, 5, 79, 0, 0, 423, 424, 5, 78, - 0, 0, 424, 425, 5, 95, 0, 0, 425, 426, 5, 67, 0, 0, 426, 427, 5, 79, 0, - 0, 427, 428, 5, 78, 0, 0, 428, 429, 5, 84, 0, 0, 429, 430, 5, 65, 0, 0, - 430, 431, 5, 73, 0, 0, 431, 432, 5, 78, 0, 0, 432, 433, 5, 83, 0, 0, 433, - 434, 5, 95, 0, 0, 434, 435, 5, 65, 0, 0, 435, 436, 5, 78, 0, 0, 436, 438, - 5, 89, 0, 0, 437, 403, 1, 0, 0, 0, 437, 420, 1, 0, 0, 0, 438, 76, 1, 0, - 0, 0, 439, 440, 5, 97, 0, 0, 440, 441, 5, 114, 0, 0, 441, 442, 5, 114, - 0, 0, 442, 443, 5, 97, 0, 0, 443, 444, 5, 121, 0, 0, 444, 445, 5, 95, 0, - 0, 445, 446, 5, 99, 0, 0, 446, 447, 5, 111, 0, 0, 447, 448, 5, 110, 0, - 0, 448, 449, 5, 116, 0, 0, 449, 450, 5, 97, 0, 0, 450, 451, 5, 105, 0, - 0, 451, 452, 5, 110, 0, 0, 452, 468, 5, 115, 0, 0, 453, 454, 5, 65, 0, - 0, 454, 455, 5, 82, 0, 0, 455, 456, 5, 82, 0, 0, 456, 457, 5, 65, 0, 0, - 457, 458, 5, 89, 0, 0, 458, 459, 5, 95, 0, 0, 459, 460, 5, 67, 0, 0, 460, - 461, 5, 79, 0, 0, 461, 462, 5, 78, 0, 0, 462, 463, 5, 84, 0, 0, 463, 464, - 5, 65, 0, 0, 464, 465, 5, 73, 0, 0, 465, 466, 5, 78, 0, 0, 466, 468, 5, - 83, 0, 0, 467, 439, 1, 0, 0, 0, 467, 453, 1, 0, 0, 0, 468, 78, 1, 0, 0, - 0, 469, 470, 5, 97, 0, 0, 470, 471, 5, 114, 0, 0, 471, 472, 5, 114, 0, - 0, 472, 473, 5, 97, 0, 0, 473, 474, 5, 121, 0, 0, 474, 475, 5, 95, 0, 0, - 475, 476, 5, 99, 0, 0, 476, 477, 5, 111, 0, 0, 477, 478, 5, 110, 0, 0, - 478, 479, 5, 116, 0, 0, 479, 480, 5, 97, 0, 0, 480, 481, 5, 105, 0, 0, - 481, 482, 5, 110, 0, 0, 482, 483, 5, 115, 0, 0, 483, 484, 5, 95, 0, 0, - 484, 485, 5, 97, 0, 0, 485, 486, 5, 108, 0, 0, 486, 506, 5, 108, 0, 0, - 487, 488, 5, 65, 0, 0, 488, 489, 5, 82, 0, 0, 489, 490, 5, 82, 0, 0, 490, - 491, 5, 65, 0, 0, 491, 492, 5, 89, 0, 0, 492, 493, 5, 95, 0, 0, 493, 494, - 5, 67, 0, 0, 494, 495, 5, 79, 0, 0, 495, 496, 5, 78, 0, 0, 496, 497, 5, - 84, 0, 0, 497, 498, 5, 65, 0, 0, 498, 499, 5, 73, 0, 0, 499, 500, 5, 78, - 0, 0, 500, 501, 5, 83, 0, 0, 501, 502, 5, 95, 0, 0, 502, 503, 5, 65, 0, - 0, 503, 504, 5, 76, 0, 0, 504, 506, 5, 76, 0, 0, 505, 469, 1, 0, 0, 0, - 505, 487, 1, 0, 0, 0, 506, 80, 1, 0, 0, 0, 507, 508, 5, 97, 0, 0, 508, - 509, 5, 114, 0, 0, 509, 510, 5, 114, 0, 0, 510, 511, 5, 97, 0, 0, 511, - 512, 5, 121, 0, 0, 512, 513, 5, 95, 0, 0, 513, 514, 5, 99, 0, 0, 514, 515, - 5, 111, 0, 0, 515, 516, 5, 110, 0, 0, 516, 517, 5, 116, 0, 0, 517, 518, - 5, 97, 0, 0, 518, 519, 5, 105, 0, 0, 519, 520, 5, 110, 0, 0, 520, 521, - 5, 115, 0, 0, 521, 522, 5, 95, 0, 0, 522, 523, 5, 97, 0, 0, 523, 524, 5, - 110, 0, 0, 524, 544, 5, 121, 0, 0, 525, 526, 5, 65, 0, 0, 526, 527, 5, - 82, 0, 0, 527, 528, 5, 82, 0, 0, 528, 529, 5, 65, 0, 0, 529, 530, 5, 89, - 0, 0, 530, 531, 5, 95, 0, 0, 531, 532, 5, 67, 0, 0, 532, 533, 5, 79, 0, - 0, 533, 534, 5, 78, 0, 0, 534, 535, 5, 84, 0, 0, 535, 536, 5, 65, 0, 0, - 536, 537, 5, 73, 0, 0, 537, 538, 5, 78, 0, 0, 538, 539, 5, 83, 0, 0, 539, - 540, 5, 95, 0, 0, 540, 541, 5, 65, 0, 0, 541, 542, 5, 78, 0, 0, 542, 544, - 5, 89, 0, 0, 543, 507, 1, 0, 0, 0, 543, 525, 1, 0, 0, 0, 544, 82, 1, 0, - 0, 0, 545, 546, 5, 97, 0, 0, 546, 547, 5, 114, 0, 0, 547, 548, 5, 114, - 0, 0, 548, 549, 5, 97, 0, 0, 549, 550, 5, 121, 0, 0, 550, 551, 5, 95, 0, - 0, 551, 552, 5, 108, 0, 0, 552, 553, 5, 101, 0, 0, 553, 554, 5, 110, 0, - 0, 554, 555, 5, 103, 0, 0, 555, 556, 5, 116, 0, 0, 556, 570, 5, 104, 0, - 0, 557, 558, 5, 65, 0, 0, 558, 559, 5, 82, 0, 0, 559, 560, 5, 82, 0, 0, - 560, 561, 5, 65, 0, 0, 561, 562, 5, 89, 0, 0, 562, 563, 5, 95, 0, 0, 563, - 564, 5, 76, 0, 0, 564, 565, 5, 69, 0, 0, 565, 566, 5, 78, 0, 0, 566, 567, - 5, 71, 0, 0, 567, 568, 5, 84, 0, 0, 568, 570, 5, 72, 0, 0, 569, 545, 1, - 0, 0, 0, 569, 557, 1, 0, 0, 0, 570, 84, 1, 0, 0, 0, 571, 572, 5, 116, 0, - 0, 572, 573, 5, 114, 0, 0, 573, 574, 5, 117, 0, 0, 574, 599, 5, 101, 0, - 0, 575, 576, 5, 84, 0, 0, 576, 577, 5, 114, 0, 0, 577, 578, 5, 117, 0, - 0, 578, 599, 5, 101, 0, 0, 579, 580, 5, 84, 0, 0, 580, 581, 5, 82, 0, 0, - 581, 582, 5, 85, 0, 0, 582, 599, 5, 69, 0, 0, 583, 584, 5, 102, 0, 0, 584, - 585, 5, 97, 0, 0, 585, 586, 5, 108, 0, 0, 586, 587, 5, 115, 0, 0, 587, - 599, 5, 101, 0, 0, 588, 589, 5, 70, 0, 0, 589, 590, 5, 97, 0, 0, 590, 591, - 5, 108, 0, 0, 591, 592, 5, 115, 0, 0, 592, 599, 5, 101, 0, 0, 593, 594, - 5, 70, 0, 0, 594, 595, 5, 65, 0, 0, 595, 596, 5, 76, 0, 0, 596, 597, 5, - 83, 0, 0, 597, 599, 5, 69, 0, 0, 598, 571, 1, 0, 0, 0, 598, 575, 1, 0, - 0, 0, 598, 579, 1, 0, 0, 0, 598, 583, 1, 0, 0, 0, 598, 588, 1, 0, 0, 0, - 598, 593, 1, 0, 0, 0, 599, 86, 1, 0, 0, 0, 600, 605, 3, 115, 57, 0, 601, - 605, 3, 117, 58, 0, 602, 605, 3, 119, 59, 0, 603, 605, 3, 113, 56, 0, 604, - 600, 1, 0, 0, 0, 604, 601, 1, 0, 0, 0, 604, 602, 1, 0, 0, 0, 604, 603, - 1, 0, 0, 0, 605, 88, 1, 0, 0, 0, 606, 609, 3, 131, 65, 0, 607, 609, 3, - 133, 66, 0, 608, 606, 1, 0, 0, 0, 608, 607, 1, 0, 0, 0, 609, 90, 1, 0, - 0, 0, 610, 615, 3, 109, 54, 0, 611, 614, 3, 109, 54, 0, 612, 614, 3, 111, - 55, 0, 613, 611, 1, 0, 0, 0, 613, 612, 1, 0, 0, 0, 614, 617, 1, 0, 0, 0, - 615, 613, 1, 0, 0, 0, 615, 616, 1, 0, 0, 0, 616, 92, 1, 0, 0, 0, 617, 615, - 1, 0, 0, 0, 618, 619, 5, 36, 0, 0, 619, 620, 5, 109, 0, 0, 620, 621, 5, - 101, 0, 0, 621, 622, 5, 116, 0, 0, 622, 623, 5, 97, 0, 0, 623, 94, 1, 0, - 0, 0, 624, 626, 3, 99, 49, 0, 625, 624, 1, 0, 0, 0, 625, 626, 1, 0, 0, - 0, 626, 637, 1, 0, 0, 0, 627, 629, 5, 34, 0, 0, 628, 630, 3, 101, 50, 0, - 629, 628, 1, 0, 0, 0, 629, 630, 1, 0, 0, 0, 630, 631, 1, 0, 0, 0, 631, - 638, 5, 34, 0, 0, 632, 634, 5, 39, 0, 0, 633, 635, 3, 103, 51, 0, 634, - 633, 1, 0, 0, 0, 634, 635, 1, 0, 0, 0, 635, 636, 1, 0, 0, 0, 636, 638, - 5, 39, 0, 0, 637, 627, 1, 0, 0, 0, 637, 632, 1, 0, 0, 0, 638, 96, 1, 0, - 0, 0, 639, 642, 3, 91, 45, 0, 640, 642, 3, 93, 46, 0, 641, 639, 1, 0, 0, - 0, 641, 640, 1, 0, 0, 0, 642, 650, 1, 0, 0, 0, 643, 646, 5, 91, 0, 0, 644, - 647, 3, 95, 47, 0, 645, 647, 3, 115, 57, 0, 646, 644, 1, 0, 0, 0, 646, - 645, 1, 0, 0, 0, 647, 648, 1, 0, 0, 0, 648, 649, 5, 93, 0, 0, 649, 651, - 1, 0, 0, 0, 650, 643, 1, 0, 0, 0, 651, 652, 1, 0, 0, 0, 652, 650, 1, 0, - 0, 0, 652, 653, 1, 0, 0, 0, 653, 98, 1, 0, 0, 0, 654, 655, 5, 117, 0, 0, - 655, 658, 5, 56, 0, 0, 656, 658, 7, 0, 0, 0, 657, 654, 1, 0, 0, 0, 657, - 656, 1, 0, 0, 0, 658, 100, 1, 0, 0, 0, 659, 661, 3, 105, 52, 0, 660, 659, - 1, 0, 0, 0, 661, 662, 1, 0, 0, 0, 662, 660, 1, 0, 0, 0, 662, 663, 1, 0, - 0, 0, 663, 102, 1, 0, 0, 0, 664, 666, 3, 107, 53, 0, 665, 664, 1, 0, 0, - 0, 666, 667, 1, 0, 0, 0, 667, 665, 1, 0, 0, 0, 667, 668, 1, 0, 0, 0, 668, - 104, 1, 0, 0, 0, 669, 677, 8, 1, 0, 0, 670, 677, 3, 147, 73, 0, 671, 672, - 5, 92, 0, 0, 672, 677, 5, 10, 0, 0, 673, 674, 5, 92, 0, 0, 674, 675, 5, - 13, 0, 0, 675, 677, 5, 10, 0, 0, 676, 669, 1, 0, 0, 0, 676, 670, 1, 0, - 0, 0, 676, 671, 1, 0, 0, 0, 676, 673, 1, 0, 0, 0, 677, 106, 1, 0, 0, 0, - 678, 686, 8, 2, 0, 0, 679, 686, 3, 147, 73, 0, 680, 681, 5, 92, 0, 0, 681, - 686, 5, 10, 0, 0, 682, 683, 5, 92, 0, 0, 683, 684, 5, 13, 0, 0, 684, 686, - 5, 10, 0, 0, 685, 678, 1, 0, 0, 0, 685, 679, 1, 0, 0, 0, 685, 680, 1, 0, - 0, 0, 685, 682, 1, 0, 0, 0, 686, 108, 1, 0, 0, 0, 687, 688, 7, 3, 0, 0, - 688, 110, 1, 0, 0, 0, 689, 690, 7, 4, 0, 0, 690, 112, 1, 0, 0, 0, 691, - 692, 5, 48, 0, 0, 692, 694, 7, 5, 0, 0, 693, 695, 7, 6, 0, 0, 694, 693, - 1, 0, 0, 0, 695, 696, 1, 0, 0, 0, 696, 694, 1, 0, 0, 0, 696, 697, 1, 0, - 0, 0, 697, 114, 1, 0, 0, 0, 698, 702, 3, 121, 60, 0, 699, 701, 3, 111, - 55, 0, 700, 699, 1, 0, 0, 0, 701, 704, 1, 0, 0, 0, 702, 700, 1, 0, 0, 0, - 702, 703, 1, 0, 0, 0, 703, 707, 1, 0, 0, 0, 704, 702, 1, 0, 0, 0, 705, - 707, 5, 48, 0, 0, 706, 698, 1, 0, 0, 0, 706, 705, 1, 0, 0, 0, 707, 116, - 1, 0, 0, 0, 708, 712, 5, 48, 0, 0, 709, 711, 3, 123, 61, 0, 710, 709, 1, - 0, 0, 0, 711, 714, 1, 0, 0, 0, 712, 710, 1, 0, 0, 0, 712, 713, 1, 0, 0, - 0, 713, 118, 1, 0, 0, 0, 714, 712, 1, 0, 0, 0, 715, 716, 5, 48, 0, 0, 716, - 717, 7, 7, 0, 0, 717, 718, 3, 143, 71, 0, 718, 120, 1, 0, 0, 0, 719, 720, - 7, 8, 0, 0, 720, 122, 1, 0, 0, 0, 721, 722, 7, 9, 0, 0, 722, 124, 1, 0, - 0, 0, 723, 724, 7, 10, 0, 0, 724, 126, 1, 0, 0, 0, 725, 726, 3, 125, 62, - 0, 726, 727, 3, 125, 62, 0, 727, 728, 3, 125, 62, 0, 728, 729, 3, 125, - 62, 0, 729, 128, 1, 0, 0, 0, 730, 731, 5, 92, 0, 0, 731, 732, 5, 117, 0, - 0, 732, 733, 1, 0, 0, 0, 733, 741, 3, 127, 63, 0, 734, 735, 5, 92, 0, 0, - 735, 736, 5, 85, 0, 0, 736, 737, 1, 0, 0, 0, 737, 738, 3, 127, 63, 0, 738, - 739, 3, 127, 63, 0, 739, 741, 1, 0, 0, 0, 740, 730, 1, 0, 0, 0, 740, 734, - 1, 0, 0, 0, 741, 130, 1, 0, 0, 0, 742, 744, 3, 135, 67, 0, 743, 745, 3, - 137, 68, 0, 744, 743, 1, 0, 0, 0, 744, 745, 1, 0, 0, 0, 745, 750, 1, 0, - 0, 0, 746, 747, 3, 139, 69, 0, 747, 748, 3, 137, 68, 0, 748, 750, 1, 0, - 0, 0, 749, 742, 1, 0, 0, 0, 749, 746, 1, 0, 0, 0, 750, 132, 1, 0, 0, 0, - 751, 752, 5, 48, 0, 0, 752, 755, 7, 7, 0, 0, 753, 756, 3, 141, 70, 0, 754, - 756, 3, 143, 71, 0, 755, 753, 1, 0, 0, 0, 755, 754, 1, 0, 0, 0, 756, 757, - 1, 0, 0, 0, 757, 758, 3, 145, 72, 0, 758, 134, 1, 0, 0, 0, 759, 761, 3, - 139, 69, 0, 760, 759, 1, 0, 0, 0, 760, 761, 1, 0, 0, 0, 761, 762, 1, 0, - 0, 0, 762, 763, 5, 46, 0, 0, 763, 768, 3, 139, 69, 0, 764, 765, 3, 139, - 69, 0, 765, 766, 5, 46, 0, 0, 766, 768, 1, 0, 0, 0, 767, 760, 1, 0, 0, - 0, 767, 764, 1, 0, 0, 0, 768, 136, 1, 0, 0, 0, 769, 771, 7, 11, 0, 0, 770, - 772, 7, 12, 0, 0, 771, 770, 1, 0, 0, 0, 771, 772, 1, 0, 0, 0, 772, 773, - 1, 0, 0, 0, 773, 774, 3, 139, 69, 0, 774, 138, 1, 0, 0, 0, 775, 777, 3, - 111, 55, 0, 776, 775, 1, 0, 0, 0, 777, 778, 1, 0, 0, 0, 778, 776, 1, 0, - 0, 0, 778, 779, 1, 0, 0, 0, 779, 140, 1, 0, 0, 0, 780, 782, 3, 143, 71, - 0, 781, 780, 1, 0, 0, 0, 781, 782, 1, 0, 0, 0, 782, 783, 1, 0, 0, 0, 783, - 784, 5, 46, 0, 0, 784, 789, 3, 143, 71, 0, 785, 786, 3, 143, 71, 0, 786, - 787, 5, 46, 0, 0, 787, 789, 1, 0, 0, 0, 788, 781, 1, 0, 0, 0, 788, 785, - 1, 0, 0, 0, 789, 142, 1, 0, 0, 0, 790, 792, 3, 125, 62, 0, 791, 790, 1, - 0, 0, 0, 792, 793, 1, 0, 0, 0, 793, 791, 1, 0, 0, 0, 793, 794, 1, 0, 0, - 0, 794, 144, 1, 0, 0, 0, 795, 797, 7, 13, 0, 0, 796, 798, 7, 12, 0, 0, - 797, 796, 1, 0, 0, 0, 797, 798, 1, 0, 0, 0, 798, 799, 1, 0, 0, 0, 799, - 800, 3, 139, 69, 0, 800, 146, 1, 0, 0, 0, 801, 802, 5, 92, 0, 0, 802, 817, - 7, 14, 0, 0, 803, 804, 5, 92, 0, 0, 804, 806, 3, 123, 61, 0, 805, 807, - 3, 123, 61, 0, 806, 805, 1, 0, 0, 0, 806, 807, 1, 0, 0, 0, 807, 809, 1, - 0, 0, 0, 808, 810, 3, 123, 61, 0, 809, 808, 1, 0, 0, 0, 809, 810, 1, 0, - 0, 0, 810, 817, 1, 0, 0, 0, 811, 812, 5, 92, 0, 0, 812, 813, 5, 120, 0, - 0, 813, 814, 1, 0, 0, 0, 814, 817, 3, 143, 71, 0, 815, 817, 3, 129, 64, - 0, 816, 801, 1, 0, 0, 0, 816, 803, 1, 0, 0, 0, 816, 811, 1, 0, 0, 0, 816, - 815, 1, 0, 0, 0, 817, 148, 1, 0, 0, 0, 818, 820, 7, 15, 0, 0, 819, 818, - 1, 0, 0, 0, 820, 821, 1, 0, 0, 0, 821, 819, 1, 0, 0, 0, 821, 822, 1, 0, - 0, 0, 822, 823, 1, 0, 0, 0, 823, 824, 6, 74, 0, 0, 824, 150, 1, 0, 0, 0, - 825, 827, 5, 13, 0, 0, 826, 828, 5, 10, 0, 0, 827, 826, 1, 0, 0, 0, 827, - 828, 1, 0, 0, 0, 828, 831, 1, 0, 0, 0, 829, 831, 5, 10, 0, 0, 830, 825, - 1, 0, 0, 0, 830, 829, 1, 0, 0, 0, 831, 832, 1, 0, 0, 0, 832, 833, 6, 75, - 0, 0, 833, 152, 1, 0, 0, 0, 58, 0, 191, 205, 227, 262, 270, 286, 310, 321, - 327, 332, 334, 365, 401, 437, 467, 505, 543, 569, 598, 604, 608, 613, 615, - 625, 629, 634, 637, 641, 646, 652, 657, 662, 667, 676, 685, 696, 702, 706, - 712, 740, 744, 749, 755, 760, 767, 771, 778, 781, 788, 793, 797, 806, 809, - 816, 821, 827, 830, 1, 6, 0, 0, + 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, + 1, 42, 1, 42, 1, 42, 3, 42, 598, 8, 42, 1, 43, 1, 43, 1, 43, 1, 43, 1, + 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, + 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, + 43, 1, 43, 3, 43, 627, 8, 43, 1, 44, 1, 44, 1, 44, 1, 44, 3, 44, 633, 8, + 44, 1, 45, 1, 45, 3, 45, 637, 8, 45, 1, 46, 1, 46, 1, 46, 5, 46, 642, 8, + 46, 10, 46, 12, 46, 645, 9, 46, 1, 47, 1, 47, 1, 47, 1, 47, 1, 47, 1, 47, + 1, 48, 3, 48, 654, 8, 48, 1, 48, 1, 48, 3, 48, 658, 8, 48, 1, 48, 1, 48, + 1, 48, 3, 48, 663, 8, 48, 1, 48, 3, 48, 666, 8, 48, 1, 49, 1, 49, 3, 49, + 670, 8, 49, 1, 49, 1, 49, 1, 49, 3, 49, 675, 8, 49, 1, 49, 1, 49, 4, 49, + 679, 8, 49, 11, 49, 12, 49, 680, 1, 50, 1, 50, 1, 50, 3, 50, 686, 8, 50, + 1, 51, 4, 51, 689, 8, 51, 11, 51, 12, 51, 690, 1, 52, 4, 52, 694, 8, 52, + 11, 52, 12, 52, 695, 1, 53, 1, 53, 1, 53, 1, 53, 1, 53, 1, 53, 1, 53, 3, + 53, 705, 8, 53, 1, 54, 1, 54, 1, 54, 1, 54, 1, 54, 1, 54, 1, 54, 3, 54, + 714, 8, 54, 1, 55, 1, 55, 1, 56, 1, 56, 1, 57, 1, 57, 1, 57, 4, 57, 723, + 8, 57, 11, 57, 12, 57, 724, 1, 58, 1, 58, 5, 58, 729, 8, 58, 10, 58, 12, + 58, 732, 9, 58, 1, 58, 3, 58, 735, 8, 58, 1, 59, 1, 59, 5, 59, 739, 8, + 59, 10, 59, 12, 59, 742, 9, 59, 1, 60, 1, 60, 1, 60, 1, 60, 1, 61, 1, 61, + 1, 62, 1, 62, 1, 63, 1, 63, 1, 64, 1, 64, 1, 64, 1, 64, 1, 64, 1, 65, 1, + 65, 1, 65, 1, 65, 1, 65, 1, 65, 1, 65, 1, 65, 1, 65, 1, 65, 3, 65, 769, + 8, 65, 1, 66, 1, 66, 3, 66, 773, 8, 66, 1, 66, 1, 66, 1, 66, 3, 66, 778, + 8, 66, 1, 67, 1, 67, 1, 67, 1, 67, 3, 67, 784, 8, 67, 1, 67, 1, 67, 1, + 68, 3, 68, 789, 8, 68, 1, 68, 1, 68, 1, 68, 1, 68, 1, 68, 3, 68, 796, 8, + 68, 1, 69, 1, 69, 3, 69, 800, 8, 69, 1, 69, 1, 69, 1, 70, 4, 70, 805, 8, + 70, 11, 70, 12, 70, 806, 1, 71, 3, 71, 810, 8, 71, 1, 71, 1, 71, 1, 71, + 1, 71, 1, 71, 3, 71, 817, 8, 71, 1, 72, 4, 72, 820, 8, 72, 11, 72, 12, + 72, 821, 1, 73, 1, 73, 3, 73, 826, 8, 73, 1, 73, 1, 73, 1, 74, 1, 74, 1, + 74, 1, 74, 1, 74, 3, 74, 835, 8, 74, 1, 74, 3, 74, 838, 8, 74, 1, 74, 1, + 74, 1, 74, 1, 74, 1, 74, 3, 74, 845, 8, 74, 1, 75, 4, 75, 848, 8, 75, 11, + 75, 12, 75, 849, 1, 75, 1, 75, 1, 76, 1, 76, 3, 76, 856, 8, 76, 1, 76, + 3, 76, 859, 8, 76, 1, 76, 1, 76, 0, 0, 77, 1, 1, 3, 2, 5, 3, 7, 4, 9, 5, + 11, 6, 13, 7, 15, 8, 17, 9, 19, 10, 21, 11, 23, 12, 25, 13, 27, 14, 29, + 15, 31, 16, 33, 17, 35, 18, 37, 19, 39, 20, 41, 21, 43, 22, 45, 23, 47, + 24, 49, 25, 51, 26, 53, 27, 55, 28, 57, 29, 59, 30, 61, 31, 63, 32, 65, + 33, 67, 34, 69, 35, 71, 36, 73, 37, 75, 38, 77, 39, 79, 40, 81, 41, 83, + 42, 85, 43, 87, 44, 89, 45, 91, 46, 93, 47, 95, 48, 97, 49, 99, 50, 101, + 0, 103, 0, 105, 0, 107, 0, 109, 0, 111, 0, 113, 0, 115, 0, 117, 0, 119, + 0, 121, 0, 123, 0, 125, 0, 127, 0, 129, 0, 131, 0, 133, 0, 135, 0, 137, + 0, 139, 0, 141, 0, 143, 0, 145, 0, 147, 0, 149, 0, 151, 51, 153, 52, 1, + 0, 16, 3, 0, 76, 76, 85, 85, 117, 117, 4, 0, 10, 10, 13, 13, 34, 34, 92, + 92, 4, 0, 10, 10, 13, 13, 39, 39, 92, 92, 3, 0, 65, 90, 95, 95, 97, 122, + 1, 0, 48, 57, 2, 0, 66, 66, 98, 98, 1, 0, 48, 49, 2, 0, 88, 88, 120, 120, + 1, 0, 49, 57, 1, 0, 48, 55, 3, 0, 48, 57, 65, 70, 97, 102, 2, 0, 69, 69, + 101, 101, 2, 0, 43, 43, 45, 45, 2, 0, 80, 80, 112, 112, 10, 0, 34, 34, + 39, 39, 63, 63, 92, 92, 97, 98, 102, 102, 110, 110, 114, 114, 116, 116, + 118, 118, 2, 0, 9, 9, 32, 32, 909, 0, 1, 1, 0, 0, 0, 0, 3, 1, 0, 0, 0, + 0, 5, 1, 0, 0, 0, 0, 7, 1, 0, 0, 0, 0, 9, 1, 0, 0, 0, 0, 11, 1, 0, 0, 0, + 0, 13, 1, 0, 0, 0, 0, 15, 1, 0, 0, 0, 0, 17, 1, 0, 0, 0, 0, 19, 1, 0, 0, + 0, 0, 21, 1, 0, 0, 0, 0, 23, 1, 0, 0, 0, 0, 25, 1, 0, 0, 0, 0, 27, 1, 0, + 0, 0, 0, 29, 1, 0, 0, 0, 0, 31, 1, 0, 0, 0, 0, 33, 1, 0, 0, 0, 0, 35, 1, + 0, 0, 0, 0, 37, 1, 0, 0, 0, 0, 39, 1, 0, 0, 0, 0, 41, 1, 0, 0, 0, 0, 43, + 1, 0, 0, 0, 0, 45, 1, 0, 0, 0, 0, 47, 1, 0, 0, 0, 0, 49, 1, 0, 0, 0, 0, + 51, 1, 0, 0, 0, 0, 53, 1, 0, 0, 0, 0, 55, 1, 0, 0, 0, 0, 57, 1, 0, 0, 0, + 0, 59, 1, 0, 0, 0, 0, 61, 1, 0, 0, 0, 0, 63, 1, 0, 0, 0, 0, 65, 1, 0, 0, + 0, 0, 67, 1, 0, 0, 0, 0, 69, 1, 0, 0, 0, 0, 71, 1, 0, 0, 0, 0, 73, 1, 0, + 0, 0, 0, 75, 1, 0, 0, 0, 0, 77, 1, 0, 0, 0, 0, 79, 1, 0, 0, 0, 0, 81, 1, + 0, 0, 0, 0, 83, 1, 0, 0, 0, 0, 85, 1, 0, 0, 0, 0, 87, 1, 0, 0, 0, 0, 89, + 1, 0, 0, 0, 0, 91, 1, 0, 0, 0, 0, 93, 1, 0, 0, 0, 0, 95, 1, 0, 0, 0, 0, + 97, 1, 0, 0, 0, 0, 99, 1, 0, 0, 0, 0, 151, 1, 0, 0, 0, 0, 153, 1, 0, 0, + 0, 1, 155, 1, 0, 0, 0, 3, 157, 1, 0, 0, 0, 5, 159, 1, 0, 0, 0, 7, 161, + 1, 0, 0, 0, 9, 163, 1, 0, 0, 0, 11, 165, 1, 0, 0, 0, 13, 167, 1, 0, 0, + 0, 15, 169, 1, 0, 0, 0, 17, 171, 1, 0, 0, 0, 19, 174, 1, 0, 0, 0, 21, 176, + 1, 0, 0, 0, 23, 179, 1, 0, 0, 0, 25, 182, 1, 0, 0, 0, 27, 193, 1, 0, 0, + 0, 29, 207, 1, 0, 0, 0, 31, 229, 1, 0, 0, 0, 33, 255, 1, 0, 0, 0, 35, 257, + 1, 0, 0, 0, 37, 259, 1, 0, 0, 0, 39, 261, 1, 0, 0, 0, 41, 263, 1, 0, 0, + 0, 43, 265, 1, 0, 0, 0, 45, 267, 1, 0, 0, 0, 47, 270, 1, 0, 0, 0, 49, 273, + 1, 0, 0, 0, 51, 276, 1, 0, 0, 0, 53, 278, 1, 0, 0, 0, 55, 280, 1, 0, 0, + 0, 57, 290, 1, 0, 0, 0, 59, 298, 1, 0, 0, 0, 61, 314, 1, 0, 0, 0, 63, 338, + 1, 0, 0, 0, 65, 340, 1, 0, 0, 0, 67, 349, 1, 0, 0, 0, 69, 355, 1, 0, 0, + 0, 71, 357, 1, 0, 0, 0, 73, 393, 1, 0, 0, 0, 75, 429, 1, 0, 0, 0, 77, 465, + 1, 0, 0, 0, 79, 495, 1, 0, 0, 0, 81, 533, 1, 0, 0, 0, 83, 571, 1, 0, 0, + 0, 85, 597, 1, 0, 0, 0, 87, 626, 1, 0, 0, 0, 89, 632, 1, 0, 0, 0, 91, 636, + 1, 0, 0, 0, 93, 638, 1, 0, 0, 0, 95, 646, 1, 0, 0, 0, 97, 653, 1, 0, 0, + 0, 99, 669, 1, 0, 0, 0, 101, 685, 1, 0, 0, 0, 103, 688, 1, 0, 0, 0, 105, + 693, 1, 0, 0, 0, 107, 704, 1, 0, 0, 0, 109, 713, 1, 0, 0, 0, 111, 715, + 1, 0, 0, 0, 113, 717, 1, 0, 0, 0, 115, 719, 1, 0, 0, 0, 117, 734, 1, 0, + 0, 0, 119, 736, 1, 0, 0, 0, 121, 743, 1, 0, 0, 0, 123, 747, 1, 0, 0, 0, + 125, 749, 1, 0, 0, 0, 127, 751, 1, 0, 0, 0, 129, 753, 1, 0, 0, 0, 131, + 768, 1, 0, 0, 0, 133, 777, 1, 0, 0, 0, 135, 779, 1, 0, 0, 0, 137, 795, + 1, 0, 0, 0, 139, 797, 1, 0, 0, 0, 141, 804, 1, 0, 0, 0, 143, 816, 1, 0, + 0, 0, 145, 819, 1, 0, 0, 0, 147, 823, 1, 0, 0, 0, 149, 844, 1, 0, 0, 0, + 151, 847, 1, 0, 0, 0, 153, 858, 1, 0, 0, 0, 155, 156, 5, 40, 0, 0, 156, + 2, 1, 0, 0, 0, 157, 158, 5, 41, 0, 0, 158, 4, 1, 0, 0, 0, 159, 160, 5, + 91, 0, 0, 160, 6, 1, 0, 0, 0, 161, 162, 5, 44, 0, 0, 162, 8, 1, 0, 0, 0, + 163, 164, 5, 93, 0, 0, 164, 10, 1, 0, 0, 0, 165, 166, 5, 123, 0, 0, 166, + 12, 1, 0, 0, 0, 167, 168, 5, 125, 0, 0, 168, 14, 1, 0, 0, 0, 169, 170, + 5, 60, 0, 0, 170, 16, 1, 0, 0, 0, 171, 172, 5, 60, 0, 0, 172, 173, 5, 61, + 0, 0, 173, 18, 1, 0, 0, 0, 174, 175, 5, 62, 0, 0, 175, 20, 1, 0, 0, 0, + 176, 177, 5, 62, 0, 0, 177, 178, 5, 61, 0, 0, 178, 22, 1, 0, 0, 0, 179, + 180, 5, 61, 0, 0, 180, 181, 5, 61, 0, 0, 181, 24, 1, 0, 0, 0, 182, 183, + 5, 33, 0, 0, 183, 184, 5, 61, 0, 0, 184, 26, 1, 0, 0, 0, 185, 186, 5, 108, + 0, 0, 186, 187, 5, 105, 0, 0, 187, 188, 5, 107, 0, 0, 188, 194, 5, 101, + 0, 0, 189, 190, 5, 76, 0, 0, 190, 191, 5, 73, 0, 0, 191, 192, 5, 75, 0, + 0, 192, 194, 5, 69, 0, 0, 193, 185, 1, 0, 0, 0, 193, 189, 1, 0, 0, 0, 194, + 28, 1, 0, 0, 0, 195, 196, 5, 101, 0, 0, 196, 197, 5, 120, 0, 0, 197, 198, + 5, 105, 0, 0, 198, 199, 5, 115, 0, 0, 199, 200, 5, 116, 0, 0, 200, 208, + 5, 115, 0, 0, 201, 202, 5, 69, 0, 0, 202, 203, 5, 88, 0, 0, 203, 204, 5, + 73, 0, 0, 204, 205, 5, 83, 0, 0, 205, 206, 5, 84, 0, 0, 206, 208, 5, 83, + 0, 0, 207, 195, 1, 0, 0, 0, 207, 201, 1, 0, 0, 0, 208, 30, 1, 0, 0, 0, + 209, 210, 5, 116, 0, 0, 210, 211, 5, 101, 0, 0, 211, 212, 5, 120, 0, 0, + 212, 213, 5, 116, 0, 0, 213, 214, 5, 95, 0, 0, 214, 215, 5, 109, 0, 0, + 215, 216, 5, 97, 0, 0, 216, 217, 5, 116, 0, 0, 217, 218, 5, 99, 0, 0, 218, + 230, 5, 104, 0, 0, 219, 220, 5, 84, 0, 0, 220, 221, 5, 69, 0, 0, 221, 222, + 5, 88, 0, 0, 222, 223, 5, 84, 0, 0, 223, 224, 5, 95, 0, 0, 224, 225, 5, + 77, 0, 0, 225, 226, 5, 65, 0, 0, 226, 227, 5, 84, 0, 0, 227, 228, 5, 67, + 0, 0, 228, 230, 5, 72, 0, 0, 229, 209, 1, 0, 0, 0, 229, 219, 1, 0, 0, 0, + 230, 32, 1, 0, 0, 0, 231, 232, 5, 112, 0, 0, 232, 233, 5, 104, 0, 0, 233, + 234, 5, 114, 0, 0, 234, 235, 5, 97, 0, 0, 235, 236, 5, 115, 0, 0, 236, + 237, 5, 101, 0, 0, 237, 238, 5, 95, 0, 0, 238, 239, 5, 109, 0, 0, 239, + 240, 5, 97, 0, 0, 240, 241, 5, 116, 0, 0, 241, 242, 5, 99, 0, 0, 242, 256, + 5, 104, 0, 0, 243, 244, 5, 80, 0, 0, 244, 245, 5, 72, 0, 0, 245, 246, 5, + 82, 0, 0, 246, 247, 5, 65, 0, 0, 247, 248, 5, 83, 0, 0, 248, 249, 5, 69, + 0, 0, 249, 250, 5, 95, 0, 0, 250, 251, 5, 77, 0, 0, 251, 252, 5, 65, 0, + 0, 252, 253, 5, 84, 0, 0, 253, 254, 5, 67, 0, 0, 254, 256, 5, 72, 0, 0, + 255, 231, 1, 0, 0, 0, 255, 243, 1, 0, 0, 0, 256, 34, 1, 0, 0, 0, 257, 258, + 5, 43, 0, 0, 258, 36, 1, 0, 0, 0, 259, 260, 5, 45, 0, 0, 260, 38, 1, 0, + 0, 0, 261, 262, 5, 42, 0, 0, 262, 40, 1, 0, 0, 0, 263, 264, 5, 47, 0, 0, + 264, 42, 1, 0, 0, 0, 265, 266, 5, 37, 0, 0, 266, 44, 1, 0, 0, 0, 267, 268, + 5, 42, 0, 0, 268, 269, 5, 42, 0, 0, 269, 46, 1, 0, 0, 0, 270, 271, 5, 60, + 0, 0, 271, 272, 5, 60, 0, 0, 272, 48, 1, 0, 0, 0, 273, 274, 5, 62, 0, 0, + 274, 275, 5, 62, 0, 0, 275, 50, 1, 0, 0, 0, 276, 277, 5, 38, 0, 0, 277, + 52, 1, 0, 0, 0, 278, 279, 5, 124, 0, 0, 279, 54, 1, 0, 0, 0, 280, 281, + 5, 94, 0, 0, 281, 56, 1, 0, 0, 0, 282, 283, 5, 38, 0, 0, 283, 291, 5, 38, + 0, 0, 284, 285, 5, 97, 0, 0, 285, 286, 5, 110, 0, 0, 286, 291, 5, 100, + 0, 0, 287, 288, 5, 65, 0, 0, 288, 289, 5, 78, 0, 0, 289, 291, 5, 68, 0, + 0, 290, 282, 1, 0, 0, 0, 290, 284, 1, 0, 0, 0, 290, 287, 1, 0, 0, 0, 291, + 58, 1, 0, 0, 0, 292, 293, 5, 124, 0, 0, 293, 299, 5, 124, 0, 0, 294, 295, + 5, 111, 0, 0, 295, 299, 5, 114, 0, 0, 296, 297, 5, 79, 0, 0, 297, 299, + 5, 82, 0, 0, 298, 292, 1, 0, 0, 0, 298, 294, 1, 0, 0, 0, 298, 296, 1, 0, + 0, 0, 299, 60, 1, 0, 0, 0, 300, 301, 5, 105, 0, 0, 301, 302, 5, 115, 0, + 0, 302, 303, 5, 32, 0, 0, 303, 304, 5, 110, 0, 0, 304, 305, 5, 117, 0, + 0, 305, 306, 5, 108, 0, 0, 306, 315, 5, 108, 0, 0, 307, 308, 5, 73, 0, + 0, 308, 309, 5, 83, 0, 0, 309, 310, 5, 32, 0, 0, 310, 311, 5, 78, 0, 0, + 311, 312, 5, 85, 0, 0, 312, 313, 5, 76, 0, 0, 313, 315, 5, 76, 0, 0, 314, + 300, 1, 0, 0, 0, 314, 307, 1, 0, 0, 0, 315, 62, 1, 0, 0, 0, 316, 317, 5, + 105, 0, 0, 317, 318, 5, 115, 0, 0, 318, 319, 5, 32, 0, 0, 319, 320, 5, + 110, 0, 0, 320, 321, 5, 111, 0, 0, 321, 322, 5, 116, 0, 0, 322, 323, 5, + 32, 0, 0, 323, 324, 5, 110, 0, 0, 324, 325, 5, 117, 0, 0, 325, 326, 5, + 108, 0, 0, 326, 339, 5, 108, 0, 0, 327, 328, 5, 73, 0, 0, 328, 329, 5, + 83, 0, 0, 329, 330, 5, 32, 0, 0, 330, 331, 5, 78, 0, 0, 331, 332, 5, 79, + 0, 0, 332, 333, 5, 84, 0, 0, 333, 334, 5, 32, 0, 0, 334, 335, 5, 78, 0, + 0, 335, 336, 5, 85, 0, 0, 336, 337, 5, 76, 0, 0, 337, 339, 5, 76, 0, 0, + 338, 316, 1, 0, 0, 0, 338, 327, 1, 0, 0, 0, 339, 64, 1, 0, 0, 0, 340, 341, + 5, 126, 0, 0, 341, 66, 1, 0, 0, 0, 342, 350, 5, 33, 0, 0, 343, 344, 5, + 110, 0, 0, 344, 345, 5, 111, 0, 0, 345, 350, 5, 116, 0, 0, 346, 347, 5, + 78, 0, 0, 347, 348, 5, 79, 0, 0, 348, 350, 5, 84, 0, 0, 349, 342, 1, 0, + 0, 0, 349, 343, 1, 0, 0, 0, 349, 346, 1, 0, 0, 0, 350, 68, 1, 0, 0, 0, + 351, 352, 5, 105, 0, 0, 352, 356, 5, 110, 0, 0, 353, 354, 5, 73, 0, 0, + 354, 356, 5, 78, 0, 0, 355, 351, 1, 0, 0, 0, 355, 353, 1, 0, 0, 0, 356, + 70, 1, 0, 0, 0, 357, 362, 5, 91, 0, 0, 358, 361, 3, 151, 75, 0, 359, 361, + 3, 153, 76, 0, 360, 358, 1, 0, 0, 0, 360, 359, 1, 0, 0, 0, 361, 364, 1, + 0, 0, 0, 362, 360, 1, 0, 0, 0, 362, 363, 1, 0, 0, 0, 363, 365, 1, 0, 0, + 0, 364, 362, 1, 0, 0, 0, 365, 366, 5, 93, 0, 0, 366, 72, 1, 0, 0, 0, 367, + 368, 5, 106, 0, 0, 368, 369, 5, 115, 0, 0, 369, 370, 5, 111, 0, 0, 370, + 371, 5, 110, 0, 0, 371, 372, 5, 95, 0, 0, 372, 373, 5, 99, 0, 0, 373, 374, + 5, 111, 0, 0, 374, 375, 5, 110, 0, 0, 375, 376, 5, 116, 0, 0, 376, 377, + 5, 97, 0, 0, 377, 378, 5, 105, 0, 0, 378, 379, 5, 110, 0, 0, 379, 394, + 5, 115, 0, 0, 380, 381, 5, 74, 0, 0, 381, 382, 5, 83, 0, 0, 382, 383, 5, + 79, 0, 0, 383, 384, 5, 78, 0, 0, 384, 385, 5, 95, 0, 0, 385, 386, 5, 67, + 0, 0, 386, 387, 5, 79, 0, 0, 387, 388, 5, 78, 0, 0, 388, 389, 5, 84, 0, + 0, 389, 390, 5, 65, 0, 0, 390, 391, 5, 73, 0, 0, 391, 392, 5, 78, 0, 0, + 392, 394, 5, 83, 0, 0, 393, 367, 1, 0, 0, 0, 393, 380, 1, 0, 0, 0, 394, + 74, 1, 0, 0, 0, 395, 396, 5, 106, 0, 0, 396, 397, 5, 115, 0, 0, 397, 398, + 5, 111, 0, 0, 398, 399, 5, 110, 0, 0, 399, 400, 5, 95, 0, 0, 400, 401, + 5, 99, 0, 0, 401, 402, 5, 111, 0, 0, 402, 403, 5, 110, 0, 0, 403, 404, + 5, 116, 0, 0, 404, 405, 5, 97, 0, 0, 405, 406, 5, 105, 0, 0, 406, 407, + 5, 110, 0, 0, 407, 408, 5, 115, 0, 0, 408, 409, 5, 95, 0, 0, 409, 410, + 5, 97, 0, 0, 410, 411, 5, 108, 0, 0, 411, 430, 5, 108, 0, 0, 412, 413, + 5, 74, 0, 0, 413, 414, 5, 83, 0, 0, 414, 415, 5, 79, 0, 0, 415, 416, 5, + 78, 0, 0, 416, 417, 5, 95, 0, 0, 417, 418, 5, 67, 0, 0, 418, 419, 5, 79, + 0, 0, 419, 420, 5, 78, 0, 0, 420, 421, 5, 84, 0, 0, 421, 422, 5, 65, 0, + 0, 422, 423, 5, 73, 0, 0, 423, 424, 5, 78, 0, 0, 424, 425, 5, 83, 0, 0, + 425, 426, 5, 95, 0, 0, 426, 427, 5, 65, 0, 0, 427, 428, 5, 76, 0, 0, 428, + 430, 5, 76, 0, 0, 429, 395, 1, 0, 0, 0, 429, 412, 1, 0, 0, 0, 430, 76, + 1, 0, 0, 0, 431, 432, 5, 106, 0, 0, 432, 433, 5, 115, 0, 0, 433, 434, 5, + 111, 0, 0, 434, 435, 5, 110, 0, 0, 435, 436, 5, 95, 0, 0, 436, 437, 5, + 99, 0, 0, 437, 438, 5, 111, 0, 0, 438, 439, 5, 110, 0, 0, 439, 440, 5, + 116, 0, 0, 440, 441, 5, 97, 0, 0, 441, 442, 5, 105, 0, 0, 442, 443, 5, + 110, 0, 0, 443, 444, 5, 115, 0, 0, 444, 445, 5, 95, 0, 0, 445, 446, 5, + 97, 0, 0, 446, 447, 5, 110, 0, 0, 447, 466, 5, 121, 0, 0, 448, 449, 5, + 74, 0, 0, 449, 450, 5, 83, 0, 0, 450, 451, 5, 79, 0, 0, 451, 452, 5, 78, + 0, 0, 452, 453, 5, 95, 0, 0, 453, 454, 5, 67, 0, 0, 454, 455, 5, 79, 0, + 0, 455, 456, 5, 78, 0, 0, 456, 457, 5, 84, 0, 0, 457, 458, 5, 65, 0, 0, + 458, 459, 5, 73, 0, 0, 459, 460, 5, 78, 0, 0, 460, 461, 5, 83, 0, 0, 461, + 462, 5, 95, 0, 0, 462, 463, 5, 65, 0, 0, 463, 464, 5, 78, 0, 0, 464, 466, + 5, 89, 0, 0, 465, 431, 1, 0, 0, 0, 465, 448, 1, 0, 0, 0, 466, 78, 1, 0, + 0, 0, 467, 468, 5, 97, 0, 0, 468, 469, 5, 114, 0, 0, 469, 470, 5, 114, + 0, 0, 470, 471, 5, 97, 0, 0, 471, 472, 5, 121, 0, 0, 472, 473, 5, 95, 0, + 0, 473, 474, 5, 99, 0, 0, 474, 475, 5, 111, 0, 0, 475, 476, 5, 110, 0, + 0, 476, 477, 5, 116, 0, 0, 477, 478, 5, 97, 0, 0, 478, 479, 5, 105, 0, + 0, 479, 480, 5, 110, 0, 0, 480, 496, 5, 115, 0, 0, 481, 482, 5, 65, 0, + 0, 482, 483, 5, 82, 0, 0, 483, 484, 5, 82, 0, 0, 484, 485, 5, 65, 0, 0, + 485, 486, 5, 89, 0, 0, 486, 487, 5, 95, 0, 0, 487, 488, 5, 67, 0, 0, 488, + 489, 5, 79, 0, 0, 489, 490, 5, 78, 0, 0, 490, 491, 5, 84, 0, 0, 491, 492, + 5, 65, 0, 0, 492, 493, 5, 73, 0, 0, 493, 494, 5, 78, 0, 0, 494, 496, 5, + 83, 0, 0, 495, 467, 1, 0, 0, 0, 495, 481, 1, 0, 0, 0, 496, 80, 1, 0, 0, + 0, 497, 498, 5, 97, 0, 0, 498, 499, 5, 114, 0, 0, 499, 500, 5, 114, 0, + 0, 500, 501, 5, 97, 0, 0, 501, 502, 5, 121, 0, 0, 502, 503, 5, 95, 0, 0, + 503, 504, 5, 99, 0, 0, 504, 505, 5, 111, 0, 0, 505, 506, 5, 110, 0, 0, + 506, 507, 5, 116, 0, 0, 507, 508, 5, 97, 0, 0, 508, 509, 5, 105, 0, 0, + 509, 510, 5, 110, 0, 0, 510, 511, 5, 115, 0, 0, 511, 512, 5, 95, 0, 0, + 512, 513, 5, 97, 0, 0, 513, 514, 5, 108, 0, 0, 514, 534, 5, 108, 0, 0, + 515, 516, 5, 65, 0, 0, 516, 517, 5, 82, 0, 0, 517, 518, 5, 82, 0, 0, 518, + 519, 5, 65, 0, 0, 519, 520, 5, 89, 0, 0, 520, 521, 5, 95, 0, 0, 521, 522, + 5, 67, 0, 0, 522, 523, 5, 79, 0, 0, 523, 524, 5, 78, 0, 0, 524, 525, 5, + 84, 0, 0, 525, 526, 5, 65, 0, 0, 526, 527, 5, 73, 0, 0, 527, 528, 5, 78, + 0, 0, 528, 529, 5, 83, 0, 0, 529, 530, 5, 95, 0, 0, 530, 531, 5, 65, 0, + 0, 531, 532, 5, 76, 0, 0, 532, 534, 5, 76, 0, 0, 533, 497, 1, 0, 0, 0, + 533, 515, 1, 0, 0, 0, 534, 82, 1, 0, 0, 0, 535, 536, 5, 97, 0, 0, 536, + 537, 5, 114, 0, 0, 537, 538, 5, 114, 0, 0, 538, 539, 5, 97, 0, 0, 539, + 540, 5, 121, 0, 0, 540, 541, 5, 95, 0, 0, 541, 542, 5, 99, 0, 0, 542, 543, + 5, 111, 0, 0, 543, 544, 5, 110, 0, 0, 544, 545, 5, 116, 0, 0, 545, 546, + 5, 97, 0, 0, 546, 547, 5, 105, 0, 0, 547, 548, 5, 110, 0, 0, 548, 549, + 5, 115, 0, 0, 549, 550, 5, 95, 0, 0, 550, 551, 5, 97, 0, 0, 551, 552, 5, + 110, 0, 0, 552, 572, 5, 121, 0, 0, 553, 554, 5, 65, 0, 0, 554, 555, 5, + 82, 0, 0, 555, 556, 5, 82, 0, 0, 556, 557, 5, 65, 0, 0, 557, 558, 5, 89, + 0, 0, 558, 559, 5, 95, 0, 0, 559, 560, 5, 67, 0, 0, 560, 561, 5, 79, 0, + 0, 561, 562, 5, 78, 0, 0, 562, 563, 5, 84, 0, 0, 563, 564, 5, 65, 0, 0, + 564, 565, 5, 73, 0, 0, 565, 566, 5, 78, 0, 0, 566, 567, 5, 83, 0, 0, 567, + 568, 5, 95, 0, 0, 568, 569, 5, 65, 0, 0, 569, 570, 5, 78, 0, 0, 570, 572, + 5, 89, 0, 0, 571, 535, 1, 0, 0, 0, 571, 553, 1, 0, 0, 0, 572, 84, 1, 0, + 0, 0, 573, 574, 5, 97, 0, 0, 574, 575, 5, 114, 0, 0, 575, 576, 5, 114, + 0, 0, 576, 577, 5, 97, 0, 0, 577, 578, 5, 121, 0, 0, 578, 579, 5, 95, 0, + 0, 579, 580, 5, 108, 0, 0, 580, 581, 5, 101, 0, 0, 581, 582, 5, 110, 0, + 0, 582, 583, 5, 103, 0, 0, 583, 584, 5, 116, 0, 0, 584, 598, 5, 104, 0, + 0, 585, 586, 5, 65, 0, 0, 586, 587, 5, 82, 0, 0, 587, 588, 5, 82, 0, 0, + 588, 589, 5, 65, 0, 0, 589, 590, 5, 89, 0, 0, 590, 591, 5, 95, 0, 0, 591, + 592, 5, 76, 0, 0, 592, 593, 5, 69, 0, 0, 593, 594, 5, 78, 0, 0, 594, 595, + 5, 71, 0, 0, 595, 596, 5, 84, 0, 0, 596, 598, 5, 72, 0, 0, 597, 573, 1, + 0, 0, 0, 597, 585, 1, 0, 0, 0, 598, 86, 1, 0, 0, 0, 599, 600, 5, 116, 0, + 0, 600, 601, 5, 114, 0, 0, 601, 602, 5, 117, 0, 0, 602, 627, 5, 101, 0, + 0, 603, 604, 5, 84, 0, 0, 604, 605, 5, 114, 0, 0, 605, 606, 5, 117, 0, + 0, 606, 627, 5, 101, 0, 0, 607, 608, 5, 84, 0, 0, 608, 609, 5, 82, 0, 0, + 609, 610, 5, 85, 0, 0, 610, 627, 5, 69, 0, 0, 611, 612, 5, 102, 0, 0, 612, + 613, 5, 97, 0, 0, 613, 614, 5, 108, 0, 0, 614, 615, 5, 115, 0, 0, 615, + 627, 5, 101, 0, 0, 616, 617, 5, 70, 0, 0, 617, 618, 5, 97, 0, 0, 618, 619, + 5, 108, 0, 0, 619, 620, 5, 115, 0, 0, 620, 627, 5, 101, 0, 0, 621, 622, + 5, 70, 0, 0, 622, 623, 5, 65, 0, 0, 623, 624, 5, 76, 0, 0, 624, 625, 5, + 83, 0, 0, 625, 627, 5, 69, 0, 0, 626, 599, 1, 0, 0, 0, 626, 603, 1, 0, + 0, 0, 626, 607, 1, 0, 0, 0, 626, 611, 1, 0, 0, 0, 626, 616, 1, 0, 0, 0, + 626, 621, 1, 0, 0, 0, 627, 88, 1, 0, 0, 0, 628, 633, 3, 117, 58, 0, 629, + 633, 3, 119, 59, 0, 630, 633, 3, 121, 60, 0, 631, 633, 3, 115, 57, 0, 632, + 628, 1, 0, 0, 0, 632, 629, 1, 0, 0, 0, 632, 630, 1, 0, 0, 0, 632, 631, + 1, 0, 0, 0, 633, 90, 1, 0, 0, 0, 634, 637, 3, 133, 66, 0, 635, 637, 3, + 135, 67, 0, 636, 634, 1, 0, 0, 0, 636, 635, 1, 0, 0, 0, 637, 92, 1, 0, + 0, 0, 638, 643, 3, 111, 55, 0, 639, 642, 3, 111, 55, 0, 640, 642, 3, 113, + 56, 0, 641, 639, 1, 0, 0, 0, 641, 640, 1, 0, 0, 0, 642, 645, 1, 0, 0, 0, + 643, 641, 1, 0, 0, 0, 643, 644, 1, 0, 0, 0, 644, 94, 1, 0, 0, 0, 645, 643, + 1, 0, 0, 0, 646, 647, 5, 36, 0, 0, 647, 648, 5, 109, 0, 0, 648, 649, 5, + 101, 0, 0, 649, 650, 5, 116, 0, 0, 650, 651, 5, 97, 0, 0, 651, 96, 1, 0, + 0, 0, 652, 654, 3, 101, 50, 0, 653, 652, 1, 0, 0, 0, 653, 654, 1, 0, 0, + 0, 654, 665, 1, 0, 0, 0, 655, 657, 5, 34, 0, 0, 656, 658, 3, 103, 51, 0, + 657, 656, 1, 0, 0, 0, 657, 658, 1, 0, 0, 0, 658, 659, 1, 0, 0, 0, 659, + 666, 5, 34, 0, 0, 660, 662, 5, 39, 0, 0, 661, 663, 3, 105, 52, 0, 662, + 661, 1, 0, 0, 0, 662, 663, 1, 0, 0, 0, 663, 664, 1, 0, 0, 0, 664, 666, + 5, 39, 0, 0, 665, 655, 1, 0, 0, 0, 665, 660, 1, 0, 0, 0, 666, 98, 1, 0, + 0, 0, 667, 670, 3, 93, 46, 0, 668, 670, 3, 95, 47, 0, 669, 667, 1, 0, 0, + 0, 669, 668, 1, 0, 0, 0, 670, 678, 1, 0, 0, 0, 671, 674, 5, 91, 0, 0, 672, + 675, 3, 97, 48, 0, 673, 675, 3, 117, 58, 0, 674, 672, 1, 0, 0, 0, 674, + 673, 1, 0, 0, 0, 675, 676, 1, 0, 0, 0, 676, 677, 5, 93, 0, 0, 677, 679, + 1, 0, 0, 0, 678, 671, 1, 0, 0, 0, 679, 680, 1, 0, 0, 0, 680, 678, 1, 0, + 0, 0, 680, 681, 1, 0, 0, 0, 681, 100, 1, 0, 0, 0, 682, 683, 5, 117, 0, + 0, 683, 686, 5, 56, 0, 0, 684, 686, 7, 0, 0, 0, 685, 682, 1, 0, 0, 0, 685, + 684, 1, 0, 0, 0, 686, 102, 1, 0, 0, 0, 687, 689, 3, 107, 53, 0, 688, 687, + 1, 0, 0, 0, 689, 690, 1, 0, 0, 0, 690, 688, 1, 0, 0, 0, 690, 691, 1, 0, + 0, 0, 691, 104, 1, 0, 0, 0, 692, 694, 3, 109, 54, 0, 693, 692, 1, 0, 0, + 0, 694, 695, 1, 0, 0, 0, 695, 693, 1, 0, 0, 0, 695, 696, 1, 0, 0, 0, 696, + 106, 1, 0, 0, 0, 697, 705, 8, 1, 0, 0, 698, 705, 3, 149, 74, 0, 699, 700, + 5, 92, 0, 0, 700, 705, 5, 10, 0, 0, 701, 702, 5, 92, 0, 0, 702, 703, 5, + 13, 0, 0, 703, 705, 5, 10, 0, 0, 704, 697, 1, 0, 0, 0, 704, 698, 1, 0, + 0, 0, 704, 699, 1, 0, 0, 0, 704, 701, 1, 0, 0, 0, 705, 108, 1, 0, 0, 0, + 706, 714, 8, 2, 0, 0, 707, 714, 3, 149, 74, 0, 708, 709, 5, 92, 0, 0, 709, + 714, 5, 10, 0, 0, 710, 711, 5, 92, 0, 0, 711, 712, 5, 13, 0, 0, 712, 714, + 5, 10, 0, 0, 713, 706, 1, 0, 0, 0, 713, 707, 1, 0, 0, 0, 713, 708, 1, 0, + 0, 0, 713, 710, 1, 0, 0, 0, 714, 110, 1, 0, 0, 0, 715, 716, 7, 3, 0, 0, + 716, 112, 1, 0, 0, 0, 717, 718, 7, 4, 0, 0, 718, 114, 1, 0, 0, 0, 719, + 720, 5, 48, 0, 0, 720, 722, 7, 5, 0, 0, 721, 723, 7, 6, 0, 0, 722, 721, + 1, 0, 0, 0, 723, 724, 1, 0, 0, 0, 724, 722, 1, 0, 0, 0, 724, 725, 1, 0, + 0, 0, 725, 116, 1, 0, 0, 0, 726, 730, 3, 123, 61, 0, 727, 729, 3, 113, + 56, 0, 728, 727, 1, 0, 0, 0, 729, 732, 1, 0, 0, 0, 730, 728, 1, 0, 0, 0, + 730, 731, 1, 0, 0, 0, 731, 735, 1, 0, 0, 0, 732, 730, 1, 0, 0, 0, 733, + 735, 5, 48, 0, 0, 734, 726, 1, 0, 0, 0, 734, 733, 1, 0, 0, 0, 735, 118, + 1, 0, 0, 0, 736, 740, 5, 48, 0, 0, 737, 739, 3, 125, 62, 0, 738, 737, 1, + 0, 0, 0, 739, 742, 1, 0, 0, 0, 740, 738, 1, 0, 0, 0, 740, 741, 1, 0, 0, + 0, 741, 120, 1, 0, 0, 0, 742, 740, 1, 0, 0, 0, 743, 744, 5, 48, 0, 0, 744, + 745, 7, 7, 0, 0, 745, 746, 3, 145, 72, 0, 746, 122, 1, 0, 0, 0, 747, 748, + 7, 8, 0, 0, 748, 124, 1, 0, 0, 0, 749, 750, 7, 9, 0, 0, 750, 126, 1, 0, + 0, 0, 751, 752, 7, 10, 0, 0, 752, 128, 1, 0, 0, 0, 753, 754, 3, 127, 63, + 0, 754, 755, 3, 127, 63, 0, 755, 756, 3, 127, 63, 0, 756, 757, 3, 127, + 63, 0, 757, 130, 1, 0, 0, 0, 758, 759, 5, 92, 0, 0, 759, 760, 5, 117, 0, + 0, 760, 761, 1, 0, 0, 0, 761, 769, 3, 129, 64, 0, 762, 763, 5, 92, 0, 0, + 763, 764, 5, 85, 0, 0, 764, 765, 1, 0, 0, 0, 765, 766, 3, 129, 64, 0, 766, + 767, 3, 129, 64, 0, 767, 769, 1, 0, 0, 0, 768, 758, 1, 0, 0, 0, 768, 762, + 1, 0, 0, 0, 769, 132, 1, 0, 0, 0, 770, 772, 3, 137, 68, 0, 771, 773, 3, + 139, 69, 0, 772, 771, 1, 0, 0, 0, 772, 773, 1, 0, 0, 0, 773, 778, 1, 0, + 0, 0, 774, 775, 3, 141, 70, 0, 775, 776, 3, 139, 69, 0, 776, 778, 1, 0, + 0, 0, 777, 770, 1, 0, 0, 0, 777, 774, 1, 0, 0, 0, 778, 134, 1, 0, 0, 0, + 779, 780, 5, 48, 0, 0, 780, 783, 7, 7, 0, 0, 781, 784, 3, 143, 71, 0, 782, + 784, 3, 145, 72, 0, 783, 781, 1, 0, 0, 0, 783, 782, 1, 0, 0, 0, 784, 785, + 1, 0, 0, 0, 785, 786, 3, 147, 73, 0, 786, 136, 1, 0, 0, 0, 787, 789, 3, + 141, 70, 0, 788, 787, 1, 0, 0, 0, 788, 789, 1, 0, 0, 0, 789, 790, 1, 0, + 0, 0, 790, 791, 5, 46, 0, 0, 791, 796, 3, 141, 70, 0, 792, 793, 3, 141, + 70, 0, 793, 794, 5, 46, 0, 0, 794, 796, 1, 0, 0, 0, 795, 788, 1, 0, 0, + 0, 795, 792, 1, 0, 0, 0, 796, 138, 1, 0, 0, 0, 797, 799, 7, 11, 0, 0, 798, + 800, 7, 12, 0, 0, 799, 798, 1, 0, 0, 0, 799, 800, 1, 0, 0, 0, 800, 801, + 1, 0, 0, 0, 801, 802, 3, 141, 70, 0, 802, 140, 1, 0, 0, 0, 803, 805, 3, + 113, 56, 0, 804, 803, 1, 0, 0, 0, 805, 806, 1, 0, 0, 0, 806, 804, 1, 0, + 0, 0, 806, 807, 1, 0, 0, 0, 807, 142, 1, 0, 0, 0, 808, 810, 3, 145, 72, + 0, 809, 808, 1, 0, 0, 0, 809, 810, 1, 0, 0, 0, 810, 811, 1, 0, 0, 0, 811, + 812, 5, 46, 0, 0, 812, 817, 3, 145, 72, 0, 813, 814, 3, 145, 72, 0, 814, + 815, 5, 46, 0, 0, 815, 817, 1, 0, 0, 0, 816, 809, 1, 0, 0, 0, 816, 813, + 1, 0, 0, 0, 817, 144, 1, 0, 0, 0, 818, 820, 3, 127, 63, 0, 819, 818, 1, + 0, 0, 0, 820, 821, 1, 0, 0, 0, 821, 819, 1, 0, 0, 0, 821, 822, 1, 0, 0, + 0, 822, 146, 1, 0, 0, 0, 823, 825, 7, 13, 0, 0, 824, 826, 7, 12, 0, 0, + 825, 824, 1, 0, 0, 0, 825, 826, 1, 0, 0, 0, 826, 827, 1, 0, 0, 0, 827, + 828, 3, 141, 70, 0, 828, 148, 1, 0, 0, 0, 829, 830, 5, 92, 0, 0, 830, 845, + 7, 14, 0, 0, 831, 832, 5, 92, 0, 0, 832, 834, 3, 125, 62, 0, 833, 835, + 3, 125, 62, 0, 834, 833, 1, 0, 0, 0, 834, 835, 1, 0, 0, 0, 835, 837, 1, + 0, 0, 0, 836, 838, 3, 125, 62, 0, 837, 836, 1, 0, 0, 0, 837, 838, 1, 0, + 0, 0, 838, 845, 1, 0, 0, 0, 839, 840, 5, 92, 0, 0, 840, 841, 5, 120, 0, + 0, 841, 842, 1, 0, 0, 0, 842, 845, 3, 145, 72, 0, 843, 845, 3, 131, 65, + 0, 844, 829, 1, 0, 0, 0, 844, 831, 1, 0, 0, 0, 844, 839, 1, 0, 0, 0, 844, + 843, 1, 0, 0, 0, 845, 150, 1, 0, 0, 0, 846, 848, 7, 15, 0, 0, 847, 846, + 1, 0, 0, 0, 848, 849, 1, 0, 0, 0, 849, 847, 1, 0, 0, 0, 849, 850, 1, 0, + 0, 0, 850, 851, 1, 0, 0, 0, 851, 852, 6, 75, 0, 0, 852, 152, 1, 0, 0, 0, + 853, 855, 5, 13, 0, 0, 854, 856, 5, 10, 0, 0, 855, 854, 1, 0, 0, 0, 855, + 856, 1, 0, 0, 0, 856, 859, 1, 0, 0, 0, 857, 859, 5, 10, 0, 0, 858, 853, + 1, 0, 0, 0, 858, 857, 1, 0, 0, 0, 859, 860, 1, 0, 0, 0, 860, 861, 6, 76, + 0, 0, 861, 154, 1, 0, 0, 0, 59, 0, 193, 207, 229, 255, 290, 298, 314, 338, + 349, 355, 360, 362, 393, 429, 465, 495, 533, 571, 597, 626, 632, 636, 641, + 643, 653, 657, 662, 665, 669, 674, 680, 685, 690, 695, 704, 713, 724, 730, + 734, 740, 768, 772, 777, 783, 788, 795, 799, 806, 809, 816, 821, 825, 834, + 837, 844, 849, 855, 858, 1, 6, 0, 0, } deserializer := antlr.NewATNDeserializer(nil) staticData.atn = deserializer.Deserialize(staticData.serializedATN) @@ -510,39 +522,40 @@ const ( PlanLexerLIKE = 14 PlanLexerEXISTS = 15 PlanLexerTEXTMATCH = 16 - PlanLexerADD = 17 - PlanLexerSUB = 18 - PlanLexerMUL = 19 - PlanLexerDIV = 20 - PlanLexerMOD = 21 - PlanLexerPOW = 22 - PlanLexerSHL = 23 - PlanLexerSHR = 24 - PlanLexerBAND = 25 - PlanLexerBOR = 26 - PlanLexerBXOR = 27 - PlanLexerAND = 28 - PlanLexerOR = 29 - PlanLexerISNULL = 30 - PlanLexerISNOTNULL = 31 - PlanLexerBNOT = 32 - PlanLexerNOT = 33 - PlanLexerIN = 34 - PlanLexerEmptyArray = 35 - PlanLexerJSONContains = 36 - PlanLexerJSONContainsAll = 37 - PlanLexerJSONContainsAny = 38 - PlanLexerArrayContains = 39 - PlanLexerArrayContainsAll = 40 - PlanLexerArrayContainsAny = 41 - PlanLexerArrayLength = 42 - PlanLexerBooleanConstant = 43 - PlanLexerIntegerConstant = 44 - PlanLexerFloatingConstant = 45 - PlanLexerIdentifier = 46 - PlanLexerMeta = 47 - PlanLexerStringLiteral = 48 - PlanLexerJSONIdentifier = 49 - PlanLexerWhitespace = 50 - PlanLexerNewline = 51 + PlanLexerPHRASEMATCH = 17 + PlanLexerADD = 18 + PlanLexerSUB = 19 + PlanLexerMUL = 20 + PlanLexerDIV = 21 + PlanLexerMOD = 22 + PlanLexerPOW = 23 + PlanLexerSHL = 24 + PlanLexerSHR = 25 + PlanLexerBAND = 26 + PlanLexerBOR = 27 + PlanLexerBXOR = 28 + PlanLexerAND = 29 + PlanLexerOR = 30 + PlanLexerISNULL = 31 + PlanLexerISNOTNULL = 32 + PlanLexerBNOT = 33 + PlanLexerNOT = 34 + PlanLexerIN = 35 + PlanLexerEmptyArray = 36 + PlanLexerJSONContains = 37 + PlanLexerJSONContainsAll = 38 + PlanLexerJSONContainsAny = 39 + PlanLexerArrayContains = 40 + PlanLexerArrayContainsAll = 41 + PlanLexerArrayContainsAny = 42 + PlanLexerArrayLength = 43 + PlanLexerBooleanConstant = 44 + PlanLexerIntegerConstant = 45 + PlanLexerFloatingConstant = 46 + PlanLexerIdentifier = 47 + PlanLexerMeta = 48 + PlanLexerStringLiteral = 49 + PlanLexerJSONIdentifier = 50 + PlanLexerWhitespace = 51 + PlanLexerNewline = 52 ) diff --git a/internal/parser/planparserv2/generated/plan_parser.go b/internal/parser/planparserv2/generated/plan_parser.go index b1037c2f1c6bd..c00b5d6d52960 100644 --- a/internal/parser/planparserv2/generated/plan_parser.go +++ b/internal/parser/planparserv2/generated/plan_parser.go @@ -33,96 +33,100 @@ func planParserInit() { staticData := &PlanParserStaticData staticData.LiteralNames = []string{ "", "'('", "')'", "'['", "','", "']'", "'{'", "'}'", "'<'", "'<='", - "'>'", "'>='", "'=='", "'!='", "", "", "", "'+'", "'-'", "'*'", "'/'", - "'%'", "'**'", "'<<'", "'>>'", "'&'", "'|'", "'^'", "", "", "", "", - "'~'", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "'$meta'", + "'>'", "'>='", "'=='", "'!='", "", "", "", "", "'+'", "'-'", "'*'", + "'/'", "'%'", "'**'", "'<<'", "'>>'", "'&'", "'|'", "'^'", "", "", "", + "", "'~'", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "'$meta'", } staticData.SymbolicNames = []string{ "", "", "", "", "", "", "LBRACE", "RBRACE", "LT", "LE", "GT", "GE", - "EQ", "NE", "LIKE", "EXISTS", "TEXTMATCH", "ADD", "SUB", "MUL", "DIV", - "MOD", "POW", "SHL", "SHR", "BAND", "BOR", "BXOR", "AND", "OR", "ISNULL", - "ISNOTNULL", "BNOT", "NOT", "IN", "EmptyArray", "JSONContains", "JSONContainsAll", - "JSONContainsAny", "ArrayContains", "ArrayContainsAll", "ArrayContainsAny", - "ArrayLength", "BooleanConstant", "IntegerConstant", "FloatingConstant", - "Identifier", "Meta", "StringLiteral", "JSONIdentifier", "Whitespace", - "Newline", + "EQ", "NE", "LIKE", "EXISTS", "TEXTMATCH", "PHRASEMATCH", "ADD", "SUB", + "MUL", "DIV", "MOD", "POW", "SHL", "SHR", "BAND", "BOR", "BXOR", "AND", + "OR", "ISNULL", "ISNOTNULL", "BNOT", "NOT", "IN", "EmptyArray", "JSONContains", + "JSONContainsAll", "JSONContainsAny", "ArrayContains", "ArrayContainsAll", + "ArrayContainsAny", "ArrayLength", "BooleanConstant", "IntegerConstant", + "FloatingConstant", "Identifier", "Meta", "StringLiteral", "JSONIdentifier", + "Whitespace", "Newline", } staticData.RuleNames = []string{ "expr", } staticData.PredictionContextCache = antlr.NewPredictionContextCache() staticData.serializedATN = []int32{ - 4, 1, 51, 146, 2, 0, 7, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, + 4, 1, 52, 156, 2, 0, 7, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 5, 0, 21, 8, 0, 10, 0, 12, 0, 24, 9, 0, 1, 0, 3, 0, 27, 8, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, + 0, 3, 0, 45, 8, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, + 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, + 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, + 5, 0, 80, 8, 0, 10, 0, 12, 0, 83, 9, 0, 1, 0, 3, 0, 86, 8, 0, 3, 0, 88, + 8, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 3, 0, 97, 8, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, + 0, 3, 0, 113, 8, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, - 0, 1, 0, 5, 0, 70, 8, 0, 10, 0, 12, 0, 73, 9, 0, 1, 0, 3, 0, 76, 8, 0, - 3, 0, 78, 8, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 3, 0, 87, 8, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, - 0, 1, 0, 1, 0, 3, 0, 103, 8, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, - 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, - 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, - 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 5, 0, 141, 8, 0, 10, 0, 12, 0, 144, 9, - 0, 1, 0, 0, 1, 0, 1, 0, 0, 13, 1, 0, 46, 47, 2, 0, 17, 18, 32, 33, 2, 0, - 36, 36, 39, 39, 2, 0, 37, 37, 40, 40, 2, 0, 38, 38, 41, 41, 2, 0, 46, 46, - 49, 49, 1, 0, 19, 21, 1, 0, 17, 18, 1, 0, 23, 24, 1, 0, 8, 9, 1, 0, 10, - 11, 1, 0, 8, 11, 1, 0, 12, 13, 184, 0, 86, 1, 0, 0, 0, 2, 3, 6, 0, -1, - 0, 3, 87, 5, 44, 0, 0, 4, 87, 5, 45, 0, 0, 5, 87, 5, 43, 0, 0, 6, 87, 5, - 48, 0, 0, 7, 87, 7, 0, 0, 0, 8, 87, 5, 49, 0, 0, 9, 10, 5, 6, 0, 0, 10, - 11, 5, 46, 0, 0, 11, 87, 5, 7, 0, 0, 12, 13, 5, 1, 0, 0, 13, 14, 3, 0, - 0, 0, 14, 15, 5, 2, 0, 0, 15, 87, 1, 0, 0, 0, 16, 17, 5, 3, 0, 0, 17, 22, - 3, 0, 0, 0, 18, 19, 5, 4, 0, 0, 19, 21, 3, 0, 0, 0, 20, 18, 1, 0, 0, 0, - 21, 24, 1, 0, 0, 0, 22, 20, 1, 0, 0, 0, 22, 23, 1, 0, 0, 0, 23, 26, 1, - 0, 0, 0, 24, 22, 1, 0, 0, 0, 25, 27, 5, 4, 0, 0, 26, 25, 1, 0, 0, 0, 26, - 27, 1, 0, 0, 0, 27, 28, 1, 0, 0, 0, 28, 29, 5, 5, 0, 0, 29, 87, 1, 0, 0, - 0, 30, 87, 5, 35, 0, 0, 31, 32, 5, 16, 0, 0, 32, 33, 5, 1, 0, 0, 33, 34, - 5, 46, 0, 0, 34, 35, 5, 4, 0, 0, 35, 36, 5, 48, 0, 0, 36, 87, 5, 2, 0, - 0, 37, 38, 7, 1, 0, 0, 38, 87, 3, 0, 0, 22, 39, 40, 7, 2, 0, 0, 40, 41, - 5, 1, 0, 0, 41, 42, 3, 0, 0, 0, 42, 43, 5, 4, 0, 0, 43, 44, 3, 0, 0, 0, - 44, 45, 5, 2, 0, 0, 45, 87, 1, 0, 0, 0, 46, 47, 7, 3, 0, 0, 47, 48, 5, - 1, 0, 0, 48, 49, 3, 0, 0, 0, 49, 50, 5, 4, 0, 0, 50, 51, 3, 0, 0, 0, 51, - 52, 5, 2, 0, 0, 52, 87, 1, 0, 0, 0, 53, 54, 7, 4, 0, 0, 54, 55, 5, 1, 0, - 0, 55, 56, 3, 0, 0, 0, 56, 57, 5, 4, 0, 0, 57, 58, 3, 0, 0, 0, 58, 59, - 5, 2, 0, 0, 59, 87, 1, 0, 0, 0, 60, 61, 5, 42, 0, 0, 61, 62, 5, 1, 0, 0, - 62, 63, 7, 5, 0, 0, 63, 87, 5, 2, 0, 0, 64, 65, 5, 46, 0, 0, 65, 77, 5, - 1, 0, 0, 66, 71, 3, 0, 0, 0, 67, 68, 5, 4, 0, 0, 68, 70, 3, 0, 0, 0, 69, - 67, 1, 0, 0, 0, 70, 73, 1, 0, 0, 0, 71, 69, 1, 0, 0, 0, 71, 72, 1, 0, 0, - 0, 72, 75, 1, 0, 0, 0, 73, 71, 1, 0, 0, 0, 74, 76, 5, 4, 0, 0, 75, 74, - 1, 0, 0, 0, 75, 76, 1, 0, 0, 0, 76, 78, 1, 0, 0, 0, 77, 66, 1, 0, 0, 0, - 77, 78, 1, 0, 0, 0, 78, 79, 1, 0, 0, 0, 79, 87, 5, 2, 0, 0, 80, 81, 5, - 46, 0, 0, 81, 87, 5, 30, 0, 0, 82, 83, 5, 46, 0, 0, 83, 87, 5, 31, 0, 0, - 84, 85, 5, 15, 0, 0, 85, 87, 3, 0, 0, 1, 86, 2, 1, 0, 0, 0, 86, 4, 1, 0, - 0, 0, 86, 5, 1, 0, 0, 0, 86, 6, 1, 0, 0, 0, 86, 7, 1, 0, 0, 0, 86, 8, 1, - 0, 0, 0, 86, 9, 1, 0, 0, 0, 86, 12, 1, 0, 0, 0, 86, 16, 1, 0, 0, 0, 86, - 30, 1, 0, 0, 0, 86, 31, 1, 0, 0, 0, 86, 37, 1, 0, 0, 0, 86, 39, 1, 0, 0, - 0, 86, 46, 1, 0, 0, 0, 86, 53, 1, 0, 0, 0, 86, 60, 1, 0, 0, 0, 86, 64, - 1, 0, 0, 0, 86, 80, 1, 0, 0, 0, 86, 82, 1, 0, 0, 0, 86, 84, 1, 0, 0, 0, - 87, 142, 1, 0, 0, 0, 88, 89, 10, 23, 0, 0, 89, 90, 5, 22, 0, 0, 90, 141, - 3, 0, 0, 24, 91, 92, 10, 21, 0, 0, 92, 93, 7, 6, 0, 0, 93, 141, 3, 0, 0, - 22, 94, 95, 10, 20, 0, 0, 95, 96, 7, 7, 0, 0, 96, 141, 3, 0, 0, 21, 97, - 98, 10, 19, 0, 0, 98, 99, 7, 8, 0, 0, 99, 141, 3, 0, 0, 20, 100, 102, 10, - 18, 0, 0, 101, 103, 5, 33, 0, 0, 102, 101, 1, 0, 0, 0, 102, 103, 1, 0, - 0, 0, 103, 104, 1, 0, 0, 0, 104, 105, 5, 34, 0, 0, 105, 141, 3, 0, 0, 19, - 106, 107, 10, 12, 0, 0, 107, 108, 7, 9, 0, 0, 108, 109, 7, 5, 0, 0, 109, - 110, 7, 9, 0, 0, 110, 141, 3, 0, 0, 13, 111, 112, 10, 11, 0, 0, 112, 113, - 7, 10, 0, 0, 113, 114, 7, 5, 0, 0, 114, 115, 7, 10, 0, 0, 115, 141, 3, - 0, 0, 12, 116, 117, 10, 10, 0, 0, 117, 118, 7, 11, 0, 0, 118, 141, 3, 0, - 0, 11, 119, 120, 10, 9, 0, 0, 120, 121, 7, 12, 0, 0, 121, 141, 3, 0, 0, - 10, 122, 123, 10, 8, 0, 0, 123, 124, 5, 25, 0, 0, 124, 141, 3, 0, 0, 9, - 125, 126, 10, 7, 0, 0, 126, 127, 5, 27, 0, 0, 127, 141, 3, 0, 0, 8, 128, - 129, 10, 6, 0, 0, 129, 130, 5, 26, 0, 0, 130, 141, 3, 0, 0, 7, 131, 132, - 10, 5, 0, 0, 132, 133, 5, 28, 0, 0, 133, 141, 3, 0, 0, 6, 134, 135, 10, - 4, 0, 0, 135, 136, 5, 29, 0, 0, 136, 141, 3, 0, 0, 5, 137, 138, 10, 25, - 0, 0, 138, 139, 5, 14, 0, 0, 139, 141, 5, 48, 0, 0, 140, 88, 1, 0, 0, 0, - 140, 91, 1, 0, 0, 0, 140, 94, 1, 0, 0, 0, 140, 97, 1, 0, 0, 0, 140, 100, - 1, 0, 0, 0, 140, 106, 1, 0, 0, 0, 140, 111, 1, 0, 0, 0, 140, 116, 1, 0, - 0, 0, 140, 119, 1, 0, 0, 0, 140, 122, 1, 0, 0, 0, 140, 125, 1, 0, 0, 0, - 140, 128, 1, 0, 0, 0, 140, 131, 1, 0, 0, 0, 140, 134, 1, 0, 0, 0, 140, - 137, 1, 0, 0, 0, 141, 144, 1, 0, 0, 0, 142, 140, 1, 0, 0, 0, 142, 143, - 1, 0, 0, 0, 143, 1, 1, 0, 0, 0, 144, 142, 1, 0, 0, 0, 9, 22, 26, 71, 75, - 77, 86, 102, 140, 142, + 0, 1, 0, 1, 0, 1, 0, 5, 0, 151, 8, 0, 10, 0, 12, 0, 154, 9, 0, 1, 0, 0, + 1, 0, 1, 0, 0, 13, 1, 0, 47, 48, 2, 0, 18, 19, 33, 34, 2, 0, 37, 37, 40, + 40, 2, 0, 38, 38, 41, 41, 2, 0, 39, 39, 42, 42, 2, 0, 47, 47, 50, 50, 1, + 0, 20, 22, 1, 0, 18, 19, 1, 0, 24, 25, 1, 0, 8, 9, 1, 0, 10, 11, 1, 0, + 8, 11, 1, 0, 12, 13, 196, 0, 96, 1, 0, 0, 0, 2, 3, 6, 0, -1, 0, 3, 97, + 5, 45, 0, 0, 4, 97, 5, 46, 0, 0, 5, 97, 5, 44, 0, 0, 6, 97, 5, 49, 0, 0, + 7, 97, 7, 0, 0, 0, 8, 97, 5, 50, 0, 0, 9, 10, 5, 6, 0, 0, 10, 11, 5, 47, + 0, 0, 11, 97, 5, 7, 0, 0, 12, 13, 5, 1, 0, 0, 13, 14, 3, 0, 0, 0, 14, 15, + 5, 2, 0, 0, 15, 97, 1, 0, 0, 0, 16, 17, 5, 3, 0, 0, 17, 22, 3, 0, 0, 0, + 18, 19, 5, 4, 0, 0, 19, 21, 3, 0, 0, 0, 20, 18, 1, 0, 0, 0, 21, 24, 1, + 0, 0, 0, 22, 20, 1, 0, 0, 0, 22, 23, 1, 0, 0, 0, 23, 26, 1, 0, 0, 0, 24, + 22, 1, 0, 0, 0, 25, 27, 5, 4, 0, 0, 26, 25, 1, 0, 0, 0, 26, 27, 1, 0, 0, + 0, 27, 28, 1, 0, 0, 0, 28, 29, 5, 5, 0, 0, 29, 97, 1, 0, 0, 0, 30, 97, + 5, 36, 0, 0, 31, 32, 5, 16, 0, 0, 32, 33, 5, 1, 0, 0, 33, 34, 5, 47, 0, + 0, 34, 35, 5, 4, 0, 0, 35, 36, 5, 49, 0, 0, 36, 97, 5, 2, 0, 0, 37, 38, + 5, 17, 0, 0, 38, 39, 5, 1, 0, 0, 39, 40, 5, 47, 0, 0, 40, 41, 5, 4, 0, + 0, 41, 44, 5, 49, 0, 0, 42, 43, 5, 4, 0, 0, 43, 45, 5, 45, 0, 0, 44, 42, + 1, 0, 0, 0, 44, 45, 1, 0, 0, 0, 45, 46, 1, 0, 0, 0, 46, 97, 5, 2, 0, 0, + 47, 48, 7, 1, 0, 0, 48, 97, 3, 0, 0, 22, 49, 50, 7, 2, 0, 0, 50, 51, 5, + 1, 0, 0, 51, 52, 3, 0, 0, 0, 52, 53, 5, 4, 0, 0, 53, 54, 3, 0, 0, 0, 54, + 55, 5, 2, 0, 0, 55, 97, 1, 0, 0, 0, 56, 57, 7, 3, 0, 0, 57, 58, 5, 1, 0, + 0, 58, 59, 3, 0, 0, 0, 59, 60, 5, 4, 0, 0, 60, 61, 3, 0, 0, 0, 61, 62, + 5, 2, 0, 0, 62, 97, 1, 0, 0, 0, 63, 64, 7, 4, 0, 0, 64, 65, 5, 1, 0, 0, + 65, 66, 3, 0, 0, 0, 66, 67, 5, 4, 0, 0, 67, 68, 3, 0, 0, 0, 68, 69, 5, + 2, 0, 0, 69, 97, 1, 0, 0, 0, 70, 71, 5, 43, 0, 0, 71, 72, 5, 1, 0, 0, 72, + 73, 7, 5, 0, 0, 73, 97, 5, 2, 0, 0, 74, 75, 5, 47, 0, 0, 75, 87, 5, 1, + 0, 0, 76, 81, 3, 0, 0, 0, 77, 78, 5, 4, 0, 0, 78, 80, 3, 0, 0, 0, 79, 77, + 1, 0, 0, 0, 80, 83, 1, 0, 0, 0, 81, 79, 1, 0, 0, 0, 81, 82, 1, 0, 0, 0, + 82, 85, 1, 0, 0, 0, 83, 81, 1, 0, 0, 0, 84, 86, 5, 4, 0, 0, 85, 84, 1, + 0, 0, 0, 85, 86, 1, 0, 0, 0, 86, 88, 1, 0, 0, 0, 87, 76, 1, 0, 0, 0, 87, + 88, 1, 0, 0, 0, 88, 89, 1, 0, 0, 0, 89, 97, 5, 2, 0, 0, 90, 91, 5, 47, + 0, 0, 91, 97, 5, 31, 0, 0, 92, 93, 5, 47, 0, 0, 93, 97, 5, 32, 0, 0, 94, + 95, 5, 15, 0, 0, 95, 97, 3, 0, 0, 1, 96, 2, 1, 0, 0, 0, 96, 4, 1, 0, 0, + 0, 96, 5, 1, 0, 0, 0, 96, 6, 1, 0, 0, 0, 96, 7, 1, 0, 0, 0, 96, 8, 1, 0, + 0, 0, 96, 9, 1, 0, 0, 0, 96, 12, 1, 0, 0, 0, 96, 16, 1, 0, 0, 0, 96, 30, + 1, 0, 0, 0, 96, 31, 1, 0, 0, 0, 96, 37, 1, 0, 0, 0, 96, 47, 1, 0, 0, 0, + 96, 49, 1, 0, 0, 0, 96, 56, 1, 0, 0, 0, 96, 63, 1, 0, 0, 0, 96, 70, 1, + 0, 0, 0, 96, 74, 1, 0, 0, 0, 96, 90, 1, 0, 0, 0, 96, 92, 1, 0, 0, 0, 96, + 94, 1, 0, 0, 0, 97, 152, 1, 0, 0, 0, 98, 99, 10, 23, 0, 0, 99, 100, 5, + 23, 0, 0, 100, 151, 3, 0, 0, 24, 101, 102, 10, 21, 0, 0, 102, 103, 7, 6, + 0, 0, 103, 151, 3, 0, 0, 22, 104, 105, 10, 20, 0, 0, 105, 106, 7, 7, 0, + 0, 106, 151, 3, 0, 0, 21, 107, 108, 10, 19, 0, 0, 108, 109, 7, 8, 0, 0, + 109, 151, 3, 0, 0, 20, 110, 112, 10, 18, 0, 0, 111, 113, 5, 34, 0, 0, 112, + 111, 1, 0, 0, 0, 112, 113, 1, 0, 0, 0, 113, 114, 1, 0, 0, 0, 114, 115, + 5, 35, 0, 0, 115, 151, 3, 0, 0, 19, 116, 117, 10, 12, 0, 0, 117, 118, 7, + 9, 0, 0, 118, 119, 7, 5, 0, 0, 119, 120, 7, 9, 0, 0, 120, 151, 3, 0, 0, + 13, 121, 122, 10, 11, 0, 0, 122, 123, 7, 10, 0, 0, 123, 124, 7, 5, 0, 0, + 124, 125, 7, 10, 0, 0, 125, 151, 3, 0, 0, 12, 126, 127, 10, 10, 0, 0, 127, + 128, 7, 11, 0, 0, 128, 151, 3, 0, 0, 11, 129, 130, 10, 9, 0, 0, 130, 131, + 7, 12, 0, 0, 131, 151, 3, 0, 0, 10, 132, 133, 10, 8, 0, 0, 133, 134, 5, + 26, 0, 0, 134, 151, 3, 0, 0, 9, 135, 136, 10, 7, 0, 0, 136, 137, 5, 28, + 0, 0, 137, 151, 3, 0, 0, 8, 138, 139, 10, 6, 0, 0, 139, 140, 5, 27, 0, + 0, 140, 151, 3, 0, 0, 7, 141, 142, 10, 5, 0, 0, 142, 143, 5, 29, 0, 0, + 143, 151, 3, 0, 0, 6, 144, 145, 10, 4, 0, 0, 145, 146, 5, 30, 0, 0, 146, + 151, 3, 0, 0, 5, 147, 148, 10, 26, 0, 0, 148, 149, 5, 14, 0, 0, 149, 151, + 5, 49, 0, 0, 150, 98, 1, 0, 0, 0, 150, 101, 1, 0, 0, 0, 150, 104, 1, 0, + 0, 0, 150, 107, 1, 0, 0, 0, 150, 110, 1, 0, 0, 0, 150, 116, 1, 0, 0, 0, + 150, 121, 1, 0, 0, 0, 150, 126, 1, 0, 0, 0, 150, 129, 1, 0, 0, 0, 150, + 132, 1, 0, 0, 0, 150, 135, 1, 0, 0, 0, 150, 138, 1, 0, 0, 0, 150, 141, + 1, 0, 0, 0, 150, 144, 1, 0, 0, 0, 150, 147, 1, 0, 0, 0, 151, 154, 1, 0, + 0, 0, 152, 150, 1, 0, 0, 0, 152, 153, 1, 0, 0, 0, 153, 1, 1, 0, 0, 0, 154, + 152, 1, 0, 0, 0, 10, 22, 26, 44, 81, 85, 87, 96, 112, 150, 152, } deserializer := antlr.NewATNDeserializer(nil) staticData.atn = deserializer.Deserialize(staticData.serializedATN) @@ -177,41 +181,42 @@ const ( PlanParserLIKE = 14 PlanParserEXISTS = 15 PlanParserTEXTMATCH = 16 - PlanParserADD = 17 - PlanParserSUB = 18 - PlanParserMUL = 19 - PlanParserDIV = 20 - PlanParserMOD = 21 - PlanParserPOW = 22 - PlanParserSHL = 23 - PlanParserSHR = 24 - PlanParserBAND = 25 - PlanParserBOR = 26 - PlanParserBXOR = 27 - PlanParserAND = 28 - PlanParserOR = 29 - PlanParserISNULL = 30 - PlanParserISNOTNULL = 31 - PlanParserBNOT = 32 - PlanParserNOT = 33 - PlanParserIN = 34 - PlanParserEmptyArray = 35 - PlanParserJSONContains = 36 - PlanParserJSONContainsAll = 37 - PlanParserJSONContainsAny = 38 - PlanParserArrayContains = 39 - PlanParserArrayContainsAll = 40 - PlanParserArrayContainsAny = 41 - PlanParserArrayLength = 42 - PlanParserBooleanConstant = 43 - PlanParserIntegerConstant = 44 - PlanParserFloatingConstant = 45 - PlanParserIdentifier = 46 - PlanParserMeta = 47 - PlanParserStringLiteral = 48 - PlanParserJSONIdentifier = 49 - PlanParserWhitespace = 50 - PlanParserNewline = 51 + PlanParserPHRASEMATCH = 17 + PlanParserADD = 18 + PlanParserSUB = 19 + PlanParserMUL = 20 + PlanParserDIV = 21 + PlanParserMOD = 22 + PlanParserPOW = 23 + PlanParserSHL = 24 + PlanParserSHR = 25 + PlanParserBAND = 26 + PlanParserBOR = 27 + PlanParserBXOR = 28 + PlanParserAND = 29 + PlanParserOR = 30 + PlanParserISNULL = 31 + PlanParserISNOTNULL = 32 + PlanParserBNOT = 33 + PlanParserNOT = 34 + PlanParserIN = 35 + PlanParserEmptyArray = 36 + PlanParserJSONContains = 37 + PlanParserJSONContainsAll = 38 + PlanParserJSONContainsAny = 39 + PlanParserArrayContains = 40 + PlanParserArrayContainsAll = 41 + PlanParserArrayContainsAny = 42 + PlanParserArrayLength = 43 + PlanParserBooleanConstant = 44 + PlanParserIntegerConstant = 45 + PlanParserFloatingConstant = 46 + PlanParserIdentifier = 47 + PlanParserMeta = 48 + PlanParserStringLiteral = 49 + PlanParserJSONIdentifier = 50 + PlanParserWhitespace = 51 + PlanParserNewline = 52 ) // PlanParserRULE_expr is the PlanParser rule. @@ -1443,6 +1448,50 @@ func (s *AddSubContext) Accept(visitor antlr.ParseTreeVisitor) interface{} { } } +type PhraseMatchContext struct { + ExprContext +} + +func NewPhraseMatchContext(parser antlr.Parser, ctx antlr.ParserRuleContext) *PhraseMatchContext { + var p = new(PhraseMatchContext) + + InitEmptyExprContext(&p.ExprContext) + p.parser = parser + p.CopyAll(ctx.(*ExprContext)) + + return p +} + +func (s *PhraseMatchContext) GetRuleContext() antlr.RuleContext { + return s +} + +func (s *PhraseMatchContext) PHRASEMATCH() antlr.TerminalNode { + return s.GetToken(PlanParserPHRASEMATCH, 0) +} + +func (s *PhraseMatchContext) Identifier() antlr.TerminalNode { + return s.GetToken(PlanParserIdentifier, 0) +} + +func (s *PhraseMatchContext) StringLiteral() antlr.TerminalNode { + return s.GetToken(PlanParserStringLiteral, 0) +} + +func (s *PhraseMatchContext) IntegerConstant() antlr.TerminalNode { + return s.GetToken(PlanParserIntegerConstant, 0) +} + +func (s *PhraseMatchContext) Accept(visitor antlr.ParseTreeVisitor) interface{} { + switch t := visitor.(type) { + case PlanVisitor: + return t.VisitPhraseMatch(s) + + default: + return t.VisitChildren(s) + } +} + type RelationalContext struct { ExprContext op antlr.Token @@ -2439,13 +2488,13 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { var _alt int p.EnterOuterAlt(localctx, 1) - p.SetState(86) + p.SetState(96) p.GetErrorHandler().Sync(p) if p.HasError() { goto errorExit } - switch p.GetInterpreter().AdaptivePredict(p.BaseParser, p.GetTokenStream(), 5, p.GetParserRuleContext()) { + switch p.GetInterpreter().AdaptivePredict(p.BaseParser, p.GetTokenStream(), 6, p.GetParserRuleContext()) { case 1: localctx = NewIntegerContext(p, localctx) p.SetParserRuleContext(localctx) @@ -2727,11 +2776,90 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } case 12: - localctx = NewUnaryContext(p, localctx) + localctx = NewPhraseMatchContext(p, localctx) p.SetParserRuleContext(localctx) _prevctx = localctx { p.SetState(37) + p.Match(PlanParserPHRASEMATCH) + if p.HasError() { + // Recognition error - abort rule + goto errorExit + } + } + { + p.SetState(38) + p.Match(PlanParserT__0) + if p.HasError() { + // Recognition error - abort rule + goto errorExit + } + } + { + p.SetState(39) + p.Match(PlanParserIdentifier) + if p.HasError() { + // Recognition error - abort rule + goto errorExit + } + } + { + p.SetState(40) + p.Match(PlanParserT__3) + if p.HasError() { + // Recognition error - abort rule + goto errorExit + } + } + { + p.SetState(41) + p.Match(PlanParserStringLiteral) + if p.HasError() { + // Recognition error - abort rule + goto errorExit + } + } + p.SetState(44) + p.GetErrorHandler().Sync(p) + if p.HasError() { + goto errorExit + } + _la = p.GetTokenStream().LA(1) + + if _la == PlanParserT__3 { + { + p.SetState(42) + p.Match(PlanParserT__3) + if p.HasError() { + // Recognition error - abort rule + goto errorExit + } + } + { + p.SetState(43) + p.Match(PlanParserIntegerConstant) + if p.HasError() { + // Recognition error - abort rule + goto errorExit + } + } + + } + { + p.SetState(46) + p.Match(PlanParserT__1) + if p.HasError() { + // Recognition error - abort rule + goto errorExit + } + } + + case 13: + localctx = NewUnaryContext(p, localctx) + p.SetParserRuleContext(localctx) + _prevctx = localctx + { + p.SetState(47) var _lt = p.GetTokenStream().LT(1) @@ -2739,7 +2867,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { _la = p.GetTokenStream().LA(1) - if !((int64(_la) & ^0x3f) == 0 && ((int64(1)<<_la)&12885295104) != 0) { + if !((int64(_la) & ^0x3f) == 0 && ((int64(1)<<_la)&25770590208) != 0) { var _ri = p.GetErrorHandler().RecoverInline(p) localctx.(*UnaryContext).op = _ri @@ -2749,16 +2877,16 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(38) + p.SetState(48) p.expr(22) } - case 13: + case 14: localctx = NewJSONContainsContext(p, localctx) p.SetParserRuleContext(localctx) _prevctx = localctx { - p.SetState(39) + p.SetState(49) _la = p.GetTokenStream().LA(1) if !(_la == PlanParserJSONContains || _la == PlanParserArrayContains) { @@ -2769,7 +2897,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(40) + p.SetState(50) p.Match(PlanParserT__0) if p.HasError() { // Recognition error - abort rule @@ -2777,11 +2905,11 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(41) + p.SetState(51) p.expr(0) } { - p.SetState(42) + p.SetState(52) p.Match(PlanParserT__3) if p.HasError() { // Recognition error - abort rule @@ -2789,11 +2917,11 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(43) + p.SetState(53) p.expr(0) } { - p.SetState(44) + p.SetState(54) p.Match(PlanParserT__1) if p.HasError() { // Recognition error - abort rule @@ -2801,12 +2929,12 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } - case 14: + case 15: localctx = NewJSONContainsAllContext(p, localctx) p.SetParserRuleContext(localctx) _prevctx = localctx { - p.SetState(46) + p.SetState(56) _la = p.GetTokenStream().LA(1) if !(_la == PlanParserJSONContainsAll || _la == PlanParserArrayContainsAll) { @@ -2817,7 +2945,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(47) + p.SetState(57) p.Match(PlanParserT__0) if p.HasError() { // Recognition error - abort rule @@ -2825,11 +2953,11 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(48) + p.SetState(58) p.expr(0) } { - p.SetState(49) + p.SetState(59) p.Match(PlanParserT__3) if p.HasError() { // Recognition error - abort rule @@ -2837,11 +2965,11 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(50) + p.SetState(60) p.expr(0) } { - p.SetState(51) + p.SetState(61) p.Match(PlanParserT__1) if p.HasError() { // Recognition error - abort rule @@ -2849,12 +2977,12 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } - case 15: + case 16: localctx = NewJSONContainsAnyContext(p, localctx) p.SetParserRuleContext(localctx) _prevctx = localctx { - p.SetState(53) + p.SetState(63) _la = p.GetTokenStream().LA(1) if !(_la == PlanParserJSONContainsAny || _la == PlanParserArrayContainsAny) { @@ -2865,7 +2993,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(54) + p.SetState(64) p.Match(PlanParserT__0) if p.HasError() { // Recognition error - abort rule @@ -2873,11 +3001,11 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(55) + p.SetState(65) p.expr(0) } { - p.SetState(56) + p.SetState(66) p.Match(PlanParserT__3) if p.HasError() { // Recognition error - abort rule @@ -2885,11 +3013,11 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(57) + p.SetState(67) p.expr(0) } { - p.SetState(58) + p.SetState(68) p.Match(PlanParserT__1) if p.HasError() { // Recognition error - abort rule @@ -2897,12 +3025,12 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } - case 16: + case 17: localctx = NewArrayLengthContext(p, localctx) p.SetParserRuleContext(localctx) _prevctx = localctx { - p.SetState(60) + p.SetState(70) p.Match(PlanParserArrayLength) if p.HasError() { // Recognition error - abort rule @@ -2910,7 +3038,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(61) + p.SetState(71) p.Match(PlanParserT__0) if p.HasError() { // Recognition error - abort rule @@ -2918,7 +3046,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(62) + p.SetState(72) _la = p.GetTokenStream().LA(1) if !(_la == PlanParserIdentifier || _la == PlanParserJSONIdentifier) { @@ -2929,7 +3057,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(63) + p.SetState(73) p.Match(PlanParserT__1) if p.HasError() { // Recognition error - abort rule @@ -2937,12 +3065,12 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } - case 17: + case 18: localctx = NewCallContext(p, localctx) p.SetParserRuleContext(localctx) _prevctx = localctx { - p.SetState(64) + p.SetState(74) p.Match(PlanParserIdentifier) if p.HasError() { // Recognition error - abort rule @@ -2950,38 +3078,38 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(65) + p.SetState(75) p.Match(PlanParserT__0) if p.HasError() { // Recognition error - abort rule goto errorExit } } - p.SetState(77) + p.SetState(87) p.GetErrorHandler().Sync(p) if p.HasError() { goto errorExit } _la = p.GetTokenStream().LA(1) - if (int64(_la) & ^0x3f) == 0 && ((int64(1)<<_la)&1125878432497738) != 0 { + if (int64(_la) & ^0x3f) == 0 && ((int64(1)<<_la)&2251756865028170) != 0 { { - p.SetState(66) + p.SetState(76) p.expr(0) } - p.SetState(71) + p.SetState(81) p.GetErrorHandler().Sync(p) if p.HasError() { goto errorExit } - _alt = p.GetInterpreter().AdaptivePredict(p.BaseParser, p.GetTokenStream(), 2, p.GetParserRuleContext()) + _alt = p.GetInterpreter().AdaptivePredict(p.BaseParser, p.GetTokenStream(), 3, p.GetParserRuleContext()) if p.HasError() { goto errorExit } for _alt != 2 && _alt != antlr.ATNInvalidAltNumber { if _alt == 1 { { - p.SetState(67) + p.SetState(77) p.Match(PlanParserT__3) if p.HasError() { // Recognition error - abort rule @@ -2989,22 +3117,22 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(68) + p.SetState(78) p.expr(0) } } - p.SetState(73) + p.SetState(83) p.GetErrorHandler().Sync(p) if p.HasError() { goto errorExit } - _alt = p.GetInterpreter().AdaptivePredict(p.BaseParser, p.GetTokenStream(), 2, p.GetParserRuleContext()) + _alt = p.GetInterpreter().AdaptivePredict(p.BaseParser, p.GetTokenStream(), 3, p.GetParserRuleContext()) if p.HasError() { goto errorExit } } - p.SetState(75) + p.SetState(85) p.GetErrorHandler().Sync(p) if p.HasError() { goto errorExit @@ -3013,7 +3141,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { if _la == PlanParserT__3 { { - p.SetState(74) + p.SetState(84) p.Match(PlanParserT__3) if p.HasError() { // Recognition error - abort rule @@ -3025,7 +3153,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } { - p.SetState(79) + p.SetState(89) p.Match(PlanParserT__1) if p.HasError() { // Recognition error - abort rule @@ -3033,12 +3161,12 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } - case 18: + case 19: localctx = NewIsNullContext(p, localctx) p.SetParserRuleContext(localctx) _prevctx = localctx { - p.SetState(80) + p.SetState(90) p.Match(PlanParserIdentifier) if p.HasError() { // Recognition error - abort rule @@ -3046,7 +3174,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(81) + p.SetState(91) p.Match(PlanParserISNULL) if p.HasError() { // Recognition error - abort rule @@ -3054,12 +3182,12 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } - case 19: + case 20: localctx = NewIsNotNullContext(p, localctx) p.SetParserRuleContext(localctx) _prevctx = localctx { - p.SetState(82) + p.SetState(92) p.Match(PlanParserIdentifier) if p.HasError() { // Recognition error - abort rule @@ -3067,7 +3195,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(83) + p.SetState(93) p.Match(PlanParserISNOTNULL) if p.HasError() { // Recognition error - abort rule @@ -3075,12 +3203,12 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } - case 20: + case 21: localctx = NewExistsContext(p, localctx) p.SetParserRuleContext(localctx) _prevctx = localctx { - p.SetState(84) + p.SetState(94) p.Match(PlanParserEXISTS) if p.HasError() { // Recognition error - abort rule @@ -3088,7 +3216,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(85) + p.SetState(95) p.expr(1) } @@ -3096,12 +3224,12 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { goto errorExit } p.GetParserRuleContext().SetStop(p.GetTokenStream().LT(-1)) - p.SetState(142) + p.SetState(152) p.GetErrorHandler().Sync(p) if p.HasError() { goto errorExit } - _alt = p.GetInterpreter().AdaptivePredict(p.BaseParser, p.GetTokenStream(), 8, p.GetParserRuleContext()) + _alt = p.GetInterpreter().AdaptivePredict(p.BaseParser, p.GetTokenStream(), 9, p.GetParserRuleContext()) if p.HasError() { goto errorExit } @@ -3111,24 +3239,24 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { p.TriggerExitRuleEvent() } _prevctx = localctx - p.SetState(140) + p.SetState(150) p.GetErrorHandler().Sync(p) if p.HasError() { goto errorExit } - switch p.GetInterpreter().AdaptivePredict(p.BaseParser, p.GetTokenStream(), 7, p.GetParserRuleContext()) { + switch p.GetInterpreter().AdaptivePredict(p.BaseParser, p.GetTokenStream(), 8, p.GetParserRuleContext()) { case 1: localctx = NewPowerContext(p, NewExprContext(p, _parentctx, _parentState)) p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr) - p.SetState(88) + p.SetState(98) if !(p.Precpred(p.GetParserRuleContext(), 23)) { p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 23)", "")) goto errorExit } { - p.SetState(89) + p.SetState(99) p.Match(PlanParserPOW) if p.HasError() { // Recognition error - abort rule @@ -3136,21 +3264,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(90) + p.SetState(100) p.expr(24) } case 2: localctx = NewMulDivModContext(p, NewExprContext(p, _parentctx, _parentState)) p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr) - p.SetState(91) + p.SetState(101) if !(p.Precpred(p.GetParserRuleContext(), 21)) { p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 21)", "")) goto errorExit } { - p.SetState(92) + p.SetState(102) var _lt = p.GetTokenStream().LT(1) @@ -3158,7 +3286,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { _la = p.GetTokenStream().LA(1) - if !((int64(_la) & ^0x3f) == 0 && ((int64(1)<<_la)&3670016) != 0) { + if !((int64(_la) & ^0x3f) == 0 && ((int64(1)<<_la)&7340032) != 0) { var _ri = p.GetErrorHandler().RecoverInline(p) localctx.(*MulDivModContext).op = _ri @@ -3168,21 +3296,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(93) + p.SetState(103) p.expr(22) } case 3: localctx = NewAddSubContext(p, NewExprContext(p, _parentctx, _parentState)) p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr) - p.SetState(94) + p.SetState(104) if !(p.Precpred(p.GetParserRuleContext(), 20)) { p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 20)", "")) goto errorExit } { - p.SetState(95) + p.SetState(105) var _lt = p.GetTokenStream().LT(1) @@ -3200,21 +3328,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(96) + p.SetState(106) p.expr(21) } case 4: localctx = NewShiftContext(p, NewExprContext(p, _parentctx, _parentState)) p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr) - p.SetState(97) + p.SetState(107) if !(p.Precpred(p.GetParserRuleContext(), 19)) { p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 19)", "")) goto errorExit } { - p.SetState(98) + p.SetState(108) var _lt = p.GetTokenStream().LT(1) @@ -3232,20 +3360,20 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(99) + p.SetState(109) p.expr(20) } case 5: localctx = NewTermContext(p, NewExprContext(p, _parentctx, _parentState)) p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr) - p.SetState(100) + p.SetState(110) if !(p.Precpred(p.GetParserRuleContext(), 18)) { p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 18)", "")) goto errorExit } - p.SetState(102) + p.SetState(112) p.GetErrorHandler().Sync(p) if p.HasError() { goto errorExit @@ -3254,7 +3382,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { if _la == PlanParserNOT { { - p.SetState(101) + p.SetState(111) var _m = p.Match(PlanParserNOT) @@ -3267,7 +3395,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } { - p.SetState(104) + p.SetState(114) p.Match(PlanParserIN) if p.HasError() { // Recognition error - abort rule @@ -3275,21 +3403,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(105) + p.SetState(115) p.expr(19) } case 6: localctx = NewRangeContext(p, NewExprContext(p, _parentctx, _parentState)) p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr) - p.SetState(106) + p.SetState(116) if !(p.Precpred(p.GetParserRuleContext(), 12)) { p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 12)", "")) goto errorExit } { - p.SetState(107) + p.SetState(117) var _lt = p.GetTokenStream().LT(1) @@ -3307,7 +3435,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(108) + p.SetState(118) _la = p.GetTokenStream().LA(1) if !(_la == PlanParserIdentifier || _la == PlanParserJSONIdentifier) { @@ -3318,7 +3446,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(109) + p.SetState(119) var _lt = p.GetTokenStream().LT(1) @@ -3336,21 +3464,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(110) + p.SetState(120) p.expr(13) } case 7: localctx = NewReverseRangeContext(p, NewExprContext(p, _parentctx, _parentState)) p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr) - p.SetState(111) + p.SetState(121) if !(p.Precpred(p.GetParserRuleContext(), 11)) { p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 11)", "")) goto errorExit } { - p.SetState(112) + p.SetState(122) var _lt = p.GetTokenStream().LT(1) @@ -3368,7 +3496,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(113) + p.SetState(123) _la = p.GetTokenStream().LA(1) if !(_la == PlanParserIdentifier || _la == PlanParserJSONIdentifier) { @@ -3379,7 +3507,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(114) + p.SetState(124) var _lt = p.GetTokenStream().LT(1) @@ -3397,21 +3525,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(115) + p.SetState(125) p.expr(12) } case 8: localctx = NewRelationalContext(p, NewExprContext(p, _parentctx, _parentState)) p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr) - p.SetState(116) + p.SetState(126) if !(p.Precpred(p.GetParserRuleContext(), 10)) { p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 10)", "")) goto errorExit } { - p.SetState(117) + p.SetState(127) var _lt = p.GetTokenStream().LT(1) @@ -3429,21 +3557,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(118) + p.SetState(128) p.expr(11) } case 9: localctx = NewEqualityContext(p, NewExprContext(p, _parentctx, _parentState)) p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr) - p.SetState(119) + p.SetState(129) if !(p.Precpred(p.GetParserRuleContext(), 9)) { p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 9)", "")) goto errorExit } { - p.SetState(120) + p.SetState(130) var _lt = p.GetTokenStream().LT(1) @@ -3461,21 +3589,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(121) + p.SetState(131) p.expr(10) } case 10: localctx = NewBitAndContext(p, NewExprContext(p, _parentctx, _parentState)) p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr) - p.SetState(122) + p.SetState(132) if !(p.Precpred(p.GetParserRuleContext(), 8)) { p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 8)", "")) goto errorExit } { - p.SetState(123) + p.SetState(133) p.Match(PlanParserBAND) if p.HasError() { // Recognition error - abort rule @@ -3483,21 +3611,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(124) + p.SetState(134) p.expr(9) } case 11: localctx = NewBitXorContext(p, NewExprContext(p, _parentctx, _parentState)) p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr) - p.SetState(125) + p.SetState(135) if !(p.Precpred(p.GetParserRuleContext(), 7)) { p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 7)", "")) goto errorExit } { - p.SetState(126) + p.SetState(136) p.Match(PlanParserBXOR) if p.HasError() { // Recognition error - abort rule @@ -3505,21 +3633,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(127) + p.SetState(137) p.expr(8) } case 12: localctx = NewBitOrContext(p, NewExprContext(p, _parentctx, _parentState)) p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr) - p.SetState(128) + p.SetState(138) if !(p.Precpred(p.GetParserRuleContext(), 6)) { p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 6)", "")) goto errorExit } { - p.SetState(129) + p.SetState(139) p.Match(PlanParserBOR) if p.HasError() { // Recognition error - abort rule @@ -3527,21 +3655,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(130) + p.SetState(140) p.expr(7) } case 13: localctx = NewLogicalAndContext(p, NewExprContext(p, _parentctx, _parentState)) p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr) - p.SetState(131) + p.SetState(141) if !(p.Precpred(p.GetParserRuleContext(), 5)) { p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 5)", "")) goto errorExit } { - p.SetState(132) + p.SetState(142) p.Match(PlanParserAND) if p.HasError() { // Recognition error - abort rule @@ -3549,21 +3677,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(133) + p.SetState(143) p.expr(6) } case 14: localctx = NewLogicalOrContext(p, NewExprContext(p, _parentctx, _parentState)) p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr) - p.SetState(134) + p.SetState(144) if !(p.Precpred(p.GetParserRuleContext(), 4)) { p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 4)", "")) goto errorExit } { - p.SetState(135) + p.SetState(145) p.Match(PlanParserOR) if p.HasError() { // Recognition error - abort rule @@ -3571,21 +3699,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(136) + p.SetState(146) p.expr(5) } case 15: localctx = NewLikeContext(p, NewExprContext(p, _parentctx, _parentState)) p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr) - p.SetState(137) + p.SetState(147) - if !(p.Precpred(p.GetParserRuleContext(), 25)) { - p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 25)", "")) + if !(p.Precpred(p.GetParserRuleContext(), 26)) { + p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 26)", "")) goto errorExit } { - p.SetState(138) + p.SetState(148) p.Match(PlanParserLIKE) if p.HasError() { // Recognition error - abort rule @@ -3593,7 +3721,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } { - p.SetState(139) + p.SetState(149) p.Match(PlanParserStringLiteral) if p.HasError() { // Recognition error - abort rule @@ -3606,12 +3734,12 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) { } } - p.SetState(144) + p.SetState(154) p.GetErrorHandler().Sync(p) if p.HasError() { goto errorExit } - _alt = p.GetInterpreter().AdaptivePredict(p.BaseParser, p.GetTokenStream(), 8, p.GetParserRuleContext()) + _alt = p.GetInterpreter().AdaptivePredict(p.BaseParser, p.GetTokenStream(), 9, p.GetParserRuleContext()) if p.HasError() { goto errorExit } @@ -3689,7 +3817,7 @@ func (p *PlanParser) Expr_Sempred(localctx antlr.RuleContext, predIndex int) boo return p.Precpred(p.GetParserRuleContext(), 4) case 14: - return p.Precpred(p.GetParserRuleContext(), 25) + return p.Precpred(p.GetParserRuleContext(), 26) default: panic("No predicate with index: " + fmt.Sprint(predIndex)) diff --git a/internal/parser/planparserv2/generated/plan_visitor.go b/internal/parser/planparserv2/generated/plan_visitor.go index 29f923ab1fc36..264996e51c098 100644 --- a/internal/parser/planparserv2/generated/plan_visitor.go +++ b/internal/parser/planparserv2/generated/plan_visitor.go @@ -67,6 +67,9 @@ type PlanVisitor interface { // Visit a parse tree produced by PlanParser#AddSub. VisitAddSub(ctx *AddSubContext) interface{} + // Visit a parse tree produced by PlanParser#PhraseMatch. + VisitPhraseMatch(ctx *PhraseMatchContext) interface{} + // Visit a parse tree produced by PlanParser#Relational. VisitRelational(ctx *RelationalContext) interface{} diff --git a/internal/parser/planparserv2/parser_visitor.go b/internal/parser/planparserv2/parser_visitor.go index 95fa0c4f14945..633e19d4fa9ee 100644 --- a/internal/parser/planparserv2/parser_visitor.go +++ b/internal/parser/planparserv2/parser_visitor.go @@ -513,6 +513,41 @@ func (v *ParserVisitor) VisitTextMatch(ctx *parser.TextMatchContext) interface{} } } +func (v *ParserVisitor) VisitPhraseMatch(ctx *parser.PhraseMatchContext) interface{} { + column, err := v.translateIdentifier(ctx.Identifier().GetText()) + if err != nil { + return err + } + if !typeutil.IsStringType(column.dataType) { + return fmt.Errorf("phrase match operation on non-string is unsupported") + } + queryText, err := convertEscapeSingle(ctx.StringLiteral().GetText()) + if err != nil { + return err + } + var slop int64 + if ctx.IntegerConstant() != nil { + slop, err = strconv.ParseInt(ctx.IntegerConstant().GetText(), 10, 64) + if err != nil { + return err + } + } + + return &ExprWithType{ + expr: &planpb.Expr{ + Expr: &planpb.Expr_UnaryRangeExpr{ + UnaryRangeExpr: &planpb.UnaryRangeExpr{ + ColumnInfo: toColumnInfo(column), + Op: planpb.OpType_PhraseMatch, + Value: NewString(queryText), + ExtraValues: []*planpb.GenericValue{NewInt(slop)}, + }, + }, + }, + dataType: schemapb.DataType_Bool, + } +} + // VisitTerm translates expr to term plan. func (v *ParserVisitor) VisitTerm(ctx *parser.TermContext) interface{} { child := ctx.Expr(0).Accept(v) diff --git a/internal/parser/planparserv2/plan_parser_v2_test.go b/internal/parser/planparserv2/plan_parser_v2_test.go index b9c9a737c4d73..f33dac1aa6501 100644 --- a/internal/parser/planparserv2/plan_parser_v2_test.go +++ b/internal/parser/planparserv2/plan_parser_v2_test.go @@ -250,6 +250,31 @@ func TestExpr_TextMatch(t *testing.T) { } } +func TestExpr_PhraseMatch(t *testing.T) { + schema := newTestSchema(true) + helper, err := typeutil.CreateSchemaHelper(schema) + assert.NoError(t, err) + + exprStrs := []string{ + `phrase_match(VarCharField, "phrase")`, + `phrase_match(StringField, "phrase")`, + `phrase_match(StringField, "phrase", 1)`, + `phrase_match(VarCharField, "phrase", 11223)`, + } + for _, exprStr := range exprStrs { + assertValidExpr(t, helper, exprStr) + } + + unsupported := []string{ + `phrase_match(not_exist, "phrase")`, + `phrase_match(BoolField, "phrase")`, + `phrase_match(StringField, "phrase", -1)`, + } + for _, exprStr := range unsupported { + assertInvalidExpr(t, helper, exprStr) + } +} + func TestExpr_IsNull(t *testing.T) { schema := newTestSchema(false) schema.EnableDynamicField = false diff --git a/internal/parser/planparserv2/show_visitor.go b/internal/parser/planparserv2/show_visitor.go index ff8d80394943a..ed647b1d3e0a1 100644 --- a/internal/parser/planparserv2/show_visitor.go +++ b/internal/parser/planparserv2/show_visitor.go @@ -127,6 +127,11 @@ func (v *ShowExprVisitor) VisitUnaryRangeExpr(expr *planpb.UnaryRangeExpr) inter js["op"] = expr.Op.String() js["column_info"] = extractColumnInfo(expr.GetColumnInfo()) js["operand"] = extractGenericValue(expr.Value) + var extraValues []interface{} + for _, v := range expr.ExtraValues { + extraValues = append(extraValues, extractGenericValue(v)) + } + js["extra_values"] = extraValues return js } diff --git a/pkg/proto/plan.proto b/pkg/proto/plan.proto index 43c4494bd9ccb..e8abed2846d99 100644 --- a/pkg/proto/plan.proto +++ b/pkg/proto/plan.proto @@ -19,6 +19,7 @@ enum OpType { In = 11; // TODO:: used for term expr NotIn = 12; TextMatch = 13; // text match + PhraseMatch = 14; // phrase match }; enum ArithOpType { @@ -106,6 +107,7 @@ message UnaryRangeExpr { OpType op = 2; GenericValue value = 3; string template_variable_name = 4; + repeated GenericValue extra_values = 5; } message BinaryRangeExpr { diff --git a/pkg/proto/planpb/plan.pb.go b/pkg/proto/planpb/plan.pb.go index 7c8ec954c7bb3..655c3f07621d6 100644 --- a/pkg/proto/planpb/plan.pb.go +++ b/pkg/proto/planpb/plan.pb.go @@ -38,6 +38,7 @@ const ( OpType_In OpType = 11 // TODO:: used for term expr OpType_NotIn OpType = 12 OpType_TextMatch OpType = 13 // text match + OpType_PhraseMatch OpType = 14 // phrase match ) // Enum value maps for OpType. @@ -57,6 +58,7 @@ var ( 11: "In", 12: "NotIn", 13: "TextMatch", + 14: "PhraseMatch", } OpType_value = map[string]int32{ "Invalid": 0, @@ -73,6 +75,7 @@ var ( "In": 11, "NotIn": 12, "TextMatch": 13, + "PhraseMatch": 14, } ) @@ -1068,10 +1071,11 @@ type UnaryRangeExpr struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - ColumnInfo *ColumnInfo `protobuf:"bytes,1,opt,name=column_info,json=columnInfo,proto3" json:"column_info,omitempty"` - Op OpType `protobuf:"varint,2,opt,name=op,proto3,enum=milvus.proto.plan.OpType" json:"op,omitempty"` - Value *GenericValue `protobuf:"bytes,3,opt,name=value,proto3" json:"value,omitempty"` - TemplateVariableName string `protobuf:"bytes,4,opt,name=template_variable_name,json=templateVariableName,proto3" json:"template_variable_name,omitempty"` + ColumnInfo *ColumnInfo `protobuf:"bytes,1,opt,name=column_info,json=columnInfo,proto3" json:"column_info,omitempty"` + Op OpType `protobuf:"varint,2,opt,name=op,proto3,enum=milvus.proto.plan.OpType" json:"op,omitempty"` + Value *GenericValue `protobuf:"bytes,3,opt,name=value,proto3" json:"value,omitempty"` + TemplateVariableName string `protobuf:"bytes,4,opt,name=template_variable_name,json=templateVariableName,proto3" json:"template_variable_name,omitempty"` + ExtraValues []*GenericValue `protobuf:"bytes,5,rep,name=extra_values,json=extraValues,proto3" json:"extra_values,omitempty"` } func (x *UnaryRangeExpr) Reset() { @@ -1134,6 +1138,13 @@ func (x *UnaryRangeExpr) GetTemplateVariableName() string { return "" } +func (x *UnaryRangeExpr) GetExtraValues() []*GenericValue { + if x != nil { + return x.ExtraValues + } + return nil +} + type BinaryRangeExpr struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -2563,7 +2574,7 @@ var file_plan_proto_rawDesc = []byte{ 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x14, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, - 0x65, 0x22, 0xe8, 0x01, 0x0a, 0x0e, 0x55, 0x6e, 0x61, 0x72, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, + 0x65, 0x22, 0xac, 0x02, 0x0a, 0x0e, 0x55, 0x6e, 0x61, 0x72, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x3e, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, @@ -2577,321 +2588,327 @@ var file_plan_proto_rawDesc = []byte{ 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x34, 0x0a, 0x16, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x14, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, - 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xa9, 0x03, 0x0a, - 0x0f, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, - 0x12, 0x3e, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, - 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, - 0x12, 0x27, 0x0a, 0x0f, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x73, - 0x69, 0x76, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x6c, 0x6f, 0x77, 0x65, 0x72, - 0x49, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x75, 0x70, 0x70, - 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, 0x65, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x0e, 0x75, 0x70, 0x70, 0x65, 0x72, 0x49, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, - 0x76, 0x65, 0x12, 0x40, 0x0a, 0x0b, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x5f, 0x76, 0x61, 0x6c, 0x75, - 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x47, 0x65, 0x6e, 0x65, - 0x72, 0x69, 0x63, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0a, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x56, - 0x61, 0x6c, 0x75, 0x65, 0x12, 0x40, 0x0a, 0x0b, 0x75, 0x70, 0x70, 0x65, 0x72, 0x5f, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x47, 0x65, - 0x6e, 0x65, 0x72, 0x69, 0x63, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0a, 0x75, 0x70, 0x70, 0x65, - 0x72, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x3f, 0x0a, 0x1c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x5f, - 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, - 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x19, 0x6c, 0x6f, - 0x77, 0x65, 0x72, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, - 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3f, 0x0a, 0x1c, 0x75, 0x70, 0x70, 0x65, 0x72, - 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x62, - 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x19, 0x75, - 0x70, 0x70, 0x65, 0x72, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x56, 0x61, 0x72, 0x69, - 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x79, 0x0a, 0x08, 0x43, 0x61, 0x6c, 0x6c, - 0x45, 0x78, 0x70, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x66, 0x75, 0x6e, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x48, 0x0a, 0x13, 0x66, 0x75, 0x6e, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x73, - 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, - 0x12, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, - 0x65, 0x72, 0x73, 0x22, 0xcc, 0x01, 0x0a, 0x0b, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x45, - 0x78, 0x70, 0x72, 0x12, 0x47, 0x0a, 0x10, 0x6c, 0x65, 0x66, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x75, - 0x6d, 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, + 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x42, 0x0a, 0x0c, + 0x65, 0x78, 0x74, 0x72, 0x61, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63, 0x56, 0x61, + 0x6c, 0x75, 0x65, 0x52, 0x0b, 0x65, 0x78, 0x74, 0x72, 0x61, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x73, + 0x22, 0xa9, 0x03, 0x0a, 0x0f, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, + 0x45, 0x78, 0x70, 0x72, 0x12, 0x3e, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x69, + 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, + 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x27, 0x0a, 0x0f, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x5f, 0x69, 0x6e, + 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x6c, + 0x6f, 0x77, 0x65, 0x72, 0x49, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, 0x65, 0x12, 0x27, 0x0a, + 0x0f, 0x75, 0x70, 0x70, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, 0x65, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x75, 0x70, 0x70, 0x65, 0x72, 0x49, 0x6e, 0x63, + 0x6c, 0x75, 0x73, 0x69, 0x76, 0x65, 0x12, 0x40, 0x0a, 0x0b, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x5f, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, + 0x47, 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0a, 0x6c, 0x6f, + 0x77, 0x65, 0x72, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x40, 0x0a, 0x0b, 0x75, 0x70, 0x70, 0x65, + 0x72, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, - 0x6e, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0e, 0x6c, 0x65, - 0x66, 0x74, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x49, 0x0a, 0x11, - 0x72, 0x69, 0x67, 0x68, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x69, 0x6e, 0x66, - 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, 0x6c, 0x75, - 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0f, 0x72, 0x69, 0x67, 0x68, 0x74, 0x43, 0x6f, 0x6c, - 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x29, 0x0a, 0x02, 0x6f, 0x70, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x4f, 0x70, 0x54, 0x79, 0x70, 0x65, 0x52, 0x02, - 0x6f, 0x70, 0x22, 0xd9, 0x01, 0x0a, 0x08, 0x54, 0x65, 0x72, 0x6d, 0x45, 0x78, 0x70, 0x72, 0x12, - 0x3e, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, - 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, - 0x37, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, - 0x6c, 0x61, 0x6e, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63, 0x56, 0x61, 0x6c, 0x75, 0x65, - 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x12, 0x1e, 0x0a, 0x0b, 0x69, 0x73, 0x5f, 0x69, - 0x6e, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x69, - 0x73, 0x49, 0x6e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x12, 0x34, 0x0a, 0x16, 0x74, 0x65, 0x6d, 0x70, - 0x6c, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, - 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x14, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, - 0x74, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xf6, - 0x02, 0x0a, 0x10, 0x4a, 0x53, 0x4f, 0x4e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x45, + 0x6e, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0a, + 0x75, 0x70, 0x70, 0x65, 0x72, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x3f, 0x0a, 0x1c, 0x6c, 0x6f, + 0x77, 0x65, 0x72, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x61, 0x72, + 0x69, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x19, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x56, + 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3f, 0x0a, 0x1c, 0x75, + 0x70, 0x70, 0x65, 0x72, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x61, + 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x19, 0x75, 0x70, 0x70, 0x65, 0x72, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, + 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x79, 0x0a, 0x08, + 0x43, 0x61, 0x6c, 0x6c, 0x45, 0x78, 0x70, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x66, 0x75, 0x6e, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0c, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x48, 0x0a, + 0x13, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x65, + 0x74, 0x65, 0x72, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, + 0x78, 0x70, 0x72, 0x52, 0x12, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x72, + 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x73, 0x22, 0xcc, 0x01, 0x0a, 0x0b, 0x43, 0x6f, 0x6d, 0x70, + 0x61, 0x72, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x47, 0x0a, 0x10, 0x6c, 0x65, 0x66, 0x74, 0x5f, + 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, + 0x52, 0x0e, 0x6c, 0x65, 0x66, 0x74, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, + 0x12, 0x49, 0x0a, 0x11, 0x72, 0x69, 0x67, 0x68, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, + 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, + 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0f, 0x72, 0x69, 0x67, 0x68, + 0x74, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x29, 0x0a, 0x02, 0x6f, + 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x4f, 0x70, 0x54, 0x79, + 0x70, 0x65, 0x52, 0x02, 0x6f, 0x70, 0x22, 0xd9, 0x01, 0x0a, 0x08, 0x54, 0x65, 0x72, 0x6d, 0x45, 0x78, 0x70, 0x72, 0x12, 0x3e, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, - 0x6e, 0x66, 0x6f, 0x12, 0x3b, 0x0a, 0x08, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, - 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x69, - 0x63, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x08, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, - 0x12, 0x3a, 0x0a, 0x02, 0x6f, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2a, 0x2e, 0x6d, + 0x6e, 0x66, 0x6f, 0x12, 0x37, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x02, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63, 0x56, + 0x61, 0x6c, 0x75, 0x65, 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x12, 0x1e, 0x0a, 0x0b, + 0x69, 0x73, 0x5f, 0x69, 0x6e, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x09, 0x69, 0x73, 0x49, 0x6e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x12, 0x34, 0x0a, 0x16, + 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, + 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x14, 0x74, 0x65, + 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, + 0x6d, 0x65, 0x22, 0xf6, 0x02, 0x0a, 0x10, 0x4a, 0x53, 0x4f, 0x4e, 0x43, 0x6f, 0x6e, 0x74, 0x61, + 0x69, 0x6e, 0x73, 0x45, 0x78, 0x70, 0x72, 0x12, 0x3e, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, + 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, - 0x2e, 0x4a, 0x53, 0x4f, 0x4e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x45, 0x78, 0x70, - 0x72, 0x2e, 0x4a, 0x53, 0x4f, 0x4e, 0x4f, 0x70, 0x52, 0x02, 0x6f, 0x70, 0x12, 0x2c, 0x0a, 0x12, - 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x73, 0x61, 0x6d, 0x65, 0x5f, 0x74, 0x79, - 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, - 0x74, 0x73, 0x53, 0x61, 0x6d, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x34, 0x0a, 0x16, 0x74, 0x65, - 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x5f, - 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x14, 0x74, 0x65, 0x6d, 0x70, - 0x6c, 0x61, 0x74, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, - 0x22, 0x45, 0x0a, 0x06, 0x4a, 0x53, 0x4f, 0x4e, 0x4f, 0x70, 0x12, 0x0b, 0x0a, 0x07, 0x49, 0x6e, - 0x76, 0x61, 0x6c, 0x69, 0x64, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x43, 0x6f, 0x6e, 0x74, 0x61, - 0x69, 0x6e, 0x73, 0x10, 0x01, 0x12, 0x0f, 0x0a, 0x0b, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, - 0x73, 0x41, 0x6c, 0x6c, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, - 0x6e, 0x73, 0x41, 0x6e, 0x79, 0x10, 0x03, 0x22, 0xb0, 0x01, 0x0a, 0x08, 0x4e, 0x75, 0x6c, 0x6c, - 0x45, 0x78, 0x70, 0x72, 0x12, 0x3e, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x69, - 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, - 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, - 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x32, 0x0a, 0x02, 0x6f, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, - 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x4e, 0x75, 0x6c, 0x6c, 0x45, 0x78, 0x70, 0x72, 0x2e, 0x4e, 0x75, - 0x6c, 0x6c, 0x4f, 0x70, 0x52, 0x02, 0x6f, 0x70, 0x22, 0x30, 0x0a, 0x06, 0x4e, 0x75, 0x6c, 0x6c, - 0x4f, 0x70, 0x12, 0x0b, 0x0a, 0x07, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x10, 0x00, 0x12, - 0x0a, 0x0a, 0x06, 0x49, 0x73, 0x4e, 0x75, 0x6c, 0x6c, 0x10, 0x01, 0x12, 0x0d, 0x0a, 0x09, 0x49, - 0x73, 0x4e, 0x6f, 0x74, 0x4e, 0x75, 0x6c, 0x6c, 0x10, 0x02, 0x22, 0x91, 0x01, 0x0a, 0x09, 0x55, - 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x12, 0x34, 0x0a, 0x02, 0x6f, 0x70, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x55, 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, - 0x70, 0x72, 0x2e, 0x55, 0x6e, 0x61, 0x72, 0x79, 0x4f, 0x70, 0x52, 0x02, 0x6f, 0x70, 0x12, 0x2d, - 0x0a, 0x05, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, + 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x63, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x3b, 0x0a, 0x08, 0x65, 0x6c, 0x65, 0x6d, 0x65, + 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x47, 0x65, + 0x6e, 0x65, 0x72, 0x69, 0x63, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x08, 0x65, 0x6c, 0x65, 0x6d, + 0x65, 0x6e, 0x74, 0x73, 0x12, 0x3a, 0x0a, 0x02, 0x6f, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x4a, 0x53, 0x4f, 0x4e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, + 0x73, 0x45, 0x78, 0x70, 0x72, 0x2e, 0x4a, 0x53, 0x4f, 0x4e, 0x4f, 0x70, 0x52, 0x02, 0x6f, 0x70, + 0x12, 0x2c, 0x0a, 0x12, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x73, 0x61, 0x6d, + 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x65, 0x6c, + 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x53, 0x61, 0x6d, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x34, + 0x0a, 0x16, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, + 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x14, + 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, + 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x45, 0x0a, 0x06, 0x4a, 0x53, 0x4f, 0x4e, 0x4f, 0x70, 0x12, 0x0b, + 0x0a, 0x07, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x43, + 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x10, 0x01, 0x12, 0x0f, 0x0a, 0x0b, 0x43, 0x6f, 0x6e, + 0x74, 0x61, 0x69, 0x6e, 0x73, 0x41, 0x6c, 0x6c, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, 0x43, 0x6f, + 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x41, 0x6e, 0x79, 0x10, 0x03, 0x22, 0xb0, 0x01, 0x0a, 0x08, + 0x4e, 0x75, 0x6c, 0x6c, 0x45, 0x78, 0x70, 0x72, 0x12, 0x3e, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, + 0x6d, 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, - 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x05, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x22, 0x1f, 0x0a, - 0x07, 0x55, 0x6e, 0x61, 0x72, 0x79, 0x4f, 0x70, 0x12, 0x0b, 0x0a, 0x07, 0x49, 0x6e, 0x76, 0x61, - 0x6c, 0x69, 0x64, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x4e, 0x6f, 0x74, 0x10, 0x01, 0x22, 0xd8, - 0x01, 0x0a, 0x0a, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x12, 0x36, 0x0a, - 0x02, 0x6f, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x42, 0x69, - 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x2e, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x4f, - 0x70, 0x52, 0x02, 0x6f, 0x70, 0x12, 0x2b, 0x0a, 0x04, 0x6c, 0x65, 0x66, 0x74, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x04, 0x6c, 0x65, - 0x66, 0x74, 0x12, 0x2d, 0x0a, 0x05, 0x72, 0x69, 0x67, 0x68, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x6e, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x63, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x32, 0x0a, 0x02, 0x6f, 0x70, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0e, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x4e, 0x75, 0x6c, 0x6c, 0x45, 0x78, 0x70, + 0x72, 0x2e, 0x4e, 0x75, 0x6c, 0x6c, 0x4f, 0x70, 0x52, 0x02, 0x6f, 0x70, 0x22, 0x30, 0x0a, 0x06, + 0x4e, 0x75, 0x6c, 0x6c, 0x4f, 0x70, 0x12, 0x0b, 0x0a, 0x07, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, + 0x64, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x73, 0x4e, 0x75, 0x6c, 0x6c, 0x10, 0x01, 0x12, + 0x0d, 0x0a, 0x09, 0x49, 0x73, 0x4e, 0x6f, 0x74, 0x4e, 0x75, 0x6c, 0x6c, 0x10, 0x02, 0x22, 0x91, + 0x01, 0x0a, 0x09, 0x55, 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x12, 0x34, 0x0a, 0x02, + 0x6f, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x55, 0x6e, 0x61, + 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x2e, 0x55, 0x6e, 0x61, 0x72, 0x79, 0x4f, 0x70, 0x52, 0x02, + 0x6f, 0x70, 0x12, 0x2d, 0x0a, 0x05, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x05, 0x72, 0x69, 0x67, 0x68, - 0x74, 0x22, 0x36, 0x0a, 0x08, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x4f, 0x70, 0x12, 0x0b, 0x0a, - 0x07, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x10, 0x00, 0x12, 0x0e, 0x0a, 0x0a, 0x4c, 0x6f, - 0x67, 0x69, 0x63, 0x61, 0x6c, 0x41, 0x6e, 0x64, 0x10, 0x01, 0x12, 0x0d, 0x0a, 0x09, 0x4c, 0x6f, - 0x67, 0x69, 0x63, 0x61, 0x6c, 0x4f, 0x72, 0x10, 0x02, 0x22, 0xd0, 0x01, 0x0a, 0x0d, 0x42, 0x69, - 0x6e, 0x61, 0x72, 0x79, 0x41, 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x12, 0x3e, 0x0a, 0x0b, 0x63, - 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, - 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x39, 0x0a, 0x08, 0x61, - 0x72, 0x69, 0x74, 0x68, 0x5f, 0x6f, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1e, 0x2e, + 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x05, 0x63, 0x68, 0x69, 0x6c, + 0x64, 0x22, 0x1f, 0x0a, 0x07, 0x55, 0x6e, 0x61, 0x72, 0x79, 0x4f, 0x70, 0x12, 0x0b, 0x0a, 0x07, + 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x4e, 0x6f, 0x74, + 0x10, 0x01, 0x22, 0xd8, 0x01, 0x0a, 0x0a, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, 0x70, + 0x72, 0x12, 0x36, 0x0a, 0x02, 0x6f, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, - 0x6e, 0x2e, 0x41, 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x54, 0x79, 0x70, 0x65, 0x52, 0x07, 0x61, - 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x12, 0x44, 0x0a, 0x0d, 0x72, 0x69, 0x67, 0x68, 0x74, 0x5f, - 0x6f, 0x70, 0x65, 0x72, 0x61, 0x6e, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, - 0x6e, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0c, - 0x72, 0x69, 0x67, 0x68, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x6e, 0x64, 0x22, 0x9d, 0x01, 0x0a, - 0x0f, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x41, 0x72, 0x69, 0x74, 0x68, 0x45, 0x78, 0x70, 0x72, - 0x12, 0x2b, 0x0a, 0x04, 0x6c, 0x65, 0x66, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, - 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x04, 0x6c, 0x65, 0x66, 0x74, 0x12, 0x2d, 0x0a, - 0x05, 0x72, 0x69, 0x67, 0x68, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, + 0x6e, 0x2e, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x2e, 0x42, 0x69, 0x6e, + 0x61, 0x72, 0x79, 0x4f, 0x70, 0x52, 0x02, 0x6f, 0x70, 0x12, 0x2b, 0x0a, 0x04, 0x6c, 0x65, 0x66, + 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, + 0x52, 0x04, 0x6c, 0x65, 0x66, 0x74, 0x12, 0x2d, 0x0a, 0x05, 0x72, 0x69, 0x67, 0x68, 0x74, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x05, + 0x72, 0x69, 0x67, 0x68, 0x74, 0x22, 0x36, 0x0a, 0x08, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x4f, + 0x70, 0x12, 0x0b, 0x0a, 0x07, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x10, 0x00, 0x12, 0x0e, + 0x0a, 0x0a, 0x4c, 0x6f, 0x67, 0x69, 0x63, 0x61, 0x6c, 0x41, 0x6e, 0x64, 0x10, 0x01, 0x12, 0x0d, + 0x0a, 0x09, 0x4c, 0x6f, 0x67, 0x69, 0x63, 0x61, 0x6c, 0x4f, 0x72, 0x10, 0x02, 0x22, 0xd0, 0x01, + 0x0a, 0x0d, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x41, 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x12, + 0x3e, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, + 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, + 0x39, 0x0a, 0x08, 0x61, 0x72, 0x69, 0x74, 0x68, 0x5f, 0x6f, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0e, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x41, 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x54, 0x79, 0x70, + 0x65, 0x52, 0x07, 0x61, 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x12, 0x44, 0x0a, 0x0d, 0x72, 0x69, + 0x67, 0x68, 0x74, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x6e, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63, 0x56, 0x61, 0x6c, + 0x75, 0x65, 0x52, 0x0c, 0x72, 0x69, 0x67, 0x68, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x6e, 0x64, + 0x22, 0x9d, 0x01, 0x0a, 0x0f, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x41, 0x72, 0x69, 0x74, 0x68, + 0x45, 0x78, 0x70, 0x72, 0x12, 0x2b, 0x0a, 0x04, 0x6c, 0x65, 0x66, 0x74, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x04, 0x6c, 0x65, 0x66, + 0x74, 0x12, 0x2d, 0x0a, 0x05, 0x72, 0x69, 0x67, 0x68, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x05, 0x72, 0x69, 0x67, 0x68, 0x74, + 0x12, 0x2e, 0x0a, 0x02, 0x6f, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, - 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x05, 0x72, 0x69, 0x67, 0x68, 0x74, 0x12, 0x2e, 0x0a, 0x02, - 0x6f, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x41, 0x72, 0x69, - 0x74, 0x68, 0x4f, 0x70, 0x54, 0x79, 0x70, 0x65, 0x52, 0x02, 0x6f, 0x70, 0x22, 0xc5, 0x03, 0x0a, - 0x1a, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x41, 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x45, 0x76, - 0x61, 0x6c, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x3e, 0x0a, 0x0b, 0x63, - 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, - 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x39, 0x0a, 0x08, 0x61, - 0x72, 0x69, 0x74, 0x68, 0x5f, 0x6f, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1e, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, - 0x6e, 0x2e, 0x41, 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x54, 0x79, 0x70, 0x65, 0x52, 0x07, 0x61, - 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x12, 0x44, 0x0a, 0x0d, 0x72, 0x69, 0x67, 0x68, 0x74, 0x5f, - 0x6f, 0x70, 0x65, 0x72, 0x61, 0x6e, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, - 0x6e, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0c, - 0x72, 0x69, 0x67, 0x68, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x6e, 0x64, 0x12, 0x29, 0x0a, 0x02, - 0x6f, 0x70, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x4f, 0x70, 0x54, - 0x79, 0x70, 0x65, 0x52, 0x02, 0x6f, 0x70, 0x12, 0x35, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, - 0x69, 0x63, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x43, - 0x0a, 0x1e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x6e, 0x64, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, - 0x74, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, - 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1b, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x6e, 0x64, 0x54, - 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x4e, - 0x61, 0x6d, 0x65, 0x12, 0x3f, 0x0a, 0x1c, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x5f, 0x74, 0x65, 0x6d, - 0x70, 0x6c, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, - 0x61, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x19, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, - 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x10, 0x0a, 0x0e, 0x41, 0x6c, 0x77, 0x61, 0x79, 0x73, 0x54, 0x72, - 0x75, 0x65, 0x45, 0x78, 0x70, 0x72, 0x22, 0xf9, 0x08, 0x0a, 0x04, 0x45, 0x78, 0x70, 0x72, 0x12, - 0x3a, 0x0a, 0x09, 0x74, 0x65, 0x72, 0x6d, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x54, 0x65, 0x72, 0x6d, 0x45, 0x78, 0x70, 0x72, 0x48, - 0x00, 0x52, 0x08, 0x74, 0x65, 0x72, 0x6d, 0x45, 0x78, 0x70, 0x72, 0x12, 0x3d, 0x0a, 0x0a, 0x75, - 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, - 0x6c, 0x61, 0x6e, 0x2e, 0x55, 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, - 0x09, 0x75, 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x12, 0x40, 0x0a, 0x0b, 0x62, 0x69, - 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, - 0x6c, 0x61, 0x6e, 0x2e, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, - 0x52, 0x0a, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x12, 0x43, 0x0a, 0x0c, - 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x45, 0x78, - 0x70, 0x72, 0x48, 0x00, 0x52, 0x0b, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x45, 0x78, 0x70, - 0x72, 0x12, 0x4d, 0x0a, 0x10, 0x75, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, - 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, - 0x55, 0x6e, 0x61, 0x72, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, - 0x52, 0x0e, 0x75, 0x6e, 0x61, 0x72, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, - 0x12, 0x50, 0x0a, 0x11, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, - 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, - 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, 0x48, - 0x00, 0x52, 0x0f, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, - 0x70, 0x72, 0x12, 0x74, 0x0a, 0x1f, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x61, 0x72, 0x69, - 0x74, 0x68, 0x5f, 0x6f, 0x70, 0x5f, 0x65, 0x76, 0x61, 0x6c, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, - 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6d, 0x69, + 0x2e, 0x41, 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x54, 0x79, 0x70, 0x65, 0x52, 0x02, 0x6f, 0x70, + 0x22, 0xc5, 0x03, 0x0a, 0x1a, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x41, 0x72, 0x69, 0x74, 0x68, + 0x4f, 0x70, 0x45, 0x76, 0x61, 0x6c, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, + 0x3e, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, + 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, + 0x39, 0x0a, 0x08, 0x61, 0x72, 0x69, 0x74, 0x68, 0x5f, 0x6f, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0e, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x41, 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x54, 0x79, 0x70, + 0x65, 0x52, 0x07, 0x61, 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x12, 0x44, 0x0a, 0x0d, 0x72, 0x69, + 0x67, 0x68, 0x74, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x6e, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63, 0x56, 0x61, 0x6c, + 0x75, 0x65, 0x52, 0x0c, 0x72, 0x69, 0x67, 0x68, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x6e, 0x64, + 0x12, 0x29, 0x0a, 0x02, 0x6f, 0x70, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, + 0x2e, 0x4f, 0x70, 0x54, 0x79, 0x70, 0x65, 0x52, 0x02, 0x6f, 0x70, 0x12, 0x35, 0x0a, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x47, + 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x12, 0x43, 0x0a, 0x1e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x6e, 0x64, 0x5f, 0x74, 0x65, + 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x5f, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1b, 0x6f, 0x70, 0x65, 0x72, + 0x61, 0x6e, 0x64, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, + 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3f, 0x0a, 0x1c, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x62, + 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x19, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x56, 0x61, 0x72, 0x69, + 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x10, 0x0a, 0x0e, 0x41, 0x6c, 0x77, 0x61, + 0x79, 0x73, 0x54, 0x72, 0x75, 0x65, 0x45, 0x78, 0x70, 0x72, 0x22, 0xf9, 0x08, 0x0a, 0x04, 0x45, + 0x78, 0x70, 0x72, 0x12, 0x3a, 0x0a, 0x09, 0x74, 0x65, 0x72, 0x6d, 0x5f, 0x65, 0x78, 0x70, 0x72, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x54, 0x65, 0x72, 0x6d, 0x45, + 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x08, 0x74, 0x65, 0x72, 0x6d, 0x45, 0x78, 0x70, 0x72, 0x12, + 0x3d, 0x0a, 0x0a, 0x75, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x55, 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, 0x70, + 0x72, 0x48, 0x00, 0x52, 0x09, 0x75, 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x12, 0x40, + 0x0a, 0x0b, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, + 0x70, 0x72, 0x48, 0x00, 0x52, 0x0a, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, + 0x12, 0x43, 0x0a, 0x0c, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x5f, 0x65, 0x78, 0x70, 0x72, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, + 0x72, 0x65, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x0b, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x72, + 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x4d, 0x0a, 0x10, 0x75, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x72, + 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, + 0x6c, 0x61, 0x6e, 0x2e, 0x55, 0x6e, 0x61, 0x72, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, + 0x70, 0x72, 0x48, 0x00, 0x52, 0x0e, 0x75, 0x6e, 0x61, 0x72, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, + 0x45, 0x78, 0x70, 0x72, 0x12, 0x50, 0x0a, 0x11, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x72, + 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, + 0x6c, 0x61, 0x6e, 0x2e, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, + 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x0f, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x52, 0x61, 0x6e, + 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x74, 0x0a, 0x1f, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, + 0x5f, 0x61, 0x72, 0x69, 0x74, 0x68, 0x5f, 0x6f, 0x70, 0x5f, 0x65, 0x76, 0x61, 0x6c, 0x5f, 0x72, + 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, + 0x6c, 0x61, 0x6e, 0x2e, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x41, 0x72, 0x69, 0x74, 0x68, 0x4f, + 0x70, 0x45, 0x76, 0x61, 0x6c, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, + 0x52, 0x1a, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x41, 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x45, + 0x76, 0x61, 0x6c, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x50, 0x0a, 0x11, + 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x61, 0x72, 0x69, 0x74, 0x68, 0x5f, 0x65, 0x78, 0x70, + 0x72, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x42, 0x69, 0x6e, 0x61, + 0x72, 0x79, 0x41, 0x72, 0x69, 0x74, 0x68, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x0f, 0x62, + 0x69, 0x6e, 0x61, 0x72, 0x79, 0x41, 0x72, 0x69, 0x74, 0x68, 0x45, 0x78, 0x70, 0x72, 0x12, 0x3d, + 0x0a, 0x0a, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x09, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x45, 0x78, 0x70, 0x72, + 0x48, 0x00, 0x52, 0x09, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x40, 0x0a, + 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x0a, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x45, 0x78, 0x70, + 0x72, 0x48, 0x00, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x45, 0x78, 0x70, 0x72, 0x12, + 0x40, 0x0a, 0x0b, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x0b, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x45, + 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x0a, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x45, 0x78, 0x70, + 0x72, 0x12, 0x4d, 0x0a, 0x10, 0x61, 0x6c, 0x77, 0x61, 0x79, 0x73, 0x5f, 0x74, 0x72, 0x75, 0x65, + 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, - 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x41, 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x45, 0x76, 0x61, - 0x6c, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x1a, 0x62, 0x69, - 0x6e, 0x61, 0x72, 0x79, 0x41, 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x45, 0x76, 0x61, 0x6c, 0x52, - 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x50, 0x0a, 0x11, 0x62, 0x69, 0x6e, 0x61, - 0x72, 0x79, 0x5f, 0x61, 0x72, 0x69, 0x74, 0x68, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x08, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x41, 0x72, - 0x69, 0x74, 0x68, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x0f, 0x62, 0x69, 0x6e, 0x61, 0x72, - 0x79, 0x41, 0x72, 0x69, 0x74, 0x68, 0x45, 0x78, 0x70, 0x72, 0x12, 0x3d, 0x0a, 0x0a, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, - 0x61, 0x6e, 0x2e, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x09, - 0x76, 0x61, 0x6c, 0x75, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x40, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, - 0x75, 0x6d, 0x6e, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, - 0x61, 0x6e, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, - 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x45, 0x78, 0x70, 0x72, 0x12, 0x40, 0x0a, 0x0b, 0x65, - 0x78, 0x69, 0x73, 0x74, 0x73, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x45, 0x78, 0x70, 0x72, 0x48, - 0x00, 0x52, 0x0a, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x45, 0x78, 0x70, 0x72, 0x12, 0x4d, 0x0a, - 0x10, 0x61, 0x6c, 0x77, 0x61, 0x79, 0x73, 0x5f, 0x74, 0x72, 0x75, 0x65, 0x5f, 0x65, 0x78, 0x70, - 0x72, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x41, 0x6c, 0x77, 0x61, - 0x79, 0x73, 0x54, 0x72, 0x75, 0x65, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x0e, 0x61, 0x6c, - 0x77, 0x61, 0x79, 0x73, 0x54, 0x72, 0x75, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x53, 0x0a, 0x12, - 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x5f, 0x65, 0x78, - 0x70, 0x72, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x4a, 0x53, 0x4f, - 0x4e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, - 0x10, 0x6a, 0x73, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x45, 0x78, 0x70, - 0x72, 0x12, 0x3a, 0x0a, 0x09, 0x63, 0x61, 0x6c, 0x6c, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x0e, + 0x41, 0x6c, 0x77, 0x61, 0x79, 0x73, 0x54, 0x72, 0x75, 0x65, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, + 0x52, 0x0e, 0x61, 0x6c, 0x77, 0x61, 0x79, 0x73, 0x54, 0x72, 0x75, 0x65, 0x45, 0x78, 0x70, 0x72, + 0x12, 0x53, 0x0a, 0x12, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, + 0x73, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, + 0x2e, 0x4a, 0x53, 0x4f, 0x4e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x45, 0x78, 0x70, + 0x72, 0x48, 0x00, 0x52, 0x10, 0x6a, 0x73, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, + 0x73, 0x45, 0x78, 0x70, 0x72, 0x12, 0x3a, 0x0a, 0x09, 0x63, 0x61, 0x6c, 0x6c, 0x5f, 0x65, 0x78, + 0x70, 0x72, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x61, 0x6c, + 0x6c, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x08, 0x63, 0x61, 0x6c, 0x6c, 0x45, 0x78, 0x70, + 0x72, 0x12, 0x3a, 0x0a, 0x09, 0x6e, 0x75, 0x6c, 0x6c, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x45, 0x78, 0x70, - 0x72, 0x48, 0x00, 0x52, 0x08, 0x63, 0x61, 0x6c, 0x6c, 0x45, 0x78, 0x70, 0x72, 0x12, 0x3a, 0x0a, - 0x09, 0x6e, 0x75, 0x6c, 0x6c, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x4e, 0x75, 0x6c, 0x6c, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, - 0x08, 0x6e, 0x75, 0x6c, 0x6c, 0x45, 0x78, 0x70, 0x72, 0x12, 0x1f, 0x0a, 0x0b, 0x69, 0x73, 0x5f, - 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x18, 0x14, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, - 0x69, 0x73, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x42, 0x06, 0x0a, 0x04, 0x65, 0x78, - 0x70, 0x72, 0x22, 0x86, 0x02, 0x0a, 0x0a, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x41, 0x4e, 0x4e, - 0x53, 0x12, 0x3e, 0x0a, 0x0b, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x5f, 0x74, 0x79, 0x70, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x56, 0x65, 0x63, 0x74, 0x6f, - 0x72, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0a, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x54, 0x79, 0x70, - 0x65, 0x12, 0x19, 0x0a, 0x08, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x64, 0x12, 0x37, 0x0a, 0x0a, - 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, - 0x63, 0x61, 0x74, 0x65, 0x73, 0x12, 0x3b, 0x0a, 0x0a, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x69, - 0x6e, 0x66, 0x6f, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x51, 0x75, - 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x71, 0x75, 0x65, 0x72, 0x79, 0x49, 0x6e, - 0x66, 0x6f, 0x12, 0x27, 0x0a, 0x0f, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x68, 0x6f, 0x6c, 0x64, 0x65, - 0x72, 0x5f, 0x74, 0x61, 0x67, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x70, 0x6c, 0x61, - 0x63, 0x65, 0x68, 0x6f, 0x6c, 0x64, 0x65, 0x72, 0x54, 0x61, 0x67, 0x22, 0x79, 0x0a, 0x0d, 0x51, - 0x75, 0x65, 0x72, 0x79, 0x50, 0x6c, 0x61, 0x6e, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x37, 0x0a, 0x0a, - 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, - 0x63, 0x61, 0x74, 0x65, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x69, 0x73, 0x5f, 0x63, 0x6f, 0x75, 0x6e, - 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x69, 0x73, 0x43, 0x6f, 0x75, 0x6e, 0x74, - 0x12, 0x14, 0x0a, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, - 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x22, 0x9a, 0x02, 0x0a, 0x08, 0x50, 0x6c, 0x61, 0x6e, 0x4e, - 0x6f, 0x64, 0x65, 0x12, 0x40, 0x0a, 0x0b, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x5f, 0x61, 0x6e, - 0x6e, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x56, 0x65, 0x63, - 0x74, 0x6f, 0x72, 0x41, 0x4e, 0x4e, 0x53, 0x48, 0x00, 0x52, 0x0a, 0x76, 0x65, 0x63, 0x74, 0x6f, - 0x72, 0x41, 0x6e, 0x6e, 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, - 0x74, 0x65, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, - 0x70, 0x72, 0x48, 0x00, 0x52, 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, - 0x12, 0x38, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, - 0x6c, 0x61, 0x6e, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x50, 0x6c, 0x61, 0x6e, 0x4e, 0x6f, 0x64, - 0x65, 0x48, 0x00, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x28, 0x0a, 0x10, 0x6f, 0x75, - 0x74, 0x70, 0x75, 0x74, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x03, - 0x20, 0x03, 0x28, 0x03, 0x52, 0x0e, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x46, 0x69, 0x65, 0x6c, - 0x64, 0x49, 0x64, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x64, 0x79, 0x6e, 0x61, 0x6d, 0x69, 0x63, 0x5f, - 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x64, 0x79, - 0x6e, 0x61, 0x6d, 0x69, 0x63, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x42, 0x06, 0x0a, 0x04, 0x6e, - 0x6f, 0x64, 0x65, 0x2a, 0xc9, 0x01, 0x0a, 0x06, 0x4f, 0x70, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, - 0x0a, 0x07, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x10, 0x00, 0x12, 0x0f, 0x0a, 0x0b, 0x47, - 0x72, 0x65, 0x61, 0x74, 0x65, 0x72, 0x54, 0x68, 0x61, 0x6e, 0x10, 0x01, 0x12, 0x10, 0x0a, 0x0c, - 0x47, 0x72, 0x65, 0x61, 0x74, 0x65, 0x72, 0x45, 0x71, 0x75, 0x61, 0x6c, 0x10, 0x02, 0x12, 0x0c, - 0x0a, 0x08, 0x4c, 0x65, 0x73, 0x73, 0x54, 0x68, 0x61, 0x6e, 0x10, 0x03, 0x12, 0x0d, 0x0a, 0x09, - 0x4c, 0x65, 0x73, 0x73, 0x45, 0x71, 0x75, 0x61, 0x6c, 0x10, 0x04, 0x12, 0x09, 0x0a, 0x05, 0x45, - 0x71, 0x75, 0x61, 0x6c, 0x10, 0x05, 0x12, 0x0c, 0x0a, 0x08, 0x4e, 0x6f, 0x74, 0x45, 0x71, 0x75, - 0x61, 0x6c, 0x10, 0x06, 0x12, 0x0f, 0x0a, 0x0b, 0x50, 0x72, 0x65, 0x66, 0x69, 0x78, 0x4d, 0x61, - 0x74, 0x63, 0x68, 0x10, 0x07, 0x12, 0x10, 0x0a, 0x0c, 0x50, 0x6f, 0x73, 0x74, 0x66, 0x69, 0x78, - 0x4d, 0x61, 0x74, 0x63, 0x68, 0x10, 0x08, 0x12, 0x09, 0x0a, 0x05, 0x4d, 0x61, 0x74, 0x63, 0x68, - 0x10, 0x09, 0x12, 0x09, 0x0a, 0x05, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x10, 0x0a, 0x12, 0x06, 0x0a, - 0x02, 0x49, 0x6e, 0x10, 0x0b, 0x12, 0x09, 0x0a, 0x05, 0x4e, 0x6f, 0x74, 0x49, 0x6e, 0x10, 0x0c, - 0x12, 0x0d, 0x0a, 0x09, 0x54, 0x65, 0x78, 0x74, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x10, 0x0d, 0x2a, - 0x58, 0x0a, 0x0b, 0x41, 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, - 0x0a, 0x07, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x41, - 0x64, 0x64, 0x10, 0x01, 0x12, 0x07, 0x0a, 0x03, 0x53, 0x75, 0x62, 0x10, 0x02, 0x12, 0x07, 0x0a, - 0x03, 0x4d, 0x75, 0x6c, 0x10, 0x03, 0x12, 0x07, 0x0a, 0x03, 0x44, 0x69, 0x76, 0x10, 0x04, 0x12, - 0x07, 0x0a, 0x03, 0x4d, 0x6f, 0x64, 0x10, 0x05, 0x12, 0x0f, 0x0a, 0x0b, 0x41, 0x72, 0x72, 0x61, - 0x79, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x10, 0x06, 0x2a, 0x6d, 0x0a, 0x0a, 0x56, 0x65, 0x63, - 0x74, 0x6f, 0x72, 0x54, 0x79, 0x70, 0x65, 0x12, 0x10, 0x0a, 0x0c, 0x42, 0x69, 0x6e, 0x61, 0x72, - 0x79, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, 0x00, 0x12, 0x0f, 0x0a, 0x0b, 0x46, 0x6c, 0x6f, - 0x61, 0x74, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, 0x01, 0x12, 0x11, 0x0a, 0x0d, 0x46, 0x6c, - 0x6f, 0x61, 0x74, 0x31, 0x36, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, 0x02, 0x12, 0x12, 0x0a, - 0x0e, 0x42, 0x46, 0x6c, 0x6f, 0x61, 0x74, 0x31, 0x36, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, - 0x03, 0x12, 0x15, 0x0a, 0x11, 0x53, 0x70, 0x61, 0x72, 0x73, 0x65, 0x46, 0x6c, 0x6f, 0x61, 0x74, - 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, 0x04, 0x42, 0x2e, 0x5a, 0x2c, 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, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2f, 0x70, 0x6c, 0x61, 0x6e, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x4e, 0x75, 0x6c, 0x6c, 0x45, 0x78, 0x70, + 0x72, 0x48, 0x00, 0x52, 0x08, 0x6e, 0x75, 0x6c, 0x6c, 0x45, 0x78, 0x70, 0x72, 0x12, 0x1f, 0x0a, + 0x0b, 0x69, 0x73, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x18, 0x14, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0a, 0x69, 0x73, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x42, 0x06, + 0x0a, 0x04, 0x65, 0x78, 0x70, 0x72, 0x22, 0x86, 0x02, 0x0a, 0x0a, 0x56, 0x65, 0x63, 0x74, 0x6f, + 0x72, 0x41, 0x4e, 0x4e, 0x53, 0x12, 0x3e, 0x0a, 0x0b, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x5f, + 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x56, + 0x65, 0x63, 0x74, 0x6f, 0x72, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0a, 0x76, 0x65, 0x63, 0x74, 0x6f, + 0x72, 0x54, 0x79, 0x70, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x69, + 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x64, + 0x12, 0x37, 0x0a, 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x0a, 0x70, + 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x12, 0x3b, 0x0a, 0x0a, 0x71, 0x75, 0x65, + 0x72, 0x79, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, + 0x6e, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x71, 0x75, 0x65, + 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x27, 0x0a, 0x0f, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x68, + 0x6f, 0x6c, 0x64, 0x65, 0x72, 0x5f, 0x74, 0x61, 0x67, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0e, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x68, 0x6f, 0x6c, 0x64, 0x65, 0x72, 0x54, 0x61, 0x67, 0x22, + 0x79, 0x0a, 0x0d, 0x51, 0x75, 0x65, 0x72, 0x79, 0x50, 0x6c, 0x61, 0x6e, 0x4e, 0x6f, 0x64, 0x65, + 0x12, 0x37, 0x0a, 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x0a, 0x70, + 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x69, 0x73, 0x5f, + 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x69, 0x73, 0x43, + 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x22, 0x9a, 0x02, 0x0a, 0x08, 0x50, + 0x6c, 0x61, 0x6e, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x40, 0x0a, 0x0b, 0x76, 0x65, 0x63, 0x74, 0x6f, + 0x72, 0x5f, 0x61, 0x6e, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, + 0x2e, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x41, 0x4e, 0x4e, 0x53, 0x48, 0x00, 0x52, 0x0a, 0x76, + 0x65, 0x63, 0x74, 0x6f, 0x72, 0x41, 0x6e, 0x6e, 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x70, 0x72, 0x65, + 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, + 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, + 0x61, 0x74, 0x65, 0x73, 0x12, 0x38, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x50, 0x6c, 0x61, + 0x6e, 0x4e, 0x6f, 0x64, 0x65, 0x48, 0x00, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x28, + 0x0a, 0x10, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x69, + 0x64, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0e, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, + 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x64, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x64, 0x79, 0x6e, 0x61, + 0x6d, 0x69, 0x63, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, + 0x52, 0x0d, 0x64, 0x79, 0x6e, 0x61, 0x6d, 0x69, 0x63, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x42, + 0x06, 0x0a, 0x04, 0x6e, 0x6f, 0x64, 0x65, 0x2a, 0xda, 0x01, 0x0a, 0x06, 0x4f, 0x70, 0x54, 0x79, + 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x10, 0x00, 0x12, + 0x0f, 0x0a, 0x0b, 0x47, 0x72, 0x65, 0x61, 0x74, 0x65, 0x72, 0x54, 0x68, 0x61, 0x6e, 0x10, 0x01, + 0x12, 0x10, 0x0a, 0x0c, 0x47, 0x72, 0x65, 0x61, 0x74, 0x65, 0x72, 0x45, 0x71, 0x75, 0x61, 0x6c, + 0x10, 0x02, 0x12, 0x0c, 0x0a, 0x08, 0x4c, 0x65, 0x73, 0x73, 0x54, 0x68, 0x61, 0x6e, 0x10, 0x03, + 0x12, 0x0d, 0x0a, 0x09, 0x4c, 0x65, 0x73, 0x73, 0x45, 0x71, 0x75, 0x61, 0x6c, 0x10, 0x04, 0x12, + 0x09, 0x0a, 0x05, 0x45, 0x71, 0x75, 0x61, 0x6c, 0x10, 0x05, 0x12, 0x0c, 0x0a, 0x08, 0x4e, 0x6f, + 0x74, 0x45, 0x71, 0x75, 0x61, 0x6c, 0x10, 0x06, 0x12, 0x0f, 0x0a, 0x0b, 0x50, 0x72, 0x65, 0x66, + 0x69, 0x78, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x10, 0x07, 0x12, 0x10, 0x0a, 0x0c, 0x50, 0x6f, 0x73, + 0x74, 0x66, 0x69, 0x78, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x10, 0x08, 0x12, 0x09, 0x0a, 0x05, 0x4d, + 0x61, 0x74, 0x63, 0x68, 0x10, 0x09, 0x12, 0x09, 0x0a, 0x05, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x10, + 0x0a, 0x12, 0x06, 0x0a, 0x02, 0x49, 0x6e, 0x10, 0x0b, 0x12, 0x09, 0x0a, 0x05, 0x4e, 0x6f, 0x74, + 0x49, 0x6e, 0x10, 0x0c, 0x12, 0x0d, 0x0a, 0x09, 0x54, 0x65, 0x78, 0x74, 0x4d, 0x61, 0x74, 0x63, + 0x68, 0x10, 0x0d, 0x12, 0x0f, 0x0a, 0x0b, 0x50, 0x68, 0x72, 0x61, 0x73, 0x65, 0x4d, 0x61, 0x74, + 0x63, 0x68, 0x10, 0x0e, 0x2a, 0x58, 0x0a, 0x0b, 0x41, 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x54, + 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, + 0x12, 0x07, 0x0a, 0x03, 0x41, 0x64, 0x64, 0x10, 0x01, 0x12, 0x07, 0x0a, 0x03, 0x53, 0x75, 0x62, + 0x10, 0x02, 0x12, 0x07, 0x0a, 0x03, 0x4d, 0x75, 0x6c, 0x10, 0x03, 0x12, 0x07, 0x0a, 0x03, 0x44, + 0x69, 0x76, 0x10, 0x04, 0x12, 0x07, 0x0a, 0x03, 0x4d, 0x6f, 0x64, 0x10, 0x05, 0x12, 0x0f, 0x0a, + 0x0b, 0x41, 0x72, 0x72, 0x61, 0x79, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x10, 0x06, 0x2a, 0x6d, + 0x0a, 0x0a, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x54, 0x79, 0x70, 0x65, 0x12, 0x10, 0x0a, 0x0c, + 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, 0x00, 0x12, 0x0f, + 0x0a, 0x0b, 0x46, 0x6c, 0x6f, 0x61, 0x74, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, 0x01, 0x12, + 0x11, 0x0a, 0x0d, 0x46, 0x6c, 0x6f, 0x61, 0x74, 0x31, 0x36, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, + 0x10, 0x02, 0x12, 0x12, 0x0a, 0x0e, 0x42, 0x46, 0x6c, 0x6f, 0x61, 0x74, 0x31, 0x36, 0x56, 0x65, + 0x63, 0x74, 0x6f, 0x72, 0x10, 0x03, 0x12, 0x15, 0x0a, 0x11, 0x53, 0x70, 0x61, 0x72, 0x73, 0x65, + 0x46, 0x6c, 0x6f, 0x61, 0x74, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, 0x04, 0x42, 0x2e, 0x5a, + 0x2c, 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, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x6c, 0x61, 0x6e, 0x70, 0x62, 0x62, 0x06, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -2956,63 +2973,64 @@ var file_plan_proto_depIdxs = []int32{ 11, // 9: milvus.proto.plan.UnaryRangeExpr.column_info:type_name -> milvus.proto.plan.ColumnInfo 0, // 10: milvus.proto.plan.UnaryRangeExpr.op:type_name -> milvus.proto.plan.OpType 7, // 11: milvus.proto.plan.UnaryRangeExpr.value:type_name -> milvus.proto.plan.GenericValue - 11, // 12: milvus.proto.plan.BinaryRangeExpr.column_info:type_name -> milvus.proto.plan.ColumnInfo - 7, // 13: milvus.proto.plan.BinaryRangeExpr.lower_value:type_name -> milvus.proto.plan.GenericValue - 7, // 14: milvus.proto.plan.BinaryRangeExpr.upper_value:type_name -> milvus.proto.plan.GenericValue - 28, // 15: milvus.proto.plan.CallExpr.function_parameters:type_name -> milvus.proto.plan.Expr - 11, // 16: milvus.proto.plan.CompareExpr.left_column_info:type_name -> milvus.proto.plan.ColumnInfo - 11, // 17: milvus.proto.plan.CompareExpr.right_column_info:type_name -> milvus.proto.plan.ColumnInfo - 0, // 18: milvus.proto.plan.CompareExpr.op:type_name -> milvus.proto.plan.OpType - 11, // 19: milvus.proto.plan.TermExpr.column_info:type_name -> milvus.proto.plan.ColumnInfo - 7, // 20: milvus.proto.plan.TermExpr.values:type_name -> milvus.proto.plan.GenericValue - 11, // 21: milvus.proto.plan.JSONContainsExpr.column_info:type_name -> milvus.proto.plan.ColumnInfo - 7, // 22: milvus.proto.plan.JSONContainsExpr.elements:type_name -> milvus.proto.plan.GenericValue - 3, // 23: milvus.proto.plan.JSONContainsExpr.op:type_name -> milvus.proto.plan.JSONContainsExpr.JSONOp - 11, // 24: milvus.proto.plan.NullExpr.column_info:type_name -> milvus.proto.plan.ColumnInfo - 4, // 25: milvus.proto.plan.NullExpr.op:type_name -> milvus.proto.plan.NullExpr.NullOp - 5, // 26: milvus.proto.plan.UnaryExpr.op:type_name -> milvus.proto.plan.UnaryExpr.UnaryOp - 28, // 27: milvus.proto.plan.UnaryExpr.child:type_name -> milvus.proto.plan.Expr - 6, // 28: milvus.proto.plan.BinaryExpr.op:type_name -> milvus.proto.plan.BinaryExpr.BinaryOp - 28, // 29: milvus.proto.plan.BinaryExpr.left:type_name -> milvus.proto.plan.Expr - 28, // 30: milvus.proto.plan.BinaryExpr.right:type_name -> milvus.proto.plan.Expr - 11, // 31: milvus.proto.plan.BinaryArithOp.column_info:type_name -> milvus.proto.plan.ColumnInfo - 1, // 32: milvus.proto.plan.BinaryArithOp.arith_op:type_name -> milvus.proto.plan.ArithOpType - 7, // 33: milvus.proto.plan.BinaryArithOp.right_operand:type_name -> milvus.proto.plan.GenericValue - 28, // 34: milvus.proto.plan.BinaryArithExpr.left:type_name -> milvus.proto.plan.Expr - 28, // 35: milvus.proto.plan.BinaryArithExpr.right:type_name -> milvus.proto.plan.Expr - 1, // 36: milvus.proto.plan.BinaryArithExpr.op:type_name -> milvus.proto.plan.ArithOpType - 11, // 37: milvus.proto.plan.BinaryArithOpEvalRangeExpr.column_info:type_name -> milvus.proto.plan.ColumnInfo - 1, // 38: milvus.proto.plan.BinaryArithOpEvalRangeExpr.arith_op:type_name -> milvus.proto.plan.ArithOpType - 7, // 39: milvus.proto.plan.BinaryArithOpEvalRangeExpr.right_operand:type_name -> milvus.proto.plan.GenericValue - 0, // 40: milvus.proto.plan.BinaryArithOpEvalRangeExpr.op:type_name -> milvus.proto.plan.OpType - 7, // 41: milvus.proto.plan.BinaryArithOpEvalRangeExpr.value:type_name -> milvus.proto.plan.GenericValue - 19, // 42: milvus.proto.plan.Expr.term_expr:type_name -> milvus.proto.plan.TermExpr - 22, // 43: milvus.proto.plan.Expr.unary_expr:type_name -> milvus.proto.plan.UnaryExpr - 23, // 44: milvus.proto.plan.Expr.binary_expr:type_name -> milvus.proto.plan.BinaryExpr - 18, // 45: milvus.proto.plan.Expr.compare_expr:type_name -> milvus.proto.plan.CompareExpr - 15, // 46: milvus.proto.plan.Expr.unary_range_expr:type_name -> milvus.proto.plan.UnaryRangeExpr - 16, // 47: milvus.proto.plan.Expr.binary_range_expr:type_name -> milvus.proto.plan.BinaryRangeExpr - 26, // 48: milvus.proto.plan.Expr.binary_arith_op_eval_range_expr:type_name -> milvus.proto.plan.BinaryArithOpEvalRangeExpr - 25, // 49: milvus.proto.plan.Expr.binary_arith_expr:type_name -> milvus.proto.plan.BinaryArithExpr - 14, // 50: milvus.proto.plan.Expr.value_expr:type_name -> milvus.proto.plan.ValueExpr - 12, // 51: milvus.proto.plan.Expr.column_expr:type_name -> milvus.proto.plan.ColumnExpr - 13, // 52: milvus.proto.plan.Expr.exists_expr:type_name -> milvus.proto.plan.ExistsExpr - 27, // 53: milvus.proto.plan.Expr.always_true_expr:type_name -> milvus.proto.plan.AlwaysTrueExpr - 20, // 54: milvus.proto.plan.Expr.json_contains_expr:type_name -> milvus.proto.plan.JSONContainsExpr - 17, // 55: milvus.proto.plan.Expr.call_expr:type_name -> milvus.proto.plan.CallExpr - 21, // 56: milvus.proto.plan.Expr.null_expr:type_name -> milvus.proto.plan.NullExpr - 2, // 57: milvus.proto.plan.VectorANNS.vector_type:type_name -> milvus.proto.plan.VectorType - 28, // 58: milvus.proto.plan.VectorANNS.predicates:type_name -> milvus.proto.plan.Expr - 10, // 59: milvus.proto.plan.VectorANNS.query_info:type_name -> milvus.proto.plan.QueryInfo - 28, // 60: milvus.proto.plan.QueryPlanNode.predicates:type_name -> milvus.proto.plan.Expr - 29, // 61: milvus.proto.plan.PlanNode.vector_anns:type_name -> milvus.proto.plan.VectorANNS - 28, // 62: milvus.proto.plan.PlanNode.predicates:type_name -> milvus.proto.plan.Expr - 30, // 63: milvus.proto.plan.PlanNode.query:type_name -> milvus.proto.plan.QueryPlanNode - 64, // [64:64] is the sub-list for method output_type - 64, // [64:64] is the sub-list for method input_type - 64, // [64:64] is the sub-list for extension type_name - 64, // [64:64] is the sub-list for extension extendee - 0, // [0:64] is the sub-list for field type_name + 7, // 12: milvus.proto.plan.UnaryRangeExpr.extra_values:type_name -> milvus.proto.plan.GenericValue + 11, // 13: milvus.proto.plan.BinaryRangeExpr.column_info:type_name -> milvus.proto.plan.ColumnInfo + 7, // 14: milvus.proto.plan.BinaryRangeExpr.lower_value:type_name -> milvus.proto.plan.GenericValue + 7, // 15: milvus.proto.plan.BinaryRangeExpr.upper_value:type_name -> milvus.proto.plan.GenericValue + 28, // 16: milvus.proto.plan.CallExpr.function_parameters:type_name -> milvus.proto.plan.Expr + 11, // 17: milvus.proto.plan.CompareExpr.left_column_info:type_name -> milvus.proto.plan.ColumnInfo + 11, // 18: milvus.proto.plan.CompareExpr.right_column_info:type_name -> milvus.proto.plan.ColumnInfo + 0, // 19: milvus.proto.plan.CompareExpr.op:type_name -> milvus.proto.plan.OpType + 11, // 20: milvus.proto.plan.TermExpr.column_info:type_name -> milvus.proto.plan.ColumnInfo + 7, // 21: milvus.proto.plan.TermExpr.values:type_name -> milvus.proto.plan.GenericValue + 11, // 22: milvus.proto.plan.JSONContainsExpr.column_info:type_name -> milvus.proto.plan.ColumnInfo + 7, // 23: milvus.proto.plan.JSONContainsExpr.elements:type_name -> milvus.proto.plan.GenericValue + 3, // 24: milvus.proto.plan.JSONContainsExpr.op:type_name -> milvus.proto.plan.JSONContainsExpr.JSONOp + 11, // 25: milvus.proto.plan.NullExpr.column_info:type_name -> milvus.proto.plan.ColumnInfo + 4, // 26: milvus.proto.plan.NullExpr.op:type_name -> milvus.proto.plan.NullExpr.NullOp + 5, // 27: milvus.proto.plan.UnaryExpr.op:type_name -> milvus.proto.plan.UnaryExpr.UnaryOp + 28, // 28: milvus.proto.plan.UnaryExpr.child:type_name -> milvus.proto.plan.Expr + 6, // 29: milvus.proto.plan.BinaryExpr.op:type_name -> milvus.proto.plan.BinaryExpr.BinaryOp + 28, // 30: milvus.proto.plan.BinaryExpr.left:type_name -> milvus.proto.plan.Expr + 28, // 31: milvus.proto.plan.BinaryExpr.right:type_name -> milvus.proto.plan.Expr + 11, // 32: milvus.proto.plan.BinaryArithOp.column_info:type_name -> milvus.proto.plan.ColumnInfo + 1, // 33: milvus.proto.plan.BinaryArithOp.arith_op:type_name -> milvus.proto.plan.ArithOpType + 7, // 34: milvus.proto.plan.BinaryArithOp.right_operand:type_name -> milvus.proto.plan.GenericValue + 28, // 35: milvus.proto.plan.BinaryArithExpr.left:type_name -> milvus.proto.plan.Expr + 28, // 36: milvus.proto.plan.BinaryArithExpr.right:type_name -> milvus.proto.plan.Expr + 1, // 37: milvus.proto.plan.BinaryArithExpr.op:type_name -> milvus.proto.plan.ArithOpType + 11, // 38: milvus.proto.plan.BinaryArithOpEvalRangeExpr.column_info:type_name -> milvus.proto.plan.ColumnInfo + 1, // 39: milvus.proto.plan.BinaryArithOpEvalRangeExpr.arith_op:type_name -> milvus.proto.plan.ArithOpType + 7, // 40: milvus.proto.plan.BinaryArithOpEvalRangeExpr.right_operand:type_name -> milvus.proto.plan.GenericValue + 0, // 41: milvus.proto.plan.BinaryArithOpEvalRangeExpr.op:type_name -> milvus.proto.plan.OpType + 7, // 42: milvus.proto.plan.BinaryArithOpEvalRangeExpr.value:type_name -> milvus.proto.plan.GenericValue + 19, // 43: milvus.proto.plan.Expr.term_expr:type_name -> milvus.proto.plan.TermExpr + 22, // 44: milvus.proto.plan.Expr.unary_expr:type_name -> milvus.proto.plan.UnaryExpr + 23, // 45: milvus.proto.plan.Expr.binary_expr:type_name -> milvus.proto.plan.BinaryExpr + 18, // 46: milvus.proto.plan.Expr.compare_expr:type_name -> milvus.proto.plan.CompareExpr + 15, // 47: milvus.proto.plan.Expr.unary_range_expr:type_name -> milvus.proto.plan.UnaryRangeExpr + 16, // 48: milvus.proto.plan.Expr.binary_range_expr:type_name -> milvus.proto.plan.BinaryRangeExpr + 26, // 49: milvus.proto.plan.Expr.binary_arith_op_eval_range_expr:type_name -> milvus.proto.plan.BinaryArithOpEvalRangeExpr + 25, // 50: milvus.proto.plan.Expr.binary_arith_expr:type_name -> milvus.proto.plan.BinaryArithExpr + 14, // 51: milvus.proto.plan.Expr.value_expr:type_name -> milvus.proto.plan.ValueExpr + 12, // 52: milvus.proto.plan.Expr.column_expr:type_name -> milvus.proto.plan.ColumnExpr + 13, // 53: milvus.proto.plan.Expr.exists_expr:type_name -> milvus.proto.plan.ExistsExpr + 27, // 54: milvus.proto.plan.Expr.always_true_expr:type_name -> milvus.proto.plan.AlwaysTrueExpr + 20, // 55: milvus.proto.plan.Expr.json_contains_expr:type_name -> milvus.proto.plan.JSONContainsExpr + 17, // 56: milvus.proto.plan.Expr.call_expr:type_name -> milvus.proto.plan.CallExpr + 21, // 57: milvus.proto.plan.Expr.null_expr:type_name -> milvus.proto.plan.NullExpr + 2, // 58: milvus.proto.plan.VectorANNS.vector_type:type_name -> milvus.proto.plan.VectorType + 28, // 59: milvus.proto.plan.VectorANNS.predicates:type_name -> milvus.proto.plan.Expr + 10, // 60: milvus.proto.plan.VectorANNS.query_info:type_name -> milvus.proto.plan.QueryInfo + 28, // 61: milvus.proto.plan.QueryPlanNode.predicates:type_name -> milvus.proto.plan.Expr + 29, // 62: milvus.proto.plan.PlanNode.vector_anns:type_name -> milvus.proto.plan.VectorANNS + 28, // 63: milvus.proto.plan.PlanNode.predicates:type_name -> milvus.proto.plan.Expr + 30, // 64: milvus.proto.plan.PlanNode.query:type_name -> milvus.proto.plan.QueryPlanNode + 65, // [65:65] is the sub-list for method output_type + 65, // [65:65] is the sub-list for method input_type + 65, // [65:65] is the sub-list for extension type_name + 65, // [65:65] is the sub-list for extension extendee + 0, // [0:65] is the sub-list for field type_name } func init() { file_plan_proto_init() } From 2a02bbe3ee4cebec34fc61207cca285b37e62abe Mon Sep 17 00:00:00 2001 From: Cai Yudong Date: Mon, 13 Jan 2025 09:58:57 +0800 Subject: [PATCH 18/34] enhance: Use template to remove unittest duplication (#39144) Issue: #38666 Signed-off-by: Cai Yudong --- internal/core/src/common/VectorTrait.h | 76 +- internal/core/unittest/test_c_api.cpp | 1152 +++-------------- .../core/unittest/test_c_stream_reduce.cpp | 2 +- internal/core/unittest/test_float16.cpp | 12 +- internal/core/unittest/test_query.cpp | 2 +- internal/core/unittest/test_utils/DataGen.h | 167 +-- .../unittest/test_utils/c_api_test_utils.h | 31 +- 7 files changed, 291 insertions(+), 1151 deletions(-) diff --git a/internal/core/src/common/VectorTrait.h b/internal/core/src/common/VectorTrait.h index e021558bbde2d..19c112737a670 100644 --- a/internal/core/src/common/VectorTrait.h +++ b/internal/core/src/common/VectorTrait.h @@ -15,43 +15,107 @@ // limitations under the License. #pragma once -#include "Types.h" + #include #include + #include "Array.h" +#include "Types.h" +#include "common/type_c.h" +#include "pb/common.pb.h" +#include "pb/plan.pb.h" +#include "pb/schema.pb.h" namespace milvus { +#define GET_ELEM_TYPE_FOR_VECTOR_TRAIT \ + using elem_type = std::conditional_t< \ + std::is_same_v, \ + BinaryVector::embedded_type, \ + std::conditional_t< \ + std::is_same_v, \ + Float16Vector::embedded_type, \ + std::conditional_t< \ + std::is_same_v, \ + BFloat16Vector::embedded_type, \ + FloatVector::embedded_type>>>; + +#define GET_SCHEMA_DATA_TYPE_FOR_VECTOR_TRAIT \ + auto schema_data_type = \ + std::is_same_v \ + ? FloatVector::schema_data_type \ + : std::is_same_v \ + ? Float16Vector::schema_data_type \ + : std::is_same_v \ + ? BFloat16Vector::schema_data_type \ + : BinaryVector::schema_data_type; + class VectorTrait {}; class FloatVector : public VectorTrait { public: using embedded_type = float; - static constexpr auto metric_type = DataType::VECTOR_FLOAT; + static constexpr int32_t dim_factor = 1; + static constexpr auto data_type = DataType::VECTOR_FLOAT; + static constexpr auto c_data_type = CDataType::FloatVector; + static constexpr auto schema_data_type = + proto::schema::DataType::FloatVector; + static constexpr auto vector_type = proto::plan::VectorType::FloatVector; + static constexpr auto placeholder_type = + proto::common::PlaceholderType::FloatVector; }; class BinaryVector : public VectorTrait { public: using embedded_type = uint8_t; - static constexpr auto metric_type = DataType::VECTOR_BINARY; + static constexpr int32_t dim_factor = 8; + static constexpr auto data_type = DataType::VECTOR_BINARY; + static constexpr auto c_data_type = CDataType::BinaryVector; + static constexpr auto schema_data_type = + proto::schema::DataType::BinaryVector; + static constexpr auto vector_type = proto::plan::VectorType::BinaryVector; + static constexpr auto placeholder_type = + proto::common::PlaceholderType::BinaryVector; }; class Float16Vector : public VectorTrait { public: using embedded_type = float16; - static constexpr auto metric_type = DataType::VECTOR_FLOAT16; + static constexpr int32_t dim_factor = 1; + static constexpr auto data_type = DataType::VECTOR_FLOAT16; + static constexpr auto c_data_type = CDataType::Float16Vector; + static constexpr auto schema_data_type = + proto::schema::DataType::Float16Vector; + static constexpr auto vector_type = proto::plan::VectorType::Float16Vector; + static constexpr auto placeholder_type = + proto::common::PlaceholderType::Float16Vector; }; class BFloat16Vector : public VectorTrait { public: using embedded_type = bfloat16; - static constexpr auto metric_type = DataType::VECTOR_BFLOAT16; + static constexpr int32_t dim_factor = 1; + static constexpr auto data_type = DataType::VECTOR_BFLOAT16; + static constexpr auto c_data_type = CDataType::BFloat16Vector; + static constexpr auto schema_data_type = + proto::schema::DataType::BFloat16Vector; + static constexpr auto vector_type = proto::plan::VectorType::BFloat16Vector; + static constexpr auto placeholder_type = + proto::common::PlaceholderType::BFloat16Vector; }; class SparseFloatVector : public VectorTrait { public: using embedded_type = float; - static constexpr auto metric_type = DataType::VECTOR_SPARSE_FLOAT; + static constexpr int32_t dim_factor = 1; + static constexpr auto data_type = DataType::VECTOR_SPARSE_FLOAT; + static constexpr auto c_data_type = CDataType::SparseFloatVector; + static constexpr auto schema_data_type = + proto::schema::DataType::SparseFloatVector; + static constexpr auto vector_type = + proto::plan::VectorType::SparseFloatVector; + static constexpr auto placeholder_type = + proto::common::PlaceholderType::SparseFloatVector; }; template diff --git a/internal/core/unittest/test_c_api.cpp b/internal/core/unittest/test_c_api.cpp index 61a2d86cde29e..6f7e4e7f2c75f 100644 --- a/internal/core/unittest/test_c_api.cpp +++ b/internal/core/unittest/test_c_api.cpp @@ -26,6 +26,7 @@ #include "common/LoadInfo.h" #include "common/Types.h" #include "common/type_c.h" +#include "common/VectorTrait.h" #include "index/IndexFactory.h" #include "knowhere/comp/index_param.h" #include "pb/plan.pb.h" @@ -117,58 +118,6 @@ CRetrieveByOffsets(CSegmentInterface c_segment, return status; } -const char* -get_float16_schema_config() { - static std::string conf = R"(name: "float16-collection" - fields: < - fieldID: 100 - name: "fakevec" - data_type: Float16Vector - type_params: < - key: "dim" - value: "16" - > - index_params: < - key: "metric_type" - value: "L2" - > - > - fields: < - fieldID: 101 - name: "age" - data_type: Int64 - is_primary_key: true - >)"; - static std::string fake_conf = ""; - return conf.c_str(); -} - -const char* -get_bfloat16_schema_config() { - static std::string conf = R"(name: "bfloat16-collection" - fields: < - fieldID: 100 - name: "fakevec" - data_type: BFloat16Vector - type_params: < - key: "dim" - value: "16" - > - index_params: < - key: "metric_type" - value: "L2" - > - > - fields: < - fieldID: 101 - name: "age" - data_type: Int64 - is_primary_key: true - >)"; - static std::string fake_conf = ""; - return conf.c_str(); -} - const char* get_default_index_meta() { static std::string conf = R"(maxIndexRowCount: 1000 @@ -221,75 +170,19 @@ generate_data(int N) { return std::make_tuple(raw_data, timestamps, uids); } +template std::string -generate_query_data_float16(int nq) { - namespace ser = milvus::proto::common; - std::default_random_engine e(67); - int dim = DIM; - std::normal_distribution dis(0.0, 1.0); - ser::PlaceholderGroup raw_group; - auto value = raw_group.add_placeholders(); - value->set_tag("$0"); - value->set_type(ser::PlaceholderType::Float16Vector); - for (int i = 0; i < nq; ++i) { - std::vector vec; - for (int d = 0; d < dim; ++d) { - vec.push_back(float16(dis(e))); - } - value->add_values(vec.data(), vec.size() * sizeof(float16)); - } - auto blob = raw_group.SerializeAsString(); - return blob; -} - -std::string -generate_query_data_bfloat16(int nq) { - namespace ser = milvus::proto::common; - std::default_random_engine e(67); - int dim = DIM; - std::normal_distribution dis(0.0, 1.0); - ser::PlaceholderGroup raw_group; - auto value = raw_group.add_placeholders(); - value->set_tag("$0"); - value->set_type(ser::PlaceholderType::BFloat16Vector); - for (int i = 0; i < nq; ++i) { - std::vector vec; - for (int d = 0; d < dim; ++d) { - vec.push_back(bfloat16(dis(e))); - } - value->add_values(vec.data(), vec.size() * sizeof(bfloat16)); - } - auto blob = raw_group.SerializeAsString(); - return blob; -} -// create Enum for schema::DataType::BinaryVector,schema::DataType::FloatVector -enum VectorType { - BinaryVector = 0, - FloatVector = 1, - Float16Vector = 2, - BFloat16Vector = 3, -}; - -std::string -generate_collection_schema(std::string metric_type, - int dim, - VectorType vector_type) { +generate_collection_schema(std::string metric_type, int dim) { namespace schema = milvus::proto::schema; + GET_SCHEMA_DATA_TYPE_FOR_VECTOR_TRAIT; + 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); - if (vector_type == VectorType::BinaryVector) { - vec_field_schema->set_data_type(schema::DataType::BinaryVector); - } else if (vector_type == VectorType::Float16Vector) { - vec_field_schema->set_data_type(schema::DataType::Float16Vector); - } else if (vector_type == VectorType::BFloat16Vector) { - vec_field_schema->set_data_type(schema::DataType::BFloat16Vector); - } else { - vec_field_schema->set_data_type(schema::DataType::FloatVector); - } + vec_field_schema->set_data_type(schema_data_type); auto metric_type_param = vec_field_schema->add_index_params(); metric_type_param->set_key("metric_type"); metric_type_param->set_value(metric_type); @@ -315,23 +208,6 @@ generate_collection_schema(std::string metric_type, return schema_string; } -// VecIndexPtr -// generate_index( -// void* raw_data, knowhere::Config conf, int64_t dim, int64_t topK, int64_t N, knowhere::IndexType index_type) { -// auto indexing = knowhere::VecIndexFactory::GetInstance().CreateVecIndex(index_type); -// -// auto database = knowhere::GenDataset(N, dim, raw_data); -// indexing->Train(database, conf); -// indexing->AddWithoutIds(database, conf); -// EXPECT_EQ(indexing->Count(), N); -// EXPECT_EQ(indexing->Dim(), dim); -// -// EXPECT_EQ(indexing->Count(), N); -// EXPECT_EQ(indexing->Dim(), dim); -// return indexing; -//} -//} // namespace - IndexBasePtr generate_index(void* raw_data, DataType field_type, @@ -359,7 +235,7 @@ generate_index(void* raw_data, } // namespace TEST(CApiTest, CollectionTest) { - auto collection = NewCollection(get_default_schema_config()); + auto collection = NewCollection(get_default_schema_config().c_str()); DeleteCollection(collection); } @@ -373,7 +249,7 @@ TEST(CApiTest, LoadInfoTest) { } TEST(CApiTest, SetIndexMetaTest) { - auto collection = NewCollection(get_default_schema_config()); + auto collection = NewCollection(get_default_schema_config().c_str()); milvus::proto::segcore::CollectionIndexMeta indexMeta; indexMeta.ParseFromString(get_default_index_meta()); @@ -384,7 +260,7 @@ TEST(CApiTest, SetIndexMetaTest) { } TEST(CApiTest, GetCollectionNameTest) { - auto collection = NewCollection(get_default_schema_config()); + auto collection = NewCollection(get_default_schema_config().c_str()); auto name = GetCollectionName(collection); ASSERT_EQ(strcmp(name, "default-collection"), 0); DeleteCollection(collection); @@ -392,7 +268,7 @@ TEST(CApiTest, GetCollectionNameTest) { } TEST(CApiTest, SegmentTest) { - auto collection = NewCollection(get_default_schema_config()); + auto collection = NewCollection(get_default_schema_config().c_str()); CSegmentInterface segment; auto status = NewSegment(collection, Growing, -1, &segment, false); ASSERT_EQ(status.error_code, Success); @@ -404,71 +280,16 @@ TEST(CApiTest, SegmentTest) { free((char*)status.error_msg); } -TEST(CApiTest, CPlan) { - std::string schema_string = generate_collection_schema( - knowhere::metric::JACCARD, DIM, VectorType::BinaryVector); - auto collection = NewCollection(schema_string.c_str()); - - // const char* dsl_string = R"( - // { - // "bool": { - // "vector": { - // "fakevec": { - // "metric_type": "L2", - // "params": { - // "nprobe": 10 - // }, - // "query": "$0", - // "topk": 10, - // "round_decimal": 3 - // } - // } - // } - // })"; - - milvus::proto::plan::PlanNode plan_node; - auto vector_anns = plan_node.mutable_vector_anns(); - vector_anns->set_vector_type(milvus::proto::plan::VectorType::BinaryVector); - vector_anns->set_placeholder_tag("$0"); - vector_anns->set_field_id(100); - auto query_info = vector_anns->mutable_query_info(); - query_info->set_topk(10); - query_info->set_round_decimal(3); - query_info->set_metric_type("L2"); - query_info->set_search_params(R"({"nprobe": 10})"); - auto plan_str = plan_node.SerializeAsString(); - - void* plan = nullptr; - auto status = CreateSearchPlanByExpr( - collection, plan_str.data(), plan_str.size(), &plan); - ASSERT_EQ(status.error_code, Success); - - int64_t field_id = -1; - status = GetFieldID(plan, &field_id); - ASSERT_EQ(status.error_code, Success); - - auto col = static_cast(collection); - for (auto& [target_field_id, field_meta] : - col->get_schema()->get_fields()) { - if (field_meta.is_vector()) { - ASSERT_EQ(field_id, target_field_id.get()); - } - } - ASSERT_NE(field_id, -1); - - DeleteSearchPlan(plan); - DeleteCollection(collection); -} - -TEST(CApiTest, CApiCPlan_float16) { - std::string schema_string = generate_collection_schema( - knowhere::metric::L2, 16, VectorType::Float16Vector); +template +void +Test_CPlan(const knowhere::MetricType& metric_type) { + std::string schema_string = + generate_collection_schema(knowhere::metric::JACCARD, DIM); auto collection = NewCollection(schema_string.c_str()); milvus::proto::plan::PlanNode plan_node; auto vector_anns = plan_node.mutable_vector_anns(); - vector_anns->set_vector_type( - milvus::proto::plan::VectorType::Float16Vector); + vector_anns->set_vector_type(TraitType::vector_type); vector_anns->set_placeholder_tag("$0"); vector_anns->set_field_id(100); auto query_info = vector_anns->mutable_query_info(); @@ -500,48 +321,15 @@ TEST(CApiTest, CApiCPlan_float16) { DeleteCollection(collection); } -TEST(CApiTest, CApiCPlan_bfloat16) { - std::string schema_string = generate_collection_schema( - knowhere::metric::L2, 16, VectorType::BFloat16Vector); - auto collection = NewCollection(schema_string.c_str()); - - milvus::proto::plan::PlanNode plan_node; - auto vector_anns = plan_node.mutable_vector_anns(); - vector_anns->set_vector_type( - milvus::proto::plan::VectorType::BFloat16Vector); - vector_anns->set_placeholder_tag("$0"); - vector_anns->set_field_id(100); - auto query_info = vector_anns->mutable_query_info(); - query_info->set_topk(10); - query_info->set_round_decimal(3); - query_info->set_metric_type("L2"); - query_info->set_search_params(R"({"nprobe": 10})"); - auto plan_str = plan_node.SerializeAsString(); - - void* plan = nullptr; - auto status = CreateSearchPlanByExpr( - collection, plan_str.data(), plan_str.size(), &plan); - ASSERT_EQ(status.error_code, Success); - - int64_t field_id = -1; - status = GetFieldID(plan, &field_id); - ASSERT_EQ(status.error_code, Success); - - auto col = static_cast(collection); - for (auto& [target_field_id, field_meta] : - col->get_schema()->get_fields()) { - if (field_meta.is_vector()) { - ASSERT_EQ(field_id, target_field_id.get()); - } - } - ASSERT_NE(field_id, -1); - - DeleteSearchPlan(plan); - DeleteCollection(collection); +TEST(CApiTest, CPlan) { + Test_CPlan(knowhere::metric::JACCARD); + Test_CPlan(knowhere::metric::L2); + Test_CPlan(knowhere::metric::L2); + Test_CPlan(knowhere::metric::L2); } TEST(CApiTest, InsertTest) { - auto c_collection = NewCollection(get_default_schema_config()); + auto c_collection = NewCollection(get_default_schema_config().c_str()); CSegmentInterface segment; auto status = NewSegment(c_collection, Growing, -1, &segment, false); ASSERT_EQ(status.error_code, Success); @@ -568,7 +356,7 @@ TEST(CApiTest, InsertTest) { } TEST(CApiTest, DeleteTest) { - auto collection = NewCollection(get_default_schema_config()); + auto collection = NewCollection(get_default_schema_config().c_str()); CSegmentInterface segment; auto status = NewSegment(collection, Growing, -1, &segment, false); ASSERT_EQ(status.error_code, Success); @@ -594,7 +382,7 @@ TEST(CApiTest, DeleteTest) { } TEST(CApiTest, MultiDeleteGrowingSegment) { - auto collection = NewCollection(get_default_schema_config()); + auto collection = NewCollection(get_default_schema_config().c_str()); CSegmentInterface segment; auto status = NewSegment(collection, Growing, -1, &segment, false); ASSERT_EQ(status.error_code, Success); @@ -730,7 +518,7 @@ TEST(CApiTest, GetIndexListSizeAndFeatures) { } TEST(CApiTest, MultiDeleteSealedSegment) { - auto collection = NewCollection(get_default_schema_config()); + auto collection = NewCollection(get_default_schema_config().c_str()); CSegmentInterface segment; auto status = NewSegment(collection, Sealed, -1, &segment, false); ASSERT_EQ(status.error_code, Success); @@ -842,7 +630,7 @@ TEST(CApiTest, MultiDeleteSealedSegment) { } TEST(CApiTest, DeleteRepeatedPksFromGrowingSegment) { - auto collection = NewCollection(get_default_schema_config()); + auto collection = NewCollection(get_default_schema_config().c_str()); CSegmentInterface segment; auto status = NewSegment(collection, Growing, -1, &segment, false); ASSERT_EQ(status.error_code, Success); @@ -946,7 +734,7 @@ TEST(CApiTest, DeleteRepeatedPksFromGrowingSegment) { } TEST(CApiTest, DeleteRepeatedPksFromSealedSegment) { - auto collection = NewCollection(get_default_schema_config()); + auto collection = NewCollection(get_default_schema_config().c_str()); CSegmentInterface segment; auto status = NewSegment(collection, Sealed, -1, &segment, false); ASSERT_EQ(status.error_code, Success); @@ -1064,7 +852,7 @@ TEST(CApiTest, SearcTestWhenNullable) { auto plan_str = plan_node.SerializeAsString(); int num_queries = 10; - auto blob = generate_query_data(num_queries); + auto blob = generate_query_data(num_queries); void* plan = nullptr; status = CreateSearchPlanByExpr( @@ -1096,7 +884,7 @@ TEST(CApiTest, SearcTestWhenNullable) { } TEST(CApiTest, InsertSamePkAfterDeleteOnGrowingSegment) { - auto collection = NewCollection(get_default_schema_config()); + auto collection = NewCollection(get_default_schema_config().c_str()); CSegmentInterface segment; auto status = NewSegment(collection, Growing, 111, &segment, false); ASSERT_EQ(status.error_code, Success); @@ -1202,7 +990,7 @@ TEST(CApiTest, InsertSamePkAfterDeleteOnGrowingSegment) { } TEST(CApiTest, InsertSamePkAfterDeleteOnSealedSegment) { - auto collection = NewCollection(get_default_schema_config()); + auto collection = NewCollection(get_default_schema_config().c_str()); CSegmentInterface segment; auto status = NewSegment(collection, Sealed, -1, &segment, false); ASSERT_EQ(status.error_code, Success); @@ -1274,7 +1062,7 @@ TEST(CApiTest, InsertSamePkAfterDeleteOnSealedSegment) { } TEST(CApiTest, SearchTest) { - auto c_collection = NewCollection(get_default_schema_config()); + auto c_collection = NewCollection(get_default_schema_config().c_str()); CSegmentInterface segment; auto status = NewSegment(c_collection, Growing, -1, &segment, false); ASSERT_EQ(status.error_code, Success); @@ -1310,7 +1098,7 @@ TEST(CApiTest, SearchTest) { auto plan_str = plan_node.SerializeAsString(); int num_queries = 10; - auto blob = generate_query_data(num_queries); + auto blob = generate_query_data(num_queries); void* plan = nullptr; status = CreateSearchPlanByExpr( @@ -1344,7 +1132,7 @@ TEST(CApiTest, SearchTest) { } TEST(CApiTest, SearchTestWithExpr) { - auto c_collection = NewCollection(get_default_schema_config()); + auto c_collection = NewCollection(get_default_schema_config().c_str()); CSegmentInterface segment; auto status = NewSegment(c_collection, Growing, -1, &segment, false); ASSERT_EQ(status.error_code, Success); @@ -1377,7 +1165,7 @@ TEST(CApiTest, SearchTestWithExpr) { >)"; int num_queries = 10; - auto blob = generate_query_data(num_queries); + auto blob = generate_query_data(num_queries); void* plan = nullptr; auto binary_plan = translate_text_plan_to_binary_plan(serialized_expr_plan); @@ -1411,7 +1199,7 @@ TEST(CApiTest, SearchTestWithExpr) { } TEST(CApiTest, RetrieveTestWithExpr) { - auto collection = NewCollection(get_default_schema_config()); + auto collection = NewCollection(get_default_schema_config().c_str()); CSegmentInterface segment; auto status = NewSegment(collection, Growing, -1, &segment, false); ASSERT_EQ(status.error_code, Success); @@ -1472,7 +1260,7 @@ TEST(CApiTest, RetrieveTestWithExpr) { } TEST(CApiTest, GetMemoryUsageInBytesTest) { - auto collection = NewCollection(get_default_schema_config()); + auto collection = NewCollection(get_default_schema_config().c_str()); CSegmentInterface segment; auto status = NewSegment(collection, Growing, -1, &segment, false); ASSERT_EQ(status.error_code, Success); @@ -1503,7 +1291,7 @@ TEST(CApiTest, GetMemoryUsageInBytesTest) { } TEST(CApiTest, GetDeletedCountTest) { - auto collection = NewCollection(get_default_schema_config()); + auto collection = NewCollection(get_default_schema_config().c_str()); CSegmentInterface segment; auto status = NewSegment(collection, Growing, -1, &segment, false); ASSERT_EQ(status.error_code, Success); @@ -1534,7 +1322,7 @@ TEST(CApiTest, GetDeletedCountTest) { } TEST(CApiTest, GetRowCountTest) { - auto collection = NewCollection(get_default_schema_config()); + auto collection = NewCollection(get_default_schema_config().c_str()); CSegmentInterface segment; auto status = NewSegment(collection, Growing, -1, &segment, false); ASSERT_EQ(status.error_code, Success); @@ -1564,7 +1352,7 @@ TEST(CApiTest, GetRowCountTest) { } TEST(CApiTest, GetRealCount) { - auto collection = NewCollection(get_default_schema_config()); + auto collection = NewCollection(get_default_schema_config().c_str()); CSegmentInterface segment; auto status = NewSegment(collection, Growing, -1, &segment, false); ASSERT_EQ(status.error_code, Success); @@ -1614,7 +1402,7 @@ TEST(CApiTest, GetRealCount) { } TEST(CApiTest, ReduceNullResult) { - auto collection = NewCollection(get_default_schema_config()); + auto collection = NewCollection(get_default_schema_config().c_str()); CSegmentInterface segment; auto status = NewSegment(collection, Growing, -1, &segment, false); ASSERT_EQ(status.error_code, Success); @@ -1699,7 +1487,7 @@ TEST(CApiTest, ReduceNullResult) { } TEST(CApiTest, ReduceRemoveDuplicates) { - auto collection = NewCollection(get_default_schema_config()); + auto collection = NewCollection(get_default_schema_config().c_str()); CSegmentInterface segment; auto status = NewSegment(collection, Growing, -1, &segment, false); ASSERT_EQ(status.error_code, Success); @@ -1736,7 +1524,7 @@ TEST(CApiTest, ReduceRemoveDuplicates) { int num_queries = 10; int topK = 10; - auto blob = generate_query_data(num_queries); + auto blob = generate_query_data(num_queries); void* plan = nullptr; status = CreateSearchPlanByExpr( @@ -1829,7 +1617,7 @@ TEST(CApiTest, ReduceRemoveDuplicates) { DeleteSegment(segment); } -template +template void testReduceSearchWithExpr(int N, int topK, @@ -1837,19 +1625,9 @@ testReduceSearchWithExpr(int N, bool filter_all = false) { std::cerr << "testReduceSearchWithExpr(" << N << ", " << topK << ", " << num_queries << ")" << std::endl; - std::function schema_fun; - std::function query_gen_fun; - if constexpr (std::is_same_v) { - schema_fun = get_default_schema_config; - query_gen_fun = generate_query_data; - } else if constexpr (std::is_same_v) { - schema_fun = get_float16_schema_config; - query_gen_fun = generate_query_data_float16; - } else if constexpr (std::is_same_v) { - schema_fun = get_bfloat16_schema_config; - query_gen_fun = generate_query_data_bfloat16; - } - auto collection = NewCollection(schema_fun()); + + auto collection = + NewCollection(get_default_schema_config().c_str()); CSegmentInterface segment; auto status = NewSegment(collection, Growing, -1, &segment, false); ASSERT_EQ(status.error_code, Success); @@ -1907,7 +1685,7 @@ testReduceSearchWithExpr(int N, topK % N; } auto serialized_expr_plan = fmt.str(); - auto blob = query_gen_fun(num_queries); + auto blob = generate_query_data(num_queries); void* plan = nullptr; auto binary_plan = @@ -1997,29 +1775,29 @@ testReduceSearchWithExpr(int N, } TEST(CApiTest, ReduceSearchWithExpr) { - //float32 - testReduceSearchWithExpr(2, 1, 1); - testReduceSearchWithExpr(2, 10, 10); - testReduceSearchWithExpr(100, 1, 1); - testReduceSearchWithExpr(100, 10, 10); - testReduceSearchWithExpr(10000, 1, 1); - testReduceSearchWithExpr(10000, 10, 10); - //float16 - testReduceSearchWithExpr(2, 10, 10, false); - testReduceSearchWithExpr(100, 10, 10, false); - //bfloat16 - testReduceSearchWithExpr(2, 10, 10, false); - testReduceSearchWithExpr(100, 10, 10, false); + // float32 + testReduceSearchWithExpr(2, 1, 1); + testReduceSearchWithExpr(2, 10, 10); + testReduceSearchWithExpr(100, 1, 1); + testReduceSearchWithExpr(100, 10, 10); + testReduceSearchWithExpr(10000, 1, 1); + testReduceSearchWithExpr(10000, 10, 10); + // float16 + testReduceSearchWithExpr(2, 10, 10, false); + testReduceSearchWithExpr(100, 10, 10, false); + // bfloat16 + testReduceSearchWithExpr(2, 10, 10, false); + testReduceSearchWithExpr(100, 10, 10, false); } TEST(CApiTest, ReduceSearchWithExprFilterAll) { - //float32 - testReduceSearchWithExpr(2, 1, 1, true); - testReduceSearchWithExpr(2, 10, 10, true); - //float16 - testReduceSearchWithExpr(2, 1, 1, true); - //bfloat16 - testReduceSearchWithExpr(2, 1, 1, true); + // float32 + testReduceSearchWithExpr(2, 1, 1, true); + testReduceSearchWithExpr(2, 10, 10, true); + // float16 + testReduceSearchWithExpr(2, 1, 1, true); + // bfloat16 + testReduceSearchWithExpr(2, 1, 1, true); } TEST(CApiTest, LoadIndexInfo) { @@ -2118,12 +1896,16 @@ TEST(CApiTest, LoadIndexSearch) { auto result = indexing.Search(query_dataset, conf, nullptr); } -TEST(CApiTest, Indexing_Without_Predicate) { +template +void +Test_Indexing_Without_Predicate() { + GET_ELEM_TYPE_FOR_VECTOR_TRAIT + // insert data to segment constexpr auto TOPK = 5; - std::string schema_string = generate_collection_schema( - knowhere::metric::L2, DIM, VectorType::FloatVector); + std::string schema_string = + generate_collection_schema(knowhere::metric::L2, DIM); auto collection = NewCollection(schema_string.c_str()); auto schema = ((segcore::Collection*)collection)->get_schema(); CSegmentInterface segment; @@ -2132,7 +1914,7 @@ TEST(CApiTest, Indexing_Without_Predicate) { auto N = ROW_COUNT; auto dataset = DataGen(schema, N); - auto vec_col = dataset.get_col(FieldId(100)); + auto vec_col = dataset.get_col(FieldId(100)); auto query_ptr = vec_col.data() + BIAS * DIM; int64_t offset; @@ -2150,7 +1932,7 @@ TEST(CApiTest, Indexing_Without_Predicate) { milvus::proto::plan::PlanNode plan_node; auto vector_anns = plan_node.mutable_vector_anns(); - vector_anns->set_vector_type(milvus::proto::plan::VectorType::FloatVector); + vector_anns->set_vector_type(TraitType::vector_type); vector_anns->set_placeholder_tag("$0"); vector_anns->set_field_id(100); auto query_info = vector_anns->mutable_query_info(); @@ -2163,7 +1945,7 @@ TEST(CApiTest, Indexing_Without_Predicate) { // create place_holder_group int num_queries = 5; auto raw_group = - CreatePlaceholderGroupFromBlob(num_queries, DIM, query_ptr); + CreatePlaceholderGroupFromBlob(num_queries, DIM, query_ptr); auto blob = raw_group.SerializeAsString(); // search on segment's small index @@ -2192,7 +1974,7 @@ TEST(CApiTest, Indexing_Without_Predicate) { // load index to segment auto indexing = generate_index(vec_col.data(), - DataType::VECTOR_FLOAT, + TraitType::data_type, knowhere::metric::L2, IndexEnum::INDEX_FAISS_IVFSQ8, DIM, @@ -2232,7 +2014,7 @@ TEST(CApiTest, Indexing_Without_Predicate) { AppendIndexParam( c_load_index_info, metric_type_key.c_str(), metric_type_value.c_str()); AppendFieldInfo( - c_load_index_info, 0, 0, 0, 100, CDataType::FloatVector, false, ""); + c_load_index_info, 0, 0, 0, 100, TraitType::c_data_type, false, ""); AppendIndexEngineVersionToLoadInfo( c_load_index_info, knowhere::Version::GetCurrentVersion().VersionNumber()); @@ -2267,12 +2049,18 @@ TEST(CApiTest, Indexing_Without_Predicate) { DeleteSegment(segment); } +TEST(CApiTest, Indexing_Without_Predicate) { + Test_Indexing_Without_Predicate(); + Test_Indexing_Without_Predicate(); + Test_Indexing_Without_Predicate(); +} + TEST(CApiTest, Indexing_Expr_Without_Predicate) { // insert data to segment constexpr auto TOPK = 5; - std::string schema_string = generate_collection_schema( - knowhere::metric::L2, DIM, VectorType::FloatVector); + std::string schema_string = generate_collection_schema( + knowhere::metric::L2, DIM); auto collection = NewCollection(schema_string.c_str()); auto schema = ((segcore::Collection*)collection)->get_schema(); CSegmentInterface segment; @@ -2421,8 +2209,8 @@ TEST(CApiTest, Indexing_With_float_Predicate_Range) { // insert data to segment constexpr auto TOPK = 5; - std::string schema_string = generate_collection_schema( - knowhere::metric::L2, DIM, VectorType::FloatVector); + std::string schema_string = generate_collection_schema( + knowhere::metric::L2, DIM); auto collection = NewCollection(schema_string.c_str()); auto schema = ((segcore::Collection*)collection)->get_schema(); CSegmentInterface segment; @@ -2599,8 +2387,8 @@ TEST(CApiTest, Indexing_Expr_With_float_Predicate_Range) { // insert data to segment constexpr auto TOPK = 5; - std::string schema_string = generate_collection_schema( - knowhere::metric::L2, DIM, VectorType::FloatVector); + std::string schema_string = generate_collection_schema( + knowhere::metric::L2, DIM); auto collection = NewCollection(schema_string.c_str()); auto schema = ((segcore::Collection*)collection)->get_schema(); CSegmentInterface segment; @@ -2779,8 +2567,8 @@ TEST(CApiTest, Indexing_With_float_Predicate_Term) { // insert data to segment constexpr auto TOPK = 5; - std::string schema_string = generate_collection_schema( - knowhere::metric::L2, DIM, VectorType::FloatVector); + std::string schema_string = generate_collection_schema( + knowhere::metric::L2, DIM); auto collection = NewCollection(schema_string.c_str()); auto schema = ((segcore::Collection*)collection)->get_schema(); CSegmentInterface segment; @@ -2951,8 +2739,8 @@ TEST(CApiTest, Indexing_Expr_With_float_Predicate_Term) { // insert data to segment constexpr auto TOPK = 5; - std::string schema_string = generate_collection_schema( - knowhere::metric::L2, DIM, VectorType::FloatVector); + std::string schema_string = generate_collection_schema( + knowhere::metric::L2, DIM); auto collection = NewCollection(schema_string.c_str()); auto schema = ((segcore::Collection*)collection)->get_schema(); CSegmentInterface segment; @@ -3124,8 +2912,9 @@ TEST(CApiTest, Indexing_With_binary_Predicate_Range) { // insert data to segment constexpr auto TOPK = 5; - std::string schema_string = generate_collection_schema( - knowhere::metric::JACCARD, DIM, VectorType::BinaryVector); + std::string schema_string = + generate_collection_schema( + knowhere::metric::JACCARD, DIM); auto collection = NewCollection(schema_string.c_str(), knowhere::metric::JACCARD); auto schema = ((segcore::Collection*)collection)->get_schema(); @@ -3194,8 +2983,8 @@ TEST(CApiTest, Indexing_With_binary_Predicate_Range) { // create place_holder_group int num_queries = 5; - auto raw_group = - CreateBinaryPlaceholderGroupFromBlob(num_queries, DIM, query_ptr); + auto raw_group = CreatePlaceholderGroupFromBlob( + num_queries, DIM, query_ptr); auto blob = raw_group.SerializeAsString(); // search on segment's small index @@ -3304,8 +3093,9 @@ TEST(CApiTest, Indexing_Expr_With_binary_Predicate_Range) { // insert data to segment constexpr auto TOPK = 5; - std::string schema_string = generate_collection_schema( - knowhere::metric::JACCARD, DIM, VectorType::BinaryVector); + std::string schema_string = + generate_collection_schema( + knowhere::metric::JACCARD, DIM); auto collection = NewCollection(schema_string.c_str(), knowhere::metric::JACCARD); auto schema = ((segcore::Collection*)collection)->get_schema(); @@ -3373,8 +3163,8 @@ TEST(CApiTest, Indexing_Expr_With_binary_Predicate_Range) { // create place_holder_group int num_queries = 5; - auto raw_group = - CreateBinaryPlaceholderGroupFromBlob(num_queries, DIM, query_ptr); + auto raw_group = CreatePlaceholderGroupFromBlob( + num_queries, DIM, query_ptr); auto blob = raw_group.SerializeAsString(); // search on segment's small index @@ -3484,8 +3274,9 @@ TEST(CApiTest, Indexing_With_binary_Predicate_Term) { // insert data to segment constexpr auto TOPK = 5; - std::string schema_string = generate_collection_schema( - knowhere::metric::JACCARD, DIM, VectorType::BinaryVector); + std::string schema_string = + generate_collection_schema( + knowhere::metric::JACCARD, DIM); auto collection = NewCollection(schema_string.c_str(), knowhere::metric::JACCARD); auto schema = ((segcore::Collection*)collection)->get_schema(); @@ -3549,8 +3340,8 @@ TEST(CApiTest, Indexing_With_binary_Predicate_Term) { // create place_holder_group int num_queries = 5; int topK = 5; - auto raw_group = - CreateBinaryPlaceholderGroupFromBlob(num_queries, DIM, query_ptr); + auto raw_group = CreatePlaceholderGroupFromBlob( + num_queries, DIM, query_ptr); auto blob = raw_group.SerializeAsString(); // search on segment's small index @@ -3681,8 +3472,9 @@ TEST(CApiTest, Indexing_Expr_With_binary_Predicate_Term) { // insert data to segment constexpr auto TOPK = 5; - std::string schema_string = generate_collection_schema( - knowhere::metric::JACCARD, DIM, VectorType::BinaryVector); + std::string schema_string = + generate_collection_schema( + knowhere::metric::JACCARD, DIM); auto collection = NewCollection(schema_string.c_str(), knowhere::metric::JACCARD); auto schema = ((segcore::Collection*)collection)->get_schema(); @@ -3745,8 +3537,8 @@ TEST(CApiTest, Indexing_Expr_With_binary_Predicate_Term) { // create place_holder_group int num_queries = 5; int topK = 5; - auto raw_group = - CreateBinaryPlaceholderGroupFromBlob(num_queries, DIM, query_ptr); + auto raw_group = CreatePlaceholderGroupFromBlob( + num_queries, DIM, query_ptr); auto blob = raw_group.SerializeAsString(); // search on segment's small index @@ -3872,7 +3664,7 @@ TEST(CApiTest, Indexing_Expr_With_binary_Predicate_Term) { } TEST(CApiTest, SealedSegmentTest) { - auto collection = NewCollection(get_default_schema_config()); + auto collection = NewCollection(get_default_schema_config().c_str()); CSegmentInterface segment; auto status = NewSegment(collection, Sealed, -1, &segment, false); ASSERT_EQ(status.error_code, Success); @@ -3895,8 +3687,8 @@ TEST(CApiTest, SealedSegmentTest) { TEST(CApiTest, SealedSegment_search_float_Predicate_Range) { constexpr auto TOPK = 5; - std::string schema_string = generate_collection_schema( - knowhere::metric::L2, DIM, VectorType::FloatVector); + std::string schema_string = generate_collection_schema( + knowhere::metric::L2, DIM); auto collection = NewCollection(schema_string.c_str()); auto schema = ((segcore::Collection*)collection)->get_schema(); CSegmentInterface segment; @@ -4048,8 +3840,8 @@ TEST(CApiTest, SealedSegment_search_float_Predicate_Range) { TEST(CApiTest, SealedSegment_search_without_predicates) { constexpr auto TOPK = 5; - std::string schema_string = generate_collection_schema( - knowhere::metric::L2, DIM, VectorType::FloatVector); + std::string schema_string = generate_collection_schema( + knowhere::metric::L2, DIM); auto collection = NewCollection(schema_string.c_str()); auto schema = ((segcore::Collection*)collection)->get_schema(); CSegmentInterface segment; @@ -4092,7 +3884,7 @@ TEST(CApiTest, SealedSegment_search_without_predicates) { ASSERT_EQ(status.error_code, Success); int num_queries = 10; - auto blob = generate_query_data(num_queries); + auto blob = generate_query_data(num_queries); void* plan = nullptr; status = CreateSearchPlanByExpr( @@ -4128,8 +3920,8 @@ TEST(CApiTest, SealedSegment_search_without_predicates) { TEST(CApiTest, SealedSegment_search_float_With_Expr_Predicate_Range) { constexpr auto TOPK = 5; - std::string schema_string = generate_collection_schema( - knowhere::metric::L2, DIM, VectorType::FloatVector); + std::string schema_string = generate_collection_schema( + knowhere::metric::L2, DIM); auto collection = NewCollection(schema_string.c_str()); auto schema = ((segcore::Collection*)collection)->get_schema(); CSegmentInterface segment; @@ -4510,8 +4302,11 @@ TEST(CApiTest, RetriveScalarFieldFromSealedSegmentWithIndex) { DeleteSegment(segment); } -TEST(CApiTest, RANGE_SEARCH_WITH_RADIUS_WHEN_IP) { - auto c_collection = NewCollection(get_default_schema_config()); +template +void +Test_Range_Search_With_Radius_And_Range_Filter() { + auto c_collection = + NewCollection(get_default_schema_config().c_str()); CSegmentInterface segment; auto status = NewSegment(c_collection, Growing, -1, &segment, false); ASSERT_EQ(status.error_code, Success); @@ -4539,15 +4334,15 @@ TEST(CApiTest, RANGE_SEARCH_WITH_RADIUS_WHEN_IP) { query_info: < topk: 10 round_decimal: 3 - metric_type: "IP" - search_params: "{\"nprobe\": 10,\"radius\": 10}" + metric_type: "L2" + search_params: "{\"nprobe\": 10,\"radius\": 20, \"range_filter\": 10}" > placeholder_tag: "$0" >)"; auto plan_str = translate_text_plan_to_binary_plan(raw_plan); int num_queries = 10; - auto blob = generate_query_data(num_queries); + auto blob = generate_query_data(num_queries); void* plan = nullptr; status = CreateSearchPlanByExpr( @@ -4574,232 +4369,45 @@ TEST(CApiTest, RANGE_SEARCH_WITH_RADIUS_WHEN_IP) { DeleteSegment(segment); } -TEST(CApiTest, RANGE_SEARCH_WITH_RADIUS_AND_RANGE_FILTER_WHEN_IP) { - auto c_collection = - NewCollection(get_default_schema_config(), knowhere::metric::IP); - CSegmentInterface segment; - auto status = NewSegment(c_collection, Growing, -1, &segment, false); - ASSERT_EQ(status.error_code, Success); - auto col = (milvus::segcore::Collection*)c_collection; - - int N = 10000; - auto dataset = DataGen(col->get_schema(), N); - int64_t ts_offset = 1000; - - int64_t offset; - PreInsert(segment, N, &offset); - - auto insert_data = serialize(dataset.raw_); - auto ins_res = Insert(segment, - offset, - N, - dataset.row_ids_.data(), - dataset.timestamps_.data(), - insert_data.data(), - insert_data.size()); - ASSERT_EQ(ins_res.error_code, Success); - - const char* raw_plan = R"(vector_anns: < - field_id: 100 - query_info: < - topk: 10 - round_decimal: 3 - metric_type: "IP" - search_params: "{\"nprobe\": 10,\"radius\": 10, \"range_filter\": 20}" - > - placeholder_tag: "$0" - >)"; - auto plan_str = translate_text_plan_to_binary_plan(raw_plan); - - int num_queries = 10; - auto blob = generate_query_data(num_queries); - - void* plan = nullptr; - status = CreateSearchPlanByExpr( - c_collection, plan_str.data(), plan_str.size(), &plan); - ASSERT_EQ(status.error_code, Success); - - void* placeholderGroup = nullptr; - status = ParsePlaceholderGroup( - plan, blob.data(), blob.length(), &placeholderGroup); - ASSERT_EQ(status.error_code, Success); - - std::vector placeholderGroups; - placeholderGroups.push_back(placeholderGroup); - - CSearchResult search_result; - auto res = - CSearch(segment, plan, placeholderGroup, ts_offset, &search_result); - ASSERT_EQ(res.error_code, Success); - - DeleteSearchPlan(plan); - DeletePlaceholderGroup(placeholderGroup); - DeleteSearchResult(search_result); - DeleteCollection(c_collection); - DeleteSegment(segment); +TEST(CApiTest, Range_Search_With_Radius_And_Range_Filter) { + Test_Range_Search_With_Radius_And_Range_Filter(); + Test_Range_Search_With_Radius_And_Range_Filter(); + Test_Range_Search_With_Radius_And_Range_Filter(); } -TEST(CApiTest, RANGE_SEARCH_WITH_RADIUS_WHEN_L2) { - auto c_collection = NewCollection(get_default_schema_config()); - CSegmentInterface segment; - auto status = NewSegment(c_collection, Growing, -1, &segment, false); - ASSERT_EQ(status.error_code, Success); - auto col = (milvus::segcore::Collection*)c_collection; +std::vector +search_id(const BitsetType& bitset, + Timestamp* timestamps, + Timestamp timestamp, + bool use_find) { + std::vector dst_offset; + if (use_find) { + auto i = bitset.find_first(); + while (i.has_value()) { + auto offset = SegOffset(i.value()); + if (timestamps[offset.get()] <= timestamp) { + dst_offset.push_back(offset); + } - int N = 10000; - auto dataset = DataGen(col->get_schema(), N); - int64_t ts_offset = 1000; + i = bitset.find_next(i.value()); + } - int64_t offset; - PreInsert(segment, N, &offset); + return dst_offset; + } else { + for (int i = 0; i < bitset.size(); i++) { + if (bitset[i]) { + auto offset = SegOffset(i); + if (timestamps[offset.get()] <= timestamp) { + dst_offset.push_back(offset); + } + } + } + } + return dst_offset; +} - auto insert_data = serialize(dataset.raw_); - auto ins_res = Insert(segment, - offset, - N, - dataset.row_ids_.data(), - dataset.timestamps_.data(), - insert_data.data(), - insert_data.size()); - ASSERT_EQ(ins_res.error_code, Success); - - const char* raw_plan = R"(vector_anns: < - field_id: 100 - query_info: < - topk: 10 - round_decimal: 3 - metric_type: "L2" - search_params: "{\"nprobe\": 10,\"radius\": 10}" - > - placeholder_tag: "$0" - >)"; - auto plan_str = translate_text_plan_to_binary_plan(raw_plan); - - int num_queries = 10; - auto blob = generate_query_data(num_queries); - - void* plan = nullptr; - status = CreateSearchPlanByExpr( - c_collection, plan_str.data(), plan_str.size(), &plan); - ASSERT_EQ(status.error_code, Success); - - void* placeholderGroup = nullptr; - status = ParsePlaceholderGroup( - plan, blob.data(), blob.length(), &placeholderGroup); - ASSERT_EQ(status.error_code, Success); - - std::vector placeholderGroups; - placeholderGroups.push_back(placeholderGroup); - - CSearchResult search_result; - auto res = - CSearch(segment, plan, placeholderGroup, ts_offset, &search_result); - ASSERT_EQ(res.error_code, Success); - - DeleteSearchPlan(plan); - DeletePlaceholderGroup(placeholderGroup); - DeleteSearchResult(search_result); - DeleteCollection(c_collection); - DeleteSegment(segment); -} - -TEST(CApiTest, RANGE_SEARCH_WITH_RADIUS_AND_RANGE_FILTER_WHEN_L2) { - auto c_collection = NewCollection(get_default_schema_config()); - CSegmentInterface segment; - auto status = NewSegment(c_collection, Growing, -1, &segment, false); - ASSERT_EQ(status.error_code, Success); - auto col = (milvus::segcore::Collection*)c_collection; - - int N = 10000; - auto dataset = DataGen(col->get_schema(), N); - int64_t ts_offset = 1000; - - int64_t offset; - PreInsert(segment, N, &offset); - - auto insert_data = serialize(dataset.raw_); - auto ins_res = Insert(segment, - offset, - N, - dataset.row_ids_.data(), - dataset.timestamps_.data(), - insert_data.data(), - insert_data.size()); - ASSERT_EQ(ins_res.error_code, Success); - - const char* raw_plan = R"(vector_anns: < - field_id: 100 - query_info: < - topk: 10 - round_decimal: 3 - metric_type: "L2" - search_params: "{\"nprobe\": 10,\"radius\": 20, \"range_filter\": 10}" - > - placeholder_tag: "$0" - >)"; - auto plan_str = translate_text_plan_to_binary_plan(raw_plan); - - int num_queries = 10; - auto blob = generate_query_data(num_queries); - - void* plan = nullptr; - status = CreateSearchPlanByExpr( - c_collection, plan_str.data(), plan_str.size(), &plan); - ASSERT_EQ(status.error_code, Success); - - void* placeholderGroup = nullptr; - status = ParsePlaceholderGroup( - plan, blob.data(), blob.length(), &placeholderGroup); - ASSERT_EQ(status.error_code, Success); - - std::vector placeholderGroups; - placeholderGroups.push_back(placeholderGroup); - - CSearchResult search_result; - auto res = - CSearch(segment, plan, placeholderGroup, ts_offset, &search_result); - ASSERT_EQ(res.error_code, Success); - - DeleteSearchPlan(plan); - DeletePlaceholderGroup(placeholderGroup); - DeleteSearchResult(search_result); - DeleteCollection(c_collection); - DeleteSegment(segment); -} - -std::vector -search_id(const BitsetType& bitset, - Timestamp* timestamps, - Timestamp timestamp, - bool use_find) { - std::vector dst_offset; - if (use_find) { - auto i = bitset.find_first(); - while (i.has_value()) { - auto offset = SegOffset(i.value()); - if (timestamps[offset.get()] <= timestamp) { - dst_offset.push_back(offset); - } - - i = bitset.find_next(i.value()); - } - - return dst_offset; - } else { - for (int i = 0; i < bitset.size(); i++) { - if (bitset[i]) { - auto offset = SegOffset(i); - if (timestamps[offset.get()] <= timestamp) { - dst_offset.push_back(offset); - } - } - } - } - return dst_offset; -} - -TEST(CApiTest, SearchIdTest) { - // using BitsetType = boost::dynamic_bitset<>; +TEST(CApiTest, SearchIdTest) { + // using BitsetType = boost::dynamic_bitset<>; auto test = [&](int NT) { BitsetType bitset(1000000); @@ -4848,436 +4456,6 @@ TEST(CApiTest, SearchIdTest) { } } -TEST(CApiTest, Indexing_Without_Predicate_float16) { - // insert data to segment - constexpr auto TOPK = 5; - - std::string schema_string = generate_collection_schema( - knowhere::metric::L2, DIM, VectorType::Float16Vector); - auto collection = NewCollection(schema_string.c_str()); - auto schema = ((segcore::Collection*)collection)->get_schema(); - CSegmentInterface segment; - auto status = NewSegment(collection, Growing, -1, &segment, false); - ASSERT_EQ(status.error_code, Success); - - auto N = ROW_COUNT; - auto dataset = DataGen(schema, N); - auto vec_col = dataset.get_col(FieldId(100)); - auto query_ptr = vec_col.data() + BIAS * DIM; - - int64_t offset; - PreInsert(segment, N, &offset); - - auto insert_data = serialize(dataset.raw_); - auto ins_res = Insert(segment, - offset, - N, - dataset.row_ids_.data(), - dataset.timestamps_.data(), - insert_data.data(), - insert_data.size()); - ASSERT_EQ(ins_res.error_code, Success); - - milvus::proto::plan::PlanNode plan_node; - auto vector_anns = plan_node.mutable_vector_anns(); - vector_anns->set_vector_type( - milvus::proto::plan::VectorType::Float16Vector); - vector_anns->set_placeholder_tag("$0"); - vector_anns->set_field_id(100); - auto query_info = vector_anns->mutable_query_info(); - query_info->set_topk(5); - query_info->set_round_decimal(-1); - query_info->set_metric_type("L2"); - query_info->set_search_params(R"({"nprobe": 10})"); - auto plan_str = plan_node.SerializeAsString(); - - // create place_holder_group - int num_queries = 5; - auto raw_group = - CreateFloat16PlaceholderGroupFromBlob(num_queries, DIM, query_ptr); - auto blob = raw_group.SerializeAsString(); - - // search on segment's small index - void* plan = nullptr; - status = CreateSearchPlanByExpr( - collection, plan_str.data(), plan_str.size(), &plan); - ASSERT_EQ(status.error_code, Success); - - void* placeholderGroup = nullptr; - status = ParsePlaceholderGroup( - plan, blob.data(), blob.length(), &placeholderGroup); - ASSERT_EQ(status.error_code, Success); - - std::vector placeholderGroups; - placeholderGroups.push_back(placeholderGroup); - - Timestamp timestmap = 10000000; - - CSearchResult c_search_result_on_smallIndex; - auto res_before_load_index = CSearch(segment, - plan, - placeholderGroup, - timestmap, - &c_search_result_on_smallIndex); - ASSERT_EQ(res_before_load_index.error_code, Success); - - // load index to segment - auto indexing = generate_index(vec_col.data(), - DataType::VECTOR_FLOAT16, - knowhere::metric::L2, - IndexEnum::INDEX_FAISS_IDMAP, - DIM, - N); - - // gen query dataset - auto query_dataset = knowhere::GenDataSet(num_queries, DIM, query_ptr); - auto vec_index = dynamic_cast(indexing.get()); - auto search_plan = reinterpret_cast(plan); - SearchInfo search_info = search_plan->plan_node_->search_info_; - SearchResult result_on_index; - vec_index->Query(query_dataset, search_info, nullptr, result_on_index); - auto ids = result_on_index.seg_offsets_.data(); - auto dis = result_on_index.distances_.data(); - std::vector vec_ids(ids, ids + TOPK * num_queries); - std::vector vec_dis; - for (int j = 0; j < TOPK * num_queries; ++j) { - vec_dis.push_back(dis[j] * -1); - } - - auto search_result_on_raw_index = - (SearchResult*)c_search_result_on_smallIndex; - search_result_on_raw_index->seg_offsets_ = vec_ids; - search_result_on_raw_index->distances_ = vec_dis; - - auto binary_set = indexing->Serialize(milvus::Config{}); - void* c_load_index_info = nullptr; - status = NewLoadIndexInfo(&c_load_index_info); - ASSERT_EQ(status.error_code, Success); - std::string index_type_key = "index_type"; - std::string index_type_value = IndexEnum::INDEX_FAISS_IDMAP; - std::string metric_type_key = "metric_type"; - std::string metric_type_value = knowhere::metric::L2; - - AppendIndexParam( - c_load_index_info, index_type_key.c_str(), index_type_value.c_str()); - AppendIndexParam( - c_load_index_info, metric_type_key.c_str(), metric_type_value.c_str()); - AppendFieldInfo( - c_load_index_info, 0, 0, 0, 100, CDataType::Float16Vector, false, ""); - AppendIndexEngineVersionToLoadInfo( - c_load_index_info, - knowhere::Version::GetCurrentVersion().VersionNumber()); - AppendIndex(c_load_index_info, (CBinarySet)&binary_set); - - // load index for vec field, load raw data for scalar field - auto sealed_segment = SealedCreator(schema, dataset); - sealed_segment->DropFieldData(FieldId(100)); - sealed_segment->LoadIndex(*(LoadIndexInfo*)c_load_index_info); - CSearchResult c_search_result_on_bigIndex; - auto res_after_load_index = CSearch(sealed_segment.get(), - plan, - placeholderGroup, - timestmap, - &c_search_result_on_bigIndex); - ASSERT_EQ(res_after_load_index.error_code, Success); - - auto search_result_on_raw_index_json = - SearchResultToJson(*search_result_on_raw_index); - auto search_result_on_bigIndex_json = - SearchResultToJson((*(SearchResult*)c_search_result_on_bigIndex)); - - ASSERT_EQ(search_result_on_raw_index_json.dump(1), - search_result_on_bigIndex_json.dump(1)); - - DeleteLoadIndexInfo(c_load_index_info); - DeleteSearchPlan(plan); - DeletePlaceholderGroup(placeholderGroup); - DeleteSearchResult(c_search_result_on_smallIndex); - DeleteSearchResult(c_search_result_on_bigIndex); - DeleteCollection(collection); - DeleteSegment(segment); -} - -TEST(CApiTest, Indexing_Without_Predicate_bfloat16) { - // insert data to segment - constexpr auto TOPK = 5; - - std::string schema_string = generate_collection_schema( - knowhere::metric::L2, DIM, VectorType::BFloat16Vector); - auto collection = NewCollection(schema_string.c_str()); - auto schema = ((segcore::Collection*)collection)->get_schema(); - CSegmentInterface segment; - auto status = NewSegment(collection, Growing, -1, &segment, false); - ASSERT_EQ(status.error_code, Success); - - auto N = ROW_COUNT; - auto dataset = DataGen(schema, N); - auto vec_col = dataset.get_col(FieldId(100)); - auto query_ptr = vec_col.data() + BIAS * DIM; - - int64_t offset; - PreInsert(segment, N, &offset); - - auto insert_data = serialize(dataset.raw_); - auto ins_res = Insert(segment, - offset, - N, - dataset.row_ids_.data(), - dataset.timestamps_.data(), - insert_data.data(), - insert_data.size()); - ASSERT_EQ(ins_res.error_code, Success); - - milvus::proto::plan::PlanNode plan_node; - auto vector_anns = plan_node.mutable_vector_anns(); - vector_anns->set_vector_type( - milvus::proto::plan::VectorType::BFloat16Vector); - vector_anns->set_placeholder_tag("$0"); - vector_anns->set_field_id(100); - auto query_info = vector_anns->mutable_query_info(); - query_info->set_topk(5); - query_info->set_round_decimal(-1); - query_info->set_metric_type("L2"); - query_info->set_search_params(R"({"nprobe": 10})"); - auto plan_str = plan_node.SerializeAsString(); - - // create place_holder_group - int num_queries = 5; - auto raw_group = - CreateBFloat16PlaceholderGroupFromBlob(num_queries, DIM, query_ptr); - auto blob = raw_group.SerializeAsString(); - - // search on segment's small index - void* plan = nullptr; - status = CreateSearchPlanByExpr( - collection, plan_str.data(), plan_str.size(), &plan); - ASSERT_EQ(status.error_code, Success); - - void* placeholderGroup = nullptr; - status = ParsePlaceholderGroup( - plan, blob.data(), blob.length(), &placeholderGroup); - ASSERT_EQ(status.error_code, Success); - - std::vector placeholderGroups; - placeholderGroups.push_back(placeholderGroup); - - Timestamp timestmap = 10000000; - - CSearchResult c_search_result_on_smallIndex; - auto res_before_load_index = CSearch(segment, - plan, - placeholderGroup, - timestmap, - &c_search_result_on_smallIndex); - ASSERT_EQ(res_before_load_index.error_code, Success); - - // load index to segment - auto indexing = generate_index(vec_col.data(), - DataType::VECTOR_BFLOAT16, - knowhere::metric::L2, - IndexEnum::INDEX_FAISS_IDMAP, - DIM, - N); - - // gen query dataset - auto query_dataset = knowhere::GenDataSet(num_queries, DIM, query_ptr); - auto vec_index = dynamic_cast(indexing.get()); - auto search_plan = reinterpret_cast(plan); - SearchInfo search_info = search_plan->plan_node_->search_info_; - SearchResult result_on_index; - vec_index->Query(query_dataset, search_info, nullptr, result_on_index); - auto ids = result_on_index.seg_offsets_.data(); - auto dis = result_on_index.distances_.data(); - std::vector vec_ids(ids, ids + TOPK * num_queries); - std::vector vec_dis; - for (int j = 0; j < TOPK * num_queries; ++j) { - vec_dis.push_back(dis[j] * -1); - } - - auto search_result_on_raw_index = - (SearchResult*)c_search_result_on_smallIndex; - search_result_on_raw_index->seg_offsets_ = vec_ids; - search_result_on_raw_index->distances_ = vec_dis; - - auto binary_set = indexing->Serialize(milvus::Config{}); - void* c_load_index_info = nullptr; - status = NewLoadIndexInfo(&c_load_index_info); - ASSERT_EQ(status.error_code, Success); - std::string index_type_key = "index_type"; - std::string index_type_value = IndexEnum::INDEX_FAISS_IDMAP; - std::string metric_type_key = "metric_type"; - std::string metric_type_value = knowhere::metric::L2; - - AppendIndexParam( - c_load_index_info, index_type_key.c_str(), index_type_value.c_str()); - AppendIndexParam( - c_load_index_info, metric_type_key.c_str(), metric_type_value.c_str()); - AppendFieldInfo( - c_load_index_info, 0, 0, 0, 100, CDataType::BFloat16Vector, false, ""); - AppendIndexEngineVersionToLoadInfo( - c_load_index_info, - knowhere::Version::GetCurrentVersion().VersionNumber()); - AppendIndex(c_load_index_info, (CBinarySet)&binary_set); - - // load index for vec field, load raw data for scalar field - auto sealed_segment = SealedCreator(schema, dataset); - sealed_segment->DropFieldData(FieldId(100)); - sealed_segment->LoadIndex(*(LoadIndexInfo*)c_load_index_info); - CSearchResult c_search_result_on_bigIndex; - auto res_after_load_index = CSearch(sealed_segment.get(), - plan, - placeholderGroup, - timestmap, - &c_search_result_on_bigIndex); - ASSERT_EQ(res_after_load_index.error_code, Success); - - auto search_result_on_raw_index_json = - SearchResultToJson(*search_result_on_raw_index); - auto search_result_on_bigIndex_json = - SearchResultToJson((*(SearchResult*)c_search_result_on_bigIndex)); - - ASSERT_EQ(search_result_on_raw_index_json.dump(1), - search_result_on_bigIndex_json.dump(1)); - - DeleteLoadIndexInfo(c_load_index_info); - DeleteSearchPlan(plan); - DeletePlaceholderGroup(placeholderGroup); - DeleteSearchResult(c_search_result_on_smallIndex); - DeleteSearchResult(c_search_result_on_bigIndex); - DeleteCollection(collection); - DeleteSegment(segment); -} - -TEST(CApiTest, RANGE_SEARCH_WITH_RADIUS_AND_RANGE_FILTER_WHEN_IP_FLOAT16) { - auto c_collection = - NewCollection(get_float16_schema_config(), knowhere::metric::IP); - CSegmentInterface segment; - auto status = NewSegment(c_collection, Growing, -1, &segment, false); - ASSERT_EQ(status.error_code, Success); - auto col = (milvus::segcore::Collection*)c_collection; - - int N = 10000; - auto dataset = DataGen(col->get_schema(), N); - int64_t ts_offset = 1000; - - int64_t offset; - PreInsert(segment, N, &offset); - - auto insert_data = serialize(dataset.raw_); - auto ins_res = Insert(segment, - offset, - N, - dataset.row_ids_.data(), - dataset.timestamps_.data(), - insert_data.data(), - insert_data.size()); - ASSERT_EQ(ins_res.error_code, Success); - - const char* raw_plan = R"(vector_anns: < - field_id: 100 - query_info: < - topk: 10 - round_decimal: 3 - metric_type: "IP" - search_params: "{\"nprobe\": 10,\"radius\": 10, \"range_filter\": 20}" - > - placeholder_tag: "$0" - >)"; - auto plan_str = translate_text_plan_to_binary_plan(raw_plan); - - int num_queries = 10; - auto blob = generate_query_data_float16(num_queries); - - void* plan = nullptr; - status = CreateSearchPlanByExpr( - c_collection, plan_str.data(), plan_str.size(), &plan); - ASSERT_EQ(status.error_code, Success); - - void* placeholderGroup = nullptr; - status = ParsePlaceholderGroup( - plan, blob.data(), blob.length(), &placeholderGroup); - ASSERT_EQ(status.error_code, Success); - - std::vector placeholderGroups; - placeholderGroups.push_back(placeholderGroup); - - CSearchResult search_result; - auto res = - CSearch(segment, plan, placeholderGroup, ts_offset, &search_result); - ASSERT_EQ(res.error_code, Success); - - DeleteSearchPlan(plan); - DeletePlaceholderGroup(placeholderGroup); - DeleteSearchResult(search_result); - DeleteCollection(c_collection); - DeleteSegment(segment); -} - -TEST(CApiTest, RANGE_SEARCH_WITH_RADIUS_AND_RANGE_FILTER_WHEN_IP_BFLOAT16) { - auto c_collection = - NewCollection(get_bfloat16_schema_config(), knowhere::metric::IP); - CSegmentInterface segment; - auto status = NewSegment(c_collection, Growing, -1, &segment, false); - ASSERT_EQ(status.error_code, Success); - auto col = (milvus::segcore::Collection*)c_collection; - - int N = 10000; - auto dataset = DataGen(col->get_schema(), N); - int64_t ts_offset = 1000; - - int64_t offset; - PreInsert(segment, N, &offset); - - auto insert_data = serialize(dataset.raw_); - auto ins_res = Insert(segment, - offset, - N, - dataset.row_ids_.data(), - dataset.timestamps_.data(), - insert_data.data(), - insert_data.size()); - ASSERT_EQ(ins_res.error_code, Success); - - const char* raw_plan = R"(vector_anns: < - field_id: 100 - query_info: < - topk: 10 - round_decimal: 3 - metric_type: "IP" - search_params: "{\"nprobe\": 10,\"radius\": 10, \"range_filter\": 20}" - > - placeholder_tag: "$0" - >)"; - auto plan_str = translate_text_plan_to_binary_plan(raw_plan); - - int num_queries = 10; - auto blob = generate_query_data_bfloat16(num_queries); - - void* plan = nullptr; - status = CreateSearchPlanByExpr( - c_collection, plan_str.data(), plan_str.size(), &plan); - ASSERT_EQ(status.error_code, Success); - - void* placeholderGroup = nullptr; - status = ParsePlaceholderGroup( - plan, blob.data(), blob.length(), &placeholderGroup); - ASSERT_EQ(status.error_code, Success); - - std::vector placeholderGroups; - placeholderGroups.push_back(placeholderGroup); - - CSearchResult search_result; - auto res = - CSearch(segment, plan, placeholderGroup, ts_offset, &search_result); - ASSERT_EQ(res.error_code, Success); - - DeleteSearchPlan(plan); - DeletePlaceholderGroup(placeholderGroup); - DeleteSearchResult(search_result); - DeleteCollection(c_collection); - DeleteSegment(segment); -} - TEST(CApiTest, IsLoadWithDisk) { ASSERT_TRUE(IsLoadWithDisk(INVERTED_INDEX_TYPE, 0)); } diff --git a/internal/core/unittest/test_c_stream_reduce.cpp b/internal/core/unittest/test_c_stream_reduce.cpp index 2fca0477a060a..372766eee404d 100644 --- a/internal/core/unittest/test_c_stream_reduce.cpp +++ b/internal/core/unittest/test_c_stream_reduce.cpp @@ -17,7 +17,7 @@ TEST(CApiTest, StreamReduce) { int N = 300; int topK = 100; int num_queries = 2; - auto collection = NewCollection(get_default_schema_config()); + auto collection = NewCollection(get_default_schema_config().c_str()); //1. set up segments CSegmentInterface segment_1; diff --git a/internal/core/unittest/test_float16.cpp b/internal/core/unittest/test_float16.cpp index 670855c5c330d..122adc5208b04 100644 --- a/internal/core/unittest/test_float16.cpp +++ b/internal/core/unittest/test_float16.cpp @@ -118,7 +118,8 @@ TEST(Float16, ExecWithoutPredicateFlat) { auto vec_ptr = dataset.get_col(vec_fid); auto num_queries = 5; - auto ph_group_raw = CreateFloat16PlaceholderGroup(num_queries, 32, 1024); + auto ph_group_raw = + CreatePlaceholderGroup(num_queries, 32, 1024); auto ph_group = ParsePlaceholderGroup(plan.get(), ph_group_raw.SerializeAsString()); Timestamp timestamp = 1000000; @@ -274,7 +275,8 @@ TEST(Float16, ExecWithPredicate) { auto plan = CreateSearchPlanByExpr(*schema, plan_str.data(), plan_str.size()); auto num_queries = 5; - auto ph_group_raw = CreateFloat16PlaceholderGroup(num_queries, 16, 1024); + auto ph_group_raw = + CreatePlaceholderGroup(num_queries, 16, 1024); auto ph_group = ParsePlaceholderGroup(plan.get(), ph_group_raw.SerializeAsString()); @@ -354,7 +356,8 @@ TEST(BFloat16, ExecWithoutPredicateFlat) { auto vec_ptr = dataset.get_col(vec_fid); auto num_queries = 5; - auto ph_group_raw = CreateBFloat16PlaceholderGroup(num_queries, 32, 1024); + auto ph_group_raw = + CreatePlaceholderGroup(num_queries, 32, 1024); auto ph_group = ParsePlaceholderGroup(plan.get(), ph_group_raw.SerializeAsString()); Timestamp timestamp = 1000000; @@ -510,7 +513,8 @@ TEST(BFloat16, ExecWithPredicate) { auto plan = CreateSearchPlanByExpr(*schema, plan_str.data(), plan_str.size()); auto num_queries = 5; - auto ph_group_raw = CreateBFloat16PlaceholderGroup(num_queries, 16, 1024); + auto ph_group_raw = + CreatePlaceholderGroup(num_queries, 16, 1024); auto ph_group = ParsePlaceholderGroup(plan.get(), ph_group_raw.SerializeAsString()); Timestamp timestamp = 1000000; diff --git a/internal/core/unittest/test_query.cpp b/internal/core/unittest/test_query.cpp index a9e4c80eb0ea3..2b1f0d5313c35 100644 --- a/internal/core/unittest/test_query.cpp +++ b/internal/core/unittest/test_query.cpp @@ -725,7 +725,7 @@ TEST(Query, ExecWithPredicateBinary) { auto plan = CreateSearchPlanByExpr(*schema, plan_str.data(), plan_str.size()); auto num_queries = 5; - auto ph_group_raw = CreateBinaryPlaceholderGroupFromBlob( + auto ph_group_raw = CreatePlaceholderGroupFromBlob( num_queries, 512, vec_ptr.data() + 1024 * 512 / 8); auto ph_group = ParsePlaceholderGroup(plan.get(), ph_group_raw.SerializeAsString()); diff --git a/internal/core/unittest/test_utils/DataGen.h b/internal/core/unittest/test_utils/DataGen.h index c49ee0d64e640..7f904f6e44659 100644 --- a/internal/core/unittest/test_utils/DataGen.h +++ b/internal/core/unittest/test_utils/DataGen.h @@ -743,26 +743,6 @@ DataGenForJsonArray(SchemaPtr schema, return res; } -inline auto -CreatePlaceholderGroup(int64_t num_queries, int dim, int64_t seed = 42) { - namespace ser = milvus::proto::common; - ser::PlaceholderGroup raw_group; - auto value = raw_group.add_placeholders(); - value->set_tag("$0"); - value->set_type(ser::PlaceholderType::FloatVector); - std::normal_distribution dis(0, 1); - std::default_random_engine e(seed); - for (int i = 0; i < num_queries; ++i) { - std::vector vec; - for (int d = 0; d < dim; ++d) { - vec.push_back(dis(e)); - } - // std::string line((char*)vec.data(), (char*)vec.data() + vec.size() * sizeof(float)); - value->add_values(vec.data(), vec.size() * sizeof(float)); - } - return raw_group; -} - inline auto CreatePlaceholderGroup(int64_t num_queries, int dim, @@ -782,148 +762,57 @@ CreatePlaceholderGroup(int64_t num_queries, return raw_group; } -inline auto -CreatePlaceholderGroupFromBlob(int64_t num_queries, int dim, const float* src) { - namespace ser = milvus::proto::common; - ser::PlaceholderGroup raw_group; - auto value = raw_group.add_placeholders(); - value->set_tag("$0"); - value->set_type(ser::PlaceholderType::FloatVector); - int64_t src_index = 0; - - for (int i = 0; i < num_queries; ++i) { - std::vector vec; - for (int d = 0; d < dim; ++d) { - vec.push_back(src[src_index++]); - } - // std::string line((char*)vec.data(), (char*)vec.data() + vec.size() * sizeof(float)); - value->add_values(vec.data(), vec.size() * sizeof(float)); - } - return raw_group; -} - -inline auto -CreateBinaryPlaceholderGroup(int64_t num_queries, - int64_t dim, - int64_t seed = 42) { - assert(dim % 8 == 0); - namespace ser = milvus::proto::common; - ser::PlaceholderGroup raw_group; - auto value = raw_group.add_placeholders(); - value->set_tag("$0"); - value->set_type(ser::PlaceholderType::BinaryVector); - std::default_random_engine e(seed); - for (int i = 0; i < num_queries; ++i) { - std::vector vec; - for (int d = 0; d < dim / 8; ++d) { - vec.push_back(e()); - } - // std::string line((char*)vec.data(), (char*)vec.data() + vec.size() * sizeof(float)); - value->add_values(vec.data(), vec.size()); +template +auto +CreatePlaceholderGroup(int64_t num_queries, int dim, int64_t seed = 42) { + if (std::is_same_v) { + assert(dim % 8 == 0); } - return raw_group; -} - -inline auto -CreateBinaryPlaceholderGroupFromBlob(int64_t num_queries, - int64_t dim, - const uint8_t* ptr) { - assert(dim % 8 == 0); namespace ser = milvus::proto::common; - ser::PlaceholderGroup raw_group; - auto value = raw_group.add_placeholders(); - value->set_tag("$0"); - value->set_type(ser::PlaceholderType::BinaryVector); - for (int i = 0; i < num_queries; ++i) { - std::vector vec; - for (int d = 0; d < dim / 8; ++d) { - vec.push_back(*ptr); - ++ptr; - } - // std::string line((char*)vec.data(), (char*)vec.data() + vec.size() * sizeof(float)); - value->add_values(vec.data(), vec.size()); - } - return raw_group; -} + GET_ELEM_TYPE_FOR_VECTOR_TRAIT -inline auto -CreateFloat16PlaceholderGroup(int64_t num_queries, - int64_t dim, - int64_t seed = 42) { - namespace ser = milvus::proto::common; ser::PlaceholderGroup raw_group; auto value = raw_group.add_placeholders(); value->set_tag("$0"); - value->set_type(ser::PlaceholderType::Float16Vector); + value->set_type(TraitType::placeholder_type); + // TODO caiyd: need update for Int8Vector std::normal_distribution dis(0, 1); std::default_random_engine e(seed); for (int i = 0; i < num_queries; ++i) { - std::vector vec; - for (int d = 0; d < dim; ++d) { - vec.push_back(float16(dis(e))); + std::vector vec; + for (int d = 0; d < dim / TraitType::dim_factor; ++d) { + if (std::is_same_v) { + vec.push_back(e()); + } else { + vec.push_back(elem_type(dis(e))); + } } - value->add_values(vec.data(), vec.size() * sizeof(float16)); + value->add_values(vec.data(), vec.size() * sizeof(elem_type)); } return raw_group; } +template inline auto -CreateFloat16PlaceholderGroupFromBlob(int64_t num_queries, - int64_t dim, - const float16* ptr) { - namespace ser = milvus::proto::common; - ser::PlaceholderGroup raw_group; - auto value = raw_group.add_placeholders(); - value->set_tag("$0"); - value->set_type(ser::PlaceholderType::Float16Vector); - for (int i = 0; i < num_queries; ++i) { - std::vector vec; - for (int d = 0; d < dim; ++d) { - vec.push_back(*ptr); - ++ptr; - } - value->add_values(vec.data(), vec.size() * sizeof(float16)); +CreatePlaceholderGroupFromBlob(int64_t num_queries, int dim, const void* src) { + if (std::is_same_v) { + assert(dim % 8 == 0); } - return raw_group; -} - -inline auto -CreateBFloat16PlaceholderGroup(int64_t num_queries, - int64_t dim, - int64_t seed = 42) { namespace ser = milvus::proto::common; - ser::PlaceholderGroup raw_group; - auto value = raw_group.add_placeholders(); - value->set_tag("$0"); - value->set_type(ser::PlaceholderType::BFloat16Vector); - std::normal_distribution dis(0, 1); - std::default_random_engine e(seed); - for (int i = 0; i < num_queries; ++i) { - std::vector vec; - for (int d = 0; d < dim; ++d) { - vec.push_back(bfloat16(dis(e))); - } - value->add_values(vec.data(), vec.size() * sizeof(bfloat16)); - } - return raw_group; -} + GET_ELEM_TYPE_FOR_VECTOR_TRAIT -inline auto -CreateBFloat16PlaceholderGroupFromBlob(int64_t num_queries, - int64_t dim, - const bfloat16* ptr) { - namespace ser = milvus::proto::common; ser::PlaceholderGroup raw_group; auto value = raw_group.add_placeholders(); value->set_tag("$0"); - value->set_type(ser::PlaceholderType::BFloat16Vector); + value->set_type(TraitType::placeholder_type); + int64_t src_index = 0; + for (int i = 0; i < num_queries; ++i) { - std::vector vec; - for (int d = 0; d < dim; ++d) { - vec.push_back(*ptr); - ++ptr; + std::vector vec; + for (int d = 0; d < dim / TraitType::dim_factor; ++d) { + vec.push_back(((elem_type*)src)[src_index++]); } - value->add_values(vec.data(), vec.size() * sizeof(bfloat16)); + value->add_values(vec.data(), vec.size() * sizeof(elem_type)); } return raw_group; } diff --git a/internal/core/unittest/test_utils/c_api_test_utils.h b/internal/core/unittest/test_utils/c_api_test_utils.h index b83d6e5640b92..d2d455008045e 100644 --- a/internal/core/unittest/test_utils/c_api_test_utils.h +++ b/internal/core/unittest/test_utils/c_api_test_utils.h @@ -23,6 +23,7 @@ #include "common/Types.h" #include "common/type_c.h" +#include "common/VectorTrait.h" #include "pb/plan.pb.h" #include "segcore/Collection.h" #include "segcore/reduce/Reduce.h" @@ -32,7 +33,6 @@ #include "futures/future_c.h" #include "DataGen.h" #include "PbHelper.h" -#include "c_api_test_utils.h" #include "indexbuilder_test_utils.h" using namespace milvus; @@ -66,26 +66,30 @@ generate_max_float_query_data(int all_nq, int max_float_nq) { return blob; } +template std::string generate_query_data(int nq) { namespace ser = milvus::proto::common; + GET_ELEM_TYPE_FOR_VECTOR_TRAIT + std::default_random_engine e(67); int dim = DIM; - std::normal_distribution dis(0.0, 1.0); + std::uniform_int_distribution dis(-128, 127); ser::PlaceholderGroup raw_group; auto value = raw_group.add_placeholders(); value->set_tag("$0"); - value->set_type(ser::PlaceholderType::FloatVector); + value->set_type(TraitType::placeholder_type); for (int i = 0; i < nq; ++i) { - std::vector vec; - for (int d = 0; d < dim; ++d) { - vec.push_back(dis(e)); + std::vector vec; + for (int d = 0; d < dim / TraitType::dim_factor; ++d) { + vec.push_back((elem_type)dis(e)); } - value->add_values(vec.data(), vec.size() * sizeof(float)); + value->add_values(vec.data(), vec.size() * sizeof(elem_type)); } auto blob = raw_group.SerializeAsString(); return blob; } + void CheckSearchResultDuplicate(const std::vector& results, int group_size = 1) { @@ -117,13 +121,14 @@ CheckSearchResultDuplicate(const std::vector& results, } } -const char* +template +const std::string get_default_schema_config() { - static std::string conf = R"(name: "default-collection" + auto fmt = boost::format(R"(name: "default-collection" fields: < fieldID: 100 name: "fakevec" - data_type: FloatVector + data_type: %1% type_params: < key: "dim" value: "16" @@ -138,9 +143,9 @@ get_default_schema_config() { name: "age" data_type: Int64 is_primary_key: true - >)"; - static std::string fake_conf = ""; - return conf.c_str(); + >)") % + (int(TraitType::data_type)); + return fmt.str(); } const char* From 3cd74037db73cc78e4ec239db1022711259fb431 Mon Sep 17 00:00:00 2001 From: sthuang <167743503+shaoting-huang@users.noreply.github.com> Date: Mon, 13 Jan 2025 10:02:57 +0800 Subject: [PATCH 19/34] fix: restore rbac with empty meta panic (#39141) related: https://github.com/milvus-io/milvus/issues/38985 Signed-off-by: shaoting-huang --- internal/metastore/kv/rootcoord/kv_catalog.go | 34 +++++++++---------- internal/proxy/impl.go | 3 ++ tests/integration/rbac/rbac_backup_test.go | 6 +++- 3 files changed, 25 insertions(+), 18 deletions(-) diff --git a/internal/metastore/kv/rootcoord/kv_catalog.go b/internal/metastore/kv/rootcoord/kv_catalog.go index b37a301ab8385..6063cdf9d21ef 100644 --- a/internal/metastore/kv/rootcoord/kv_catalog.go +++ b/internal/metastore/kv/rootcoord/kv_catalog.go @@ -1489,7 +1489,7 @@ func (kc *Catalog) RestoreRBAC(ctx context.Context, tenant string, meta *milvusp log.Ctx(ctx).Warn("failed to restore rbac, try to rollback", zap.Error(err)) // roll back role for _, role := range needRollbackRole { - err = kc.DropRole(ctx, tenant, role.Name) + err = kc.DropRole(ctx, tenant, role.GetName()) if err != nil { log.Ctx(ctx).Warn("failed to rollback roles after restore failed", zap.Error(err)) } @@ -1505,7 +1505,7 @@ func (kc *Catalog) RestoreRBAC(ctx context.Context, tenant string, meta *milvusp for _, user := range needRollbackUser { // roll back user - err = kc.DropCredential(ctx, user.User) + err = kc.DropCredential(ctx, user.GetUser()) if err != nil { log.Ctx(ctx).Warn("failed to rollback users after restore failed", zap.Error(err)) } @@ -1513,7 +1513,7 @@ func (kc *Catalog) RestoreRBAC(ctx context.Context, tenant string, meta *milvusp // roll back privilege group for _, group := range needRollbackPrivilegeGroups { - err = kc.DropPrivilegeGroup(ctx, group.GroupName) + err = kc.DropPrivilegeGroup(ctx, group.GetGroupName()) if err != nil { log.Ctx(ctx).Warn("failed to rollback privilege groups after restore failed", zap.Error(err)) } @@ -1527,7 +1527,7 @@ func (kc *Catalog) RestoreRBAC(ctx context.Context, tenant string, meta *milvusp return err } existRoleMap := lo.SliceToMap(existRoles, func(entity *milvuspb.RoleResult) (string, struct{}) { return entity.GetRole().GetName(), struct{}{} }) - for _, role := range meta.Roles { + for _, role := range meta.GetRoles() { if _, ok := existRoleMap[role.GetName()]; ok { log.Ctx(ctx).Warn("failed to restore, role already exists", zap.String("role", role.GetName())) err = errors.Newf("role [%s] already exists", role.GetName()) @@ -1545,11 +1545,11 @@ func (kc *Catalog) RestoreRBAC(ctx context.Context, tenant string, meta *milvusp if err != nil { return err } - existPrivGroupMap := lo.SliceToMap(existPrivGroups, func(entity *milvuspb.PrivilegeGroupInfo) (string, struct{}) { return entity.GroupName, struct{}{} }) - for _, group := range meta.PrivilegeGroups { - if _, ok := existPrivGroupMap[group.GroupName]; ok { - log.Ctx(ctx).Warn("failed to restore, privilege group already exists", zap.String("group", group.GroupName)) - err = errors.Newf("privilege group [%s] already exists", group.GroupName) + existPrivGroupMap := lo.SliceToMap(existPrivGroups, func(entity *milvuspb.PrivilegeGroupInfo) (string, struct{}) { return entity.GetGroupName(), struct{}{} }) + for _, group := range meta.GetPrivilegeGroups() { + if _, ok := existPrivGroupMap[group.GetGroupName()]; ok { + log.Ctx(ctx).Warn("failed to restore, privilege group already exists", zap.String("group", group.GetGroupName())) + err = errors.Newf("privilege group [%s] already exists", group.GetGroupName()) return err } err = kc.SavePrivilegeGroup(ctx, group) @@ -1564,9 +1564,9 @@ func (kc *Catalog) RestoreRBAC(ctx context.Context, tenant string, meta *milvusp if err != nil { return err } - existPrivGroupMap = lo.SliceToMap(existPrivGroups, func(entity *milvuspb.PrivilegeGroupInfo) (string, struct{}) { return entity.GroupName, struct{}{} }) - for _, grant := range meta.Grants { - privName := grant.Grantor.Privilege.Name + existPrivGroupMap = lo.SliceToMap(existPrivGroups, func(entity *milvuspb.PrivilegeGroupInfo) (string, struct{}) { return entity.GetGroupName(), struct{}{} }) + for _, grant := range meta.GetGrants() { + privName := grant.GetGrantor().GetPrivilege().GetName() if util.IsPrivilegeNameDefined(privName) { grant.Grantor.Privilege.Name = util.PrivilegeNameForMetastore(privName) } else if _, ok := existPrivGroupMap[privName]; ok { @@ -1589,7 +1589,7 @@ func (kc *Catalog) RestoreRBAC(ctx context.Context, tenant string, meta *milvusp return err } existUserMap := lo.SliceToMap(existUser, func(entity *milvuspb.UserResult) (string, struct{}) { return entity.GetUser().GetName(), struct{}{} }) - for _, user := range meta.Users { + for _, user := range meta.GetUsers() { if _, ok := existUserMap[user.GetUser()]; ok { log.Ctx(ctx).Info("failed to restore, user already exists", zap.String("user", user.GetUser())) err = errors.Newf("user [%s] already exists", user.GetUser()) @@ -1597,8 +1597,8 @@ func (kc *Catalog) RestoreRBAC(ctx context.Context, tenant string, meta *milvusp } // restore user err = kc.CreateCredential(ctx, &model.Credential{ - Username: user.User, - EncryptedPassword: user.Password, + Username: user.GetUser(), + EncryptedPassword: user.GetPassword(), }) if err != nil { return err @@ -1607,9 +1607,9 @@ func (kc *Catalog) RestoreRBAC(ctx context.Context, tenant string, meta *milvusp // restore user role mapping entity := &milvuspb.UserEntity{ - Name: user.User, + Name: user.GetUser(), } - for _, role := range user.Roles { + for _, role := range user.GetRoles() { err = kc.AlterUserRole(ctx, tenant, entity, role, milvuspb.OperateUserRoleType_AddUserToRole) if err != nil { return err diff --git a/internal/proxy/impl.go b/internal/proxy/impl.go index e4734c2cf3c0a..017fa52614377 100644 --- a/internal/proxy/impl.go +++ b/internal/proxy/impl.go @@ -5656,6 +5656,9 @@ func (node *Proxy) RestoreRBAC(ctx context.Context, req *milvuspb.RestoreRBACMet if err := merr.CheckHealthy(node.GetStateCode()); err != nil { return merr.Status(err), nil } + if req.RBACMeta == nil { + return merr.Success(), nil + } result, err := node.rootCoord.RestoreRBAC(ctx, req) if err != nil { diff --git a/tests/integration/rbac/rbac_backup_test.go b/tests/integration/rbac/rbac_backup_test.go index 49e61ee4dcd5d..ce526321da654 100644 --- a/tests/integration/rbac/rbac_backup_test.go +++ b/tests/integration/rbac/rbac_backup_test.go @@ -154,8 +154,12 @@ func (s *RBACBackupTestSuite) TestBackup() { s.Equal(groupName, backupRBACResp.GetRBACMeta().PrivilegeGroups[0].GroupName) s.Equal(2, len(backupRBACResp.GetRBACMeta().PrivilegeGroups[0].Privileges)) + restoreRBACResp, err := s.Cluster.Proxy.RestoreRBAC(ctx, &milvuspb.RestoreRBACMetaRequest{}) + s.NoError(err) + s.True(merr.Ok(restoreRBACResp)) + // test restore, expect to failed due to role/user already exist - restoreRBACResp, err := s.Cluster.Proxy.RestoreRBAC(ctx, &milvuspb.RestoreRBACMetaRequest{ + restoreRBACResp, err = s.Cluster.Proxy.RestoreRBAC(ctx, &milvuspb.RestoreRBACMetaRequest{ RBACMeta: backupRBACResp.GetRBACMeta(), }) s.NoError(err) From 640a49ffb67ec89bc8dd0f1fd5e12cb7c2ff0b00 Mon Sep 17 00:00:00 2001 From: Buqian Zheng Date: Mon, 13 Jan 2025 10:34:57 +0800 Subject: [PATCH 20/34] fix: fix chunk cache madvise when sparse raw data is mmaped (#39145) instead of marking as not supported, `ChunkedSparseFloatColumn::DataByteSize` can simply use the impl of super class. issue: https://github.com/milvus-io/milvus/issues/39158 Signed-off-by: Buqian Zheng --- internal/core/src/mmap/ChunkedColumn.h | 9 --------- 1 file changed, 9 deletions(-) diff --git a/internal/core/src/mmap/ChunkedColumn.h b/internal/core/src/mmap/ChunkedColumn.h index a75f648460923..2195fadf20aaa 100644 --- a/internal/core/src/mmap/ChunkedColumn.h +++ b/internal/core/src/mmap/ChunkedColumn.h @@ -282,14 +282,6 @@ class ChunkedSparseFloatColumn : public ChunkedColumnBase { std::dynamic_pointer_cast(chunk)->Dim()); } - // This is used to advice mmap prefetch, we don't currently support mmap for - // sparse float vector thus not implemented for now. - size_t - DataByteSize() const override { - PanicInfo(ErrorCode::Unsupported, - "ByteSize not supported for sparse float column"); - } - SpanBase Span(int64_t chunk_id) const override { PanicInfo(ErrorCode::Unsupported, @@ -303,7 +295,6 @@ class ChunkedSparseFloatColumn : public ChunkedColumnBase { private: int64_t dim_ = 0; - std::vector> vec_; }; template From cc5d59392ad1e31a5d82c7ce21c428f25027c19e Mon Sep 17 00:00:00 2001 From: wei liu Date: Mon, 13 Jan 2025 11:21:06 +0800 Subject: [PATCH 21/34] fix: channel unbalance during stopping balance progress (#38971) issue: #38970 cause the stopping balance channel still use the row_count_based policy, which may causes channel unbalance in multi-collection case. This PR impl a score based stopping balance channel policy. Signed-off-by: Wei Liu --- internal/querycoordv2/balance/balance.go | 11 +- .../balance/rowcount_based_balancer.go | 2 +- .../balance/score_based_balancer.go | 30 +++-- .../balance/score_based_balancer_test.go | 125 ++++++++++++++++++ 4 files changed, 155 insertions(+), 13 deletions(-) diff --git a/internal/querycoordv2/balance/balance.go b/internal/querycoordv2/balance/balance.go index e3fd1e5121384..ff8103e3b45d9 100644 --- a/internal/querycoordv2/balance/balance.go +++ b/internal/querycoordv2/balance/balance.go @@ -46,10 +46,13 @@ func (segPlan *SegmentAssignPlan) String() string { } type ChannelAssignPlan struct { - Channel *meta.DmChannel - Replica *meta.Replica - From int64 - To int64 + Channel *meta.DmChannel + Replica *meta.Replica + From int64 + To int64 + FromScore int64 + ToScore int64 + ChannelScore int64 } func (chanPlan *ChannelAssignPlan) String() string { diff --git a/internal/querycoordv2/balance/rowcount_based_balancer.go b/internal/querycoordv2/balance/rowcount_based_balancer.go index d69a676c46c74..d5be949695489 100644 --- a/internal/querycoordv2/balance/rowcount_based_balancer.go +++ b/internal/querycoordv2/balance/rowcount_based_balancer.go @@ -393,7 +393,7 @@ func newNodeItem(currentScore int, nodeID int64) nodeItem { func (b *nodeItem) getPriority() int { // if node lacks more score between assignedScore and currentScore, then higher priority - return int(b.currentScore - b.assignedScore) + return int(math.Ceil(b.currentScore - b.assignedScore)) } func (b *nodeItem) setPriority(priority int) { diff --git a/internal/querycoordv2/balance/score_based_balancer.go b/internal/querycoordv2/balance/score_based_balancer.go index f7842a4ce4ce3..8e19c8921de3d 100644 --- a/internal/querycoordv2/balance/score_based_balancer.go +++ b/internal/querycoordv2/balance/score_based_balancer.go @@ -191,19 +191,19 @@ func (b *ScoreBasedBalancer) assignChannel(br *balanceReport, collectionID int64 } from := int64(-1) - // fromScore := int64(0) + fromScore := int64(0) if sourceNode != nil { from = sourceNode.nodeID - // fromScore = int64(sourceNode.getPriority()) + fromScore = int64(sourceNode.getPriority()) } plan := ChannelAssignPlan{ - From: from, - To: targetNode.nodeID, - Channel: ch, - // FromScore: fromScore, - // ToScore: int64(targetNode.getPriority()), - // SegmentScore: int64(scoreChanges), + From: from, + To: targetNode.nodeID, + Channel: ch, + FromScore: fromScore, + ToScore: int64(targetNode.getPriority()), + ChannelScore: int64(scoreChanges), } br.AddRecord(StrRecordf("add segment plan %s", plan)) plans = append(plans, plan) @@ -487,6 +487,20 @@ func (b *ScoreBasedBalancer) BalanceReplica(ctx context.Context, replica *meta.R return segmentPlans, channelPlans } +func (b *ScoreBasedBalancer) genStoppingChannelPlan(ctx context.Context, replica *meta.Replica, rwNodes []int64, roNodes []int64) []ChannelAssignPlan { + channelPlans := make([]ChannelAssignPlan, 0) + for _, nodeID := range roNodes { + dmChannels := b.dist.ChannelDistManager.GetByCollectionAndFilter(replica.GetCollectionID(), meta.WithNodeID2Channel(nodeID)) + plans := b.AssignChannel(ctx, replica.GetCollectionID(), dmChannels, rwNodes, false) + for i := range plans { + plans[i].From = nodeID + plans[i].Replica = replica + } + channelPlans = append(channelPlans, plans...) + } + return channelPlans +} + func (b *ScoreBasedBalancer) genStoppingSegmentPlan(ctx context.Context, replica *meta.Replica, onlineNodes []int64, offlineNodes []int64) []SegmentAssignPlan { segmentPlans := make([]SegmentAssignPlan, 0) for _, nodeID := range offlineNodes { diff --git a/internal/querycoordv2/balance/score_based_balancer_test.go b/internal/querycoordv2/balance/score_based_balancer_test.go index 3bf70e7894407..f24d8394b2510 100644 --- a/internal/querycoordv2/balance/score_based_balancer_test.go +++ b/internal/querycoordv2/balance/score_based_balancer_test.go @@ -23,6 +23,7 @@ import ( "github.com/samber/lo" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/suite" + "go.uber.org/atomic" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/metastore/kv/querycoord" @@ -1470,3 +1471,127 @@ func (suite *ScoreBasedBalancerTestSuite) TestBalanceChannelOnChannelExclusive() _, channelPlans = suite.getCollectionBalancePlans(balancer, 3) suite.Len(channelPlans, 2) } + +func (suite *ScoreBasedBalancerTestSuite) TestBalanceChannelOnStoppingNode() { + ctx := context.Background() + balancer := suite.balancer + + // mock 10 collections with each collection has 1 channel + collectionNum := 10 + channelNum := 1 + for i := 1; i <= collectionNum; i++ { + collectionID := int64(i) + collection := utils.CreateTestCollection(collectionID, int32(1)) + collection.LoadPercentage = 100 + collection.Status = querypb.LoadStatus_Loaded + balancer.meta.CollectionManager.PutCollection(ctx, collection) + balancer.meta.CollectionManager.PutPartition(ctx, utils.CreateTestPartition(collectionID, collectionID)) + balancer.meta.ReplicaManager.Spawn(ctx, collectionID, map[string]int{meta.DefaultResourceGroupName: 1}, nil) + + channels := make([]*datapb.VchannelInfo, channelNum) + for i := 0; i < channelNum; i++ { + channels[i] = &datapb.VchannelInfo{CollectionID: collectionID, ChannelName: fmt.Sprintf("channel-%d-%d", collectionID, i)} + } + suite.broker.EXPECT().GetRecoveryInfoV2(mock.Anything, collectionID).Return( + channels, nil, nil) + suite.broker.EXPECT().GetPartitions(mock.Anything, collectionID).Return([]int64{collectionID}, nil).Maybe() + balancer.targetMgr.UpdateCollectionNextTarget(ctx, collectionID) + balancer.targetMgr.UpdateCollectionCurrentTarget(ctx, collectionID) + } + + // mock querynode-1 to node manager + nodeInfo := session.NewNodeInfo(session.ImmutableNodeInfo{ + NodeID: 1, + Address: "127.0.0.1:0", + Hostname: "localhost", + Version: common.Version, + }) + nodeInfo.SetState(session.NodeStateNormal) + suite.balancer.nodeManager.Add(nodeInfo) + suite.balancer.meta.ResourceManager.HandleNodeUp(ctx, 1) + utils.RecoverAllCollection(balancer.meta) + + // mock channel distribution + channelDist := make([]*meta.DmChannel, 0) + for i := 1; i <= collectionNum; i++ { + collectionID := int64(i) + for i := 0; i < channelNum; i++ { + channelDist = append(channelDist, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{CollectionID: collectionID, ChannelName: fmt.Sprintf("channel-%d-%d", collectionID, i)}, Node: 1, + }) + } + } + balancer.dist.ChannelDistManager.Update(1, channelDist...) + + // assert balance channel won't happens on 1 querynode + ret := make([]ChannelAssignPlan, 0) + for i := 1; i <= collectionNum; i++ { + collectionID := int64(i) + _, channelPlans := suite.getCollectionBalancePlans(balancer, collectionID) + ret = append(ret, channelPlans...) + } + suite.Len(ret, 0) + + // mock querynode-2 and querynode-3 to node manager + nodeInfo2 := session.NewNodeInfo(session.ImmutableNodeInfo{ + NodeID: 2, + Address: "127.0.0.1:0", + Hostname: "localhost", + Version: common.Version, + }) + suite.balancer.nodeManager.Add(nodeInfo2) + suite.balancer.meta.ResourceManager.HandleNodeUp(ctx, 2) + // mock querynode-2 and querynode-3 to node manager + nodeInfo3 := session.NewNodeInfo(session.ImmutableNodeInfo{ + NodeID: 3, + Address: "127.0.0.1:0", + Hostname: "localhost", + Version: common.Version, + }) + suite.balancer.nodeManager.Add(nodeInfo3) + suite.balancer.meta.ResourceManager.HandleNodeUp(ctx, 3) + utils.RecoverAllCollection(balancer.meta) + // mock querynode-1 to stopping, trigger stopping balance, expect to generate 10 balance channel task, and 5 for node-2, 5 for node-3 + nodeInfo.SetState(session.NodeStateStopping) + suite.balancer.meta.ResourceManager.HandleNodeDown(ctx, 1) + utils.RecoverAllCollection(balancer.meta) + + node2Counter := atomic.NewInt32(0) + node3Counter := atomic.NewInt32(0) + + suite.mockScheduler.ExpectedCalls = nil + suite.mockScheduler.EXPECT().GetSegmentTaskDelta(mock.Anything, mock.Anything).Return(0).Maybe() + suite.mockScheduler.EXPECT().GetChannelTaskDelta(mock.Anything, mock.Anything).RunAndReturn(func(nodeID, collection int64) int { + if collection == -1 { + if nodeID == 2 { + return int(node2Counter.Load()) + } + + if nodeID == 3 { + return int(node3Counter.Load()) + } + } + return 0 + }) + suite.mockScheduler.EXPECT().GetSegmentTaskNum(mock.Anything, mock.Anything).Return(0).Maybe() + suite.mockScheduler.EXPECT().GetChannelTaskNum(mock.Anything, mock.Anything).Return(0).Maybe() + + for i := 1; i <= collectionNum; i++ { + collectionID := int64(i) + _, channelPlans := suite.getCollectionBalancePlans(balancer, collectionID) + suite.Len(channelPlans, 1) + if channelPlans[0].To == 2 { + node2Counter.Inc() + } + + if channelPlans[0].To == 3 { + node3Counter.Inc() + } + + if i%2 == 0 { + suite.Equal(node2Counter.Load(), node3Counter.Load()) + } + } + suite.Equal(node2Counter.Load(), int32(5)) + suite.Equal(node3Counter.Load(), int32(5)) +} From 5f94954bb45e86e4c869b2d5cdb93cf24e234331 Mon Sep 17 00:00:00 2001 From: Zhen Ye Date: Mon, 13 Jan 2025 11:23:04 +0800 Subject: [PATCH 22/34] fix: data race when accessing field_ when retrieving (#39151) issue: #39148 Signed-off-by: chyezh --- internal/core/src/segcore/SegmentSealedImpl.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/internal/core/src/segcore/SegmentSealedImpl.cpp b/internal/core/src/segcore/SegmentSealedImpl.cpp index 1944ca1086337..4ccf68267b0b4 100644 --- a/internal/core/src/segcore/SegmentSealedImpl.cpp +++ b/internal/core/src/segcore/SegmentSealedImpl.cpp @@ -1591,7 +1591,11 @@ SegmentSealedImpl::bulk_subscript( return fill_with_empty(field_id, 0); } - auto column = fields_.at(field_id); + std::shared_ptr column; + { + std::shared_lock lck(mutex_); + column = fields_.at(field_id); + } auto ret = fill_with_empty(field_id, count); if (column->IsNullable()) { auto dst = ret->mutable_valid_data()->mutable_data(); From 5c5948cb70ec500f8bf5b7af2cb47eee891c9202 Mon Sep 17 00:00:00 2001 From: sthuang <167743503+shaoting-huang@users.noreply.github.com> Date: Mon, 13 Jan 2025 11:24:59 +0800 Subject: [PATCH 23/34] fix: rbac custom group privilege level check (#39164) related: https://github.com/milvus-io/milvus/issues/39086 Signed-off-by: shaoting-huang --- internal/proxy/impl.go | 4 - internal/proxy/util.go | 25 --- internal/rootcoord/meta_table.go | 4 +- internal/rootcoord/rbac_task.go | 12 +- internal/rootcoord/root_coord.go | 46 ++++- pkg/util/constant.go | 119 +++++++++++++ pkg/util/paramtable/rbac_config_test.go | 24 ++- pkg/util/paramtable/rbac_param.go | 160 +++++------------- .../integration/rbac/privilege_group_test.go | 3 +- tests/integration/rbac/rbac_backup_test.go | 19 ++- tests/python_client/testcases/test_utility.py | 4 +- 11 files changed, 235 insertions(+), 185 deletions(-) diff --git a/internal/proxy/impl.go b/internal/proxy/impl.go index 017fa52614377..fa8b64fff4d14 100644 --- a/internal/proxy/impl.go +++ b/internal/proxy/impl.go @@ -5439,10 +5439,6 @@ func (node *Proxy) validateOperatePrivilegeV2Params(req *milvuspb.OperatePrivile if err := ValidatePrivilege(req.Grantor.Privilege.Name); err != nil { return err } - // validate built-in privilege group params - if err := ValidateBuiltInPrivilegeGroup(req.Grantor.Privilege.Name, req.DbName, req.CollectionName); err != nil { - return err - } if req.Type != milvuspb.OperatePrivilegeType_Grant && req.Type != milvuspb.OperatePrivilegeType_Revoke { return merr.WrapErrParameterInvalidMsg("the type in the request not grant or revoke") } diff --git a/internal/proxy/util.go b/internal/proxy/util.go index 16e01cae8b7f1..057b58f2a01b3 100644 --- a/internal/proxy/util.go +++ b/internal/proxy/util.go @@ -1120,31 +1120,6 @@ func ValidatePrivilege(entity string) error { return validateName(entity, "Privilege") } -func ValidateBuiltInPrivilegeGroup(entity string, dbName string, collectionName string) error { - if !util.IsBuiltinPrivilegeGroup(entity) { - return nil - } - switch { - case strings.HasPrefix(entity, milvuspb.PrivilegeLevel_Cluster.String()): - if !util.IsAnyWord(dbName) || !util.IsAnyWord(collectionName) { - return merr.WrapErrParameterInvalidMsg("dbName and collectionName should be * for the cluster level privilege: %s", entity) - } - return nil - case strings.HasPrefix(entity, milvuspb.PrivilegeLevel_Database.String()): - if collectionName != "" && collectionName != util.AnyWord { - return merr.WrapErrParameterInvalidMsg("collectionName should be * for the database level privilege: %s", entity) - } - return nil - case strings.HasPrefix(entity, milvuspb.PrivilegeLevel_Collection.String()): - if util.IsAnyWord(dbName) && !util.IsAnyWord(collectionName) && collectionName != "" { - return merr.WrapErrParameterInvalidMsg("please specify database name for the collection level privilege: %s", entity) - } - return nil - default: - return nil - } -} - func GetCurUserFromContext(ctx context.Context) (string, error) { return contextutil.GetCurUserFromContext(ctx) } diff --git a/internal/rootcoord/meta_table.go b/internal/rootcoord/meta_table.go index 1de392ff1f792..02e91a4ac689c 100644 --- a/internal/rootcoord/meta_table.go +++ b/internal/rootcoord/meta_table.go @@ -1525,7 +1525,7 @@ func (mt *MetaTable) RestoreRBAC(ctx context.Context, tenant string, meta *milvu return mt.catalog.RestoreRBAC(ctx, tenant, meta) } -// check if the privielge group name is defined by users +// check if the privilege group name is defined by users func (mt *MetaTable) IsCustomPrivilegeGroup(ctx context.Context, groupName string) (bool, error) { privGroups, err := mt.catalog.ListPrivilegeGroups(ctx) if err != nil { @@ -1641,7 +1641,7 @@ func (mt *MetaTable) OperatePrivilegeGroup(ctx context.Context, groupName string if group.GroupName == p.Name { privileges = append(privileges, group.Privileges...) } else { - return merr.WrapErrParameterInvalidMsg("there is no privilege name or privielge group name [%s] defined in system to operate", p.Name) + return merr.WrapErrParameterInvalidMsg("there is no privilege name or privilege group name [%s] defined in system to operate", p.Name) } } } diff --git a/internal/rootcoord/rbac_task.go b/internal/rootcoord/rbac_task.go index 8d826cc62afdd..50cb42f965ec8 100644 --- a/internal/rootcoord/rbac_task.go +++ b/internal/rootcoord/rbac_task.go @@ -179,7 +179,7 @@ func executeOperatePrivilegeTaskSteps(ctx context.Context, core *Core, in *milvu } grants := []*milvuspb.GrantEntity{in.Entity} - allGroups, err := core.getPrivilegeGroups(ctx) + allGroups, err := core.getDefaultAndCustomPrivilegeGroups(ctx) if err != nil { return nil, err } @@ -275,12 +275,12 @@ func executeOperatePrivilegeGroupTaskSteps(ctx context.Context, core *Core, in * return p.Name }) - // check if privileges are the same object type - objectTypes := lo.SliceToMap(newPrivs, func(p *milvuspb.PrivilegeEntity) (string, struct{}) { - return util.GetObjectType(p.Name), struct{}{} + // check if privileges are the same privilege level + privilegeLevels := lo.SliceToMap(newPrivs, func(p *milvuspb.PrivilegeEntity) (string, struct{}) { + return util.GetPrivilegeLevel(p.Name), struct{}{} }) - if len(objectTypes) > 1 { - return nil, errors.New("privileges are not the same object type") + if len(privilegeLevels) > 1 { + return nil, errors.New("privileges are not the same privilege level") } case milvuspb.OperatePrivilegeGroupType_RemovePrivilegesFromGroup: newPrivs, _ := lo.Difference(v, in.Privileges) diff --git a/internal/rootcoord/root_coord.go b/internal/rootcoord/root_coord.go index 7839e358da477..16e319cb8cf62 100644 --- a/internal/rootcoord/root_coord.go +++ b/internal/rootcoord/root_coord.go @@ -2595,6 +2595,10 @@ func (c *Core) OperatePrivilege(ctx context.Context, in *milvuspb.OperatePrivile ctxLog.Error("", zap.Error(err)) return merr.StatusWithErrorCode(err, commonpb.ErrorCode_OperatePrivilegeFailure), nil } + if err := c.validatePrivilegeGroupParams(ctx, privName, in.Entity.DbName, in.Entity.ObjectName); err != nil { + ctxLog.Error("", zap.Error(err)) + return merr.StatusWithErrorCode(err, commonpb.ErrorCode_OperatePrivilegeFailure), nil + } // set up object type for metastore, to be compatible with v1 version in.Entity.Object.Name = util.GetObjectType(privName) default: @@ -2656,6 +2660,42 @@ func (c *Core) operatePrivilegeCommonCheck(ctx context.Context, in *milvuspb.Ope return nil } +func (c *Core) validatePrivilegeGroupParams(ctx context.Context, entity string, dbName string, collectionName string) error { + allGroups, err := c.getDefaultAndCustomPrivilegeGroups(ctx) + if err != nil { + return err + } + groups := lo.SliceToMap(allGroups, func(group *milvuspb.PrivilegeGroupInfo) (string, []*milvuspb.PrivilegeEntity) { + return group.GroupName, group.Privileges + }) + privs, exists := groups[entity] + if !exists || len(privs) == 0 { + // it is a privilege, no need to check with other params + return nil + } + // since all privileges are same level in a group, just check the first privilege + level := util.GetPrivilegeLevel(privs[0].GetName()) + switch level { + case milvuspb.PrivilegeLevel_Cluster.String(): + if !util.IsAnyWord(dbName) || !util.IsAnyWord(collectionName) { + return merr.WrapErrParameterInvalidMsg("dbName and collectionName should be * for the cluster level privilege: %s", entity) + } + return nil + case milvuspb.PrivilegeLevel_Database.String(): + if collectionName != "" && collectionName != util.AnyWord { + return merr.WrapErrParameterInvalidMsg("collectionName should be * for the database level privilege: %s", entity) + } + return nil + case milvuspb.PrivilegeLevel_Collection.String(): + if util.IsAnyWord(dbName) && !util.IsAnyWord(collectionName) && collectionName != "" { + return merr.WrapErrParameterInvalidMsg("please specify database name for the collection level privilege: %s", entity) + } + return nil + default: + return errors.New("not found the privilege level") + } +} + func (c *Core) getMetastorePrivilegeName(ctx context.Context, privName string) (string, error) { // if it is built-in privilege, return the privilege name directly if util.IsPrivilegeNameDefined(privName) { @@ -2757,7 +2797,7 @@ func (c *Core) ListPolicy(ctx context.Context, in *internalpb.ListPolicyRequest) }, nil } // expand privilege groups and turn to policies - allGroups, err := c.getPrivilegeGroups(ctx) + allGroups, err := c.getDefaultAndCustomPrivilegeGroups(ctx) if err != nil { errMsg := "fail to get privilege groups" ctxLog.Warn(errMsg, zap.Error(err)) @@ -3131,8 +3171,8 @@ func (c *Core) expandPrivilegeGroups(ctx context.Context, grants []*milvuspb.Gra }), nil } -// getPrivilegeGroups returns default privilege groups and user-defined privilege groups. -func (c *Core) getPrivilegeGroups(ctx context.Context) ([]*milvuspb.PrivilegeGroupInfo, error) { +// getDefaultAndCustomPrivilegeGroups returns default privilege groups and user-defined privilege groups. +func (c *Core) getDefaultAndCustomPrivilegeGroups(ctx context.Context) ([]*milvuspb.PrivilegeGroupInfo, error) { allGroups, err := c.meta.ListPrivilegeGroups(ctx) allGroups = append(allGroups, Params.RbacConfig.GetDefaultPrivilegeGroups()...) if err != nil { diff --git a/pkg/util/constant.go b/pkg/util/constant.go index d1ca65d396a43..70100bfbf17b6 100644 --- a/pkg/util/constant.go +++ b/pkg/util/constant.go @@ -22,6 +22,7 @@ import ( "github.com/samber/lo" "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/pkg/common" "github.com/milvus-io/milvus/pkg/util/typeutil" ) @@ -292,6 +293,124 @@ var ( } ) +// rbac v2 uses privilege level to group privileges rather than object type +var ( + CollectionReadOnlyPrivileges = ConvertPrivileges([]string{ + commonpb.ObjectPrivilege_PrivilegeQuery.String(), + commonpb.ObjectPrivilege_PrivilegeSearch.String(), + commonpb.ObjectPrivilege_PrivilegeIndexDetail.String(), + commonpb.ObjectPrivilege_PrivilegeGetFlushState.String(), + commonpb.ObjectPrivilege_PrivilegeGetLoadState.String(), + commonpb.ObjectPrivilege_PrivilegeGetLoadingProgress.String(), + commonpb.ObjectPrivilege_PrivilegeHasPartition.String(), + commonpb.ObjectPrivilege_PrivilegeShowPartitions.String(), + commonpb.ObjectPrivilege_PrivilegeDescribeCollection.String(), + commonpb.ObjectPrivilege_PrivilegeDescribeAlias.String(), + commonpb.ObjectPrivilege_PrivilegeGetStatistics.String(), + commonpb.ObjectPrivilege_PrivilegeListAliases.String(), + }) + + CollectionReadWritePrivileges = append(CollectionReadOnlyPrivileges, + ConvertPrivileges([]string{ + commonpb.ObjectPrivilege_PrivilegeLoad.String(), + commonpb.ObjectPrivilege_PrivilegeRelease.String(), + commonpb.ObjectPrivilege_PrivilegeInsert.String(), + commonpb.ObjectPrivilege_PrivilegeDelete.String(), + commonpb.ObjectPrivilege_PrivilegeUpsert.String(), + commonpb.ObjectPrivilege_PrivilegeImport.String(), + commonpb.ObjectPrivilege_PrivilegeFlush.String(), + commonpb.ObjectPrivilege_PrivilegeCompaction.String(), + commonpb.ObjectPrivilege_PrivilegeLoadBalance.String(), + commonpb.ObjectPrivilege_PrivilegeCreateIndex.String(), + commonpb.ObjectPrivilege_PrivilegeDropIndex.String(), + commonpb.ObjectPrivilege_PrivilegeCreatePartition.String(), + commonpb.ObjectPrivilege_PrivilegeDropPartition.String(), + })..., + ) + + CollectionAdminPrivileges = append(CollectionReadWritePrivileges, + ConvertPrivileges([]string{ + commonpb.ObjectPrivilege_PrivilegeCreateAlias.String(), + commonpb.ObjectPrivilege_PrivilegeDropAlias.String(), + })..., + ) + + DatabaseReadOnlyPrivileges = ConvertPrivileges([]string{ + commonpb.ObjectPrivilege_PrivilegeShowCollections.String(), + commonpb.ObjectPrivilege_PrivilegeDescribeDatabase.String(), + }) + + DatabaseReadWritePrivileges = append(DatabaseReadOnlyPrivileges, + ConvertPrivileges([]string{ + commonpb.ObjectPrivilege_PrivilegeAlterDatabase.String(), + })..., + ) + + DatabaseAdminPrivileges = append(DatabaseReadWritePrivileges, + ConvertPrivileges([]string{ + commonpb.ObjectPrivilege_PrivilegeCreateCollection.String(), + commonpb.ObjectPrivilege_PrivilegeDropCollection.String(), + })..., + ) + + ClusterReadOnlyPrivileges = ConvertPrivileges([]string{ + commonpb.ObjectPrivilege_PrivilegeListDatabases.String(), + commonpb.ObjectPrivilege_PrivilegeSelectOwnership.String(), + commonpb.ObjectPrivilege_PrivilegeSelectUser.String(), + commonpb.ObjectPrivilege_PrivilegeDescribeResourceGroup.String(), + commonpb.ObjectPrivilege_PrivilegeListResourceGroups.String(), + commonpb.ObjectPrivilege_PrivilegeListPrivilegeGroups.String(), + }) + + ClusterReadWritePrivileges = append(ClusterReadOnlyPrivileges, + ConvertPrivileges([]string{ + commonpb.ObjectPrivilege_PrivilegeFlushAll.String(), + commonpb.ObjectPrivilege_PrivilegeTransferNode.String(), + commonpb.ObjectPrivilege_PrivilegeTransferReplica.String(), + commonpb.ObjectPrivilege_PrivilegeUpdateResourceGroups.String(), + })..., + ) + + ClusterAdminPrivileges = append(ClusterReadWritePrivileges, + ConvertPrivileges([]string{ + commonpb.ObjectPrivilege_PrivilegeBackupRBAC.String(), + commonpb.ObjectPrivilege_PrivilegeRestoreRBAC.String(), + commonpb.ObjectPrivilege_PrivilegeCreateDatabase.String(), + commonpb.ObjectPrivilege_PrivilegeDropDatabase.String(), + commonpb.ObjectPrivilege_PrivilegeCreateOwnership.String(), + commonpb.ObjectPrivilege_PrivilegeDropOwnership.String(), + commonpb.ObjectPrivilege_PrivilegeManageOwnership.String(), + commonpb.ObjectPrivilege_PrivilegeCreateResourceGroup.String(), + commonpb.ObjectPrivilege_PrivilegeDropResourceGroup.String(), + commonpb.ObjectPrivilege_PrivilegeUpdateUser.String(), + commonpb.ObjectPrivilege_PrivilegeRenameCollection.String(), + commonpb.ObjectPrivilege_PrivilegeCreatePrivilegeGroup.String(), + commonpb.ObjectPrivilege_PrivilegeDropPrivilegeGroup.String(), + commonpb.ObjectPrivilege_PrivilegeOperatePrivilegeGroup.String(), + })..., + ) +) + +// ConvertPrivileges converts each privilege from metastore format to API format. +func ConvertPrivileges(privileges []string) []string { + return lo.Map(privileges, func(name string, _ int) string { + return MetaStore2API(name) + }) +} + +func GetPrivilegeLevel(privilege string) string { + if lo.Contains(ClusterAdminPrivileges, privilege) { + return milvuspb.PrivilegeLevel_Cluster.String() + } + if lo.Contains(DatabaseAdminPrivileges, privilege) { + return milvuspb.PrivilegeLevel_Database.String() + } + if lo.Contains(CollectionAdminPrivileges, privilege) { + return milvuspb.PrivilegeLevel_Collection.String() + } + return "" +} + // StringSet convert array to map for conveniently check if the array contains an element func StringSet(strings []string) map[string]struct{} { stringsMap := make(map[string]struct{}) diff --git a/pkg/util/paramtable/rbac_config_test.go b/pkg/util/paramtable/rbac_config_test.go index 9a780bcea435b..c7519d4e1eed1 100644 --- a/pkg/util/paramtable/rbac_config_test.go +++ b/pkg/util/paramtable/rbac_config_test.go @@ -22,26 +22,24 @@ import ( "github.com/stretchr/testify/assert" ) -func TestRbacConfig_Init(t *testing.T) { +func TestRbacConfig_DefaultPrivileges(t *testing.T) { params := ComponentParam{} params.Init(NewBaseTable(SkipRemote(true))) cfg := ¶ms.RbacConfig assert.Equal(t, len(cfg.GetDefaultPrivilegeGroupNames()), 9) - assert.True(t, cfg.IsCollectionPrivilegeGroup("CollectionReadOnly")) - assert.False(t, cfg.IsCollectionPrivilegeGroup("DatabaseReadOnly")) assert.Equal(t, cfg.Enabled.GetAsBool(), false) - assert.Equal(t, cfg.ClusterReadOnlyPrivileges.GetAsStrings(), builtinPrivilegeGroups["ClusterReadOnly"]) - assert.Equal(t, cfg.ClusterReadWritePrivileges.GetAsStrings(), builtinPrivilegeGroups["ClusterReadWrite"]) - assert.Equal(t, cfg.ClusterAdminPrivileges.GetAsStrings(), builtinPrivilegeGroups["ClusterAdmin"]) - assert.Equal(t, cfg.DBReadOnlyPrivileges.GetAsStrings(), builtinPrivilegeGroups["DatabaseReadOnly"]) - assert.Equal(t, cfg.DBReadWritePrivileges.GetAsStrings(), builtinPrivilegeGroups["DatabaseReadWrite"]) - assert.Equal(t, cfg.DBAdminPrivileges.GetAsStrings(), builtinPrivilegeGroups["DatabaseAdmin"]) - assert.Equal(t, cfg.CollectionReadOnlyPrivileges.GetAsStrings(), builtinPrivilegeGroups["CollectionReadOnly"]) - assert.Equal(t, cfg.CollectionReadWritePrivileges.GetAsStrings(), builtinPrivilegeGroups["CollectionReadWrite"]) - assert.Equal(t, cfg.CollectionAdminPrivileges.GetAsStrings(), builtinPrivilegeGroups["CollectionAdmin"]) + assert.Equal(t, cfg.ClusterReadOnlyPrivileges.GetAsStrings(), cfg.GetDefaultPrivilegeGroupPrivileges("ClusterReadOnly")) + assert.Equal(t, cfg.ClusterReadWritePrivileges.GetAsStrings(), cfg.GetDefaultPrivilegeGroupPrivileges("ClusterReadWrite")) + assert.Equal(t, cfg.ClusterAdminPrivileges.GetAsStrings(), cfg.GetDefaultPrivilegeGroupPrivileges("ClusterAdmin")) + assert.Equal(t, cfg.DBReadOnlyPrivileges.GetAsStrings(), cfg.GetDefaultPrivilegeGroupPrivileges("DatabaseReadOnly")) + assert.Equal(t, cfg.DBReadWritePrivileges.GetAsStrings(), cfg.GetDefaultPrivilegeGroupPrivileges("DatabaseReadWrite")) + assert.Equal(t, cfg.DBAdminPrivileges.GetAsStrings(), cfg.GetDefaultPrivilegeGroupPrivileges("DatabaseAdmin")) + assert.Equal(t, cfg.CollectionReadOnlyPrivileges.GetAsStrings(), cfg.GetDefaultPrivilegeGroupPrivileges("CollectionReadOnly")) + assert.Equal(t, cfg.CollectionReadWritePrivileges.GetAsStrings(), cfg.GetDefaultPrivilegeGroupPrivileges("CollectionReadWrite")) + assert.Equal(t, cfg.CollectionAdminPrivileges.GetAsStrings(), cfg.GetDefaultPrivilegeGroupPrivileges("CollectionAdmin")) } -func TestRbacConfig_Override(t *testing.T) { +func TestRbacConfig_OverridePrivileges(t *testing.T) { params := ComponentParam{} params.Init(NewBaseTable(SkipRemote(true))) diff --git a/pkg/util/paramtable/rbac_param.go b/pkg/util/paramtable/rbac_param.go index d793a4d93b26d..35816a5d8a09f 100644 --- a/pkg/util/paramtable/rbac_param.go +++ b/pkg/util/paramtable/rbac_param.go @@ -10,103 +10,6 @@ import ( "github.com/milvus-io/milvus/pkg/util" ) -var ( - builtinPrivilegeGroups = map[string][]string{ - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeGroupCollectionReadOnly.String()): collectionReadOnlyPrivilegeGroup, - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeGroupCollectionReadWrite.String()): collectionReadWritePrivilegeGroup, - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeGroupCollectionAdmin.String()): collectionAdminPrivilegeGroup, - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeGroupDatabaseReadOnly.String()): databaseReadOnlyPrivilegeGroup, - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeGroupDatabaseReadWrite.String()): databaseReadWritePrivilegeGroup, - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeGroupDatabaseAdmin.String()): databaseAdminPrivilegeGroup, - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeGroupClusterReadOnly.String()): clusterReadOnlyPrivilegeGroup, - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeGroupClusterReadWrite.String()): clusterReadWritePrivilegeGroup, - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeGroupClusterAdmin.String()): clusterAdminPrivilegeGroup, - } - - collectionReadOnlyPrivilegeGroup = []string{ - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeQuery.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeSearch.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeIndexDetail.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeGetFlushState.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeGetLoadState.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeGetLoadingProgress.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeHasPartition.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeShowPartitions.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeDescribeCollection.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeDescribeAlias.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeGetStatistics.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeListAliases.String()), - } - - collectionReadWritePrivilegeGroup = append(collectionReadOnlyPrivilegeGroup, - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeLoad.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeRelease.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeInsert.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeDelete.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeUpsert.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeImport.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeFlush.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeCompaction.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeLoadBalance.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeCreateIndex.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeDropIndex.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeCreatePartition.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeDropPartition.String()), - ) - - collectionAdminPrivilegeGroup = append(collectionReadWritePrivilegeGroup, - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeCreateAlias.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeDropAlias.String()), - ) - - databaseReadOnlyPrivilegeGroup = []string{ - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeShowCollections.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeDescribeDatabase.String()), - } - - databaseReadWritePrivilegeGroup = append(databaseReadOnlyPrivilegeGroup, - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeAlterDatabase.String()), - ) - - databaseAdminPrivilegeGroup = append(databaseReadWritePrivilegeGroup, - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeCreateCollection.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeDropCollection.String()), - ) - - clusterReadOnlyPrivilegeGroup = []string{ - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeListDatabases.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeSelectOwnership.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeSelectUser.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeDescribeResourceGroup.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeListResourceGroups.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeListPrivilegeGroups.String()), - } - - clusterReadWritePrivilegeGroup = append(clusterReadOnlyPrivilegeGroup, - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeFlushAll.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeTransferNode.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeTransferReplica.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeUpdateResourceGroups.String()), - ) - - clusterAdminPrivilegeGroup = append(clusterReadWritePrivilegeGroup, - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeBackupRBAC.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeRestoreRBAC.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeCreateDatabase.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeDropDatabase.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeCreateOwnership.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeDropOwnership.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeManageOwnership.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeCreateResourceGroup.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeDropResourceGroup.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeUpdateUser.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeRenameCollection.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeCreatePrivilegeGroup.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeDropPrivilegeGroup.String()), - util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeOperatePrivilegeGroup.String()), - ) -) - type rbacConfig struct { Enabled ParamItem `refreshable:"false"` ClusterReadOnlyPrivileges ParamItem `refreshable:"false"` @@ -134,7 +37,7 @@ func (p *rbacConfig) init(base *BaseTable) { p.ClusterReadOnlyPrivileges = ParamItem{ Key: "common.security.rbac.cluster.readonly.privileges", - DefaultValue: strings.Join(clusterReadOnlyPrivilegeGroup, ","), + DefaultValue: strings.Join(util.ClusterReadOnlyPrivileges, ","), Version: "2.4.16", Doc: "Cluster level readonly privileges", Export: true, @@ -143,7 +46,7 @@ func (p *rbacConfig) init(base *BaseTable) { p.ClusterReadWritePrivileges = ParamItem{ Key: "common.security.rbac.cluster.readwrite.privileges", - DefaultValue: strings.Join(clusterReadWritePrivilegeGroup, ","), + DefaultValue: strings.Join(util.ClusterReadWritePrivileges, ","), Version: "2.4.16", Doc: "Cluster level readwrite privileges", Export: true, @@ -152,7 +55,7 @@ func (p *rbacConfig) init(base *BaseTable) { p.ClusterAdminPrivileges = ParamItem{ Key: "common.security.rbac.cluster.admin.privileges", - DefaultValue: strings.Join(clusterAdminPrivilegeGroup, ","), + DefaultValue: strings.Join(util.ClusterAdminPrivileges, ","), Version: "2.4.16", Doc: "Cluster level admin privileges", Export: true, @@ -161,7 +64,7 @@ func (p *rbacConfig) init(base *BaseTable) { p.DBReadOnlyPrivileges = ParamItem{ Key: "common.security.rbac.database.readonly.privileges", - DefaultValue: strings.Join(databaseReadOnlyPrivilegeGroup, ","), + DefaultValue: strings.Join(util.DatabaseReadOnlyPrivileges, ","), Version: "2.4.16", Doc: "Database level readonly privileges", Export: true, @@ -170,7 +73,7 @@ func (p *rbacConfig) init(base *BaseTable) { p.DBReadWritePrivileges = ParamItem{ Key: "common.security.rbac.database.readwrite.privileges", - DefaultValue: strings.Join(databaseReadWritePrivilegeGroup, ","), + DefaultValue: strings.Join(util.DatabaseReadWritePrivileges, ","), Version: "2.4.16", Doc: "Database level readwrite privileges", Export: true, @@ -179,7 +82,7 @@ func (p *rbacConfig) init(base *BaseTable) { p.DBAdminPrivileges = ParamItem{ Key: "common.security.rbac.database.admin.privileges", - DefaultValue: strings.Join(databaseAdminPrivilegeGroup, ","), + DefaultValue: strings.Join(util.DatabaseAdminPrivileges, ","), Version: "2.4.16", Doc: "Database level admin privileges", Export: true, @@ -188,7 +91,7 @@ func (p *rbacConfig) init(base *BaseTable) { p.CollectionReadOnlyPrivileges = ParamItem{ Key: "common.security.rbac.collection.readonly.privileges", - DefaultValue: strings.Join(collectionReadOnlyPrivilegeGroup, ","), + DefaultValue: strings.Join(util.CollectionReadOnlyPrivileges, ","), Version: "2.4.16", Doc: "Collection level readonly privileges", Export: true, @@ -197,7 +100,7 @@ func (p *rbacConfig) init(base *BaseTable) { p.CollectionReadWritePrivileges = ParamItem{ Key: "common.security.rbac.collection.readwrite.privileges", - DefaultValue: strings.Join(collectionReadWritePrivilegeGroup, ","), + DefaultValue: strings.Join(util.CollectionReadWritePrivileges, ","), Version: "2.4.16", Doc: "Collection level readwrite privileges", Export: true, @@ -206,7 +109,7 @@ func (p *rbacConfig) init(base *BaseTable) { p.CollectionAdminPrivileges = ParamItem{ Key: "common.security.rbac.collection.admin.privileges", - DefaultValue: strings.Join(collectionAdminPrivilegeGroup, ","), + DefaultValue: strings.Join(util.CollectionAdminPrivileges, ","), Version: "2.4.16", Doc: "Collection level admin privileges", Export: true, @@ -219,15 +122,15 @@ func (p *rbacConfig) GetDefaultPrivilegeGroups() []*milvuspb.PrivilegeGroupInfo GroupName string Privileges func() []string }{ - {"ClusterReadOnly", p.ClusterReadOnlyPrivileges.GetAsStrings}, - {"ClusterReadWrite", p.ClusterReadWritePrivileges.GetAsStrings}, - {"ClusterAdmin", p.ClusterAdminPrivileges.GetAsStrings}, - {"DatabaseReadOnly", p.DBReadOnlyPrivileges.GetAsStrings}, - {"DatabaseReadWrite", p.DBReadWritePrivileges.GetAsStrings}, - {"DatabaseAdmin", p.DBAdminPrivileges.GetAsStrings}, - {"CollectionReadOnly", p.CollectionReadOnlyPrivileges.GetAsStrings}, - {"CollectionReadWrite", p.CollectionReadWritePrivileges.GetAsStrings}, - {"CollectionAdmin", p.CollectionAdminPrivileges.GetAsStrings}, + {util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeGroupClusterReadOnly.String()), p.ClusterReadOnlyPrivileges.GetAsStrings}, + {util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeGroupClusterReadWrite.String()), p.ClusterReadWritePrivileges.GetAsStrings}, + {util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeGroupClusterAdmin.String()), p.ClusterAdminPrivileges.GetAsStrings}, + {util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeGroupDatabaseReadOnly.String()), p.DBReadOnlyPrivileges.GetAsStrings}, + {util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeGroupDatabaseReadWrite.String()), p.DBReadWritePrivileges.GetAsStrings}, + {util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeGroupDatabaseAdmin.String()), p.DBAdminPrivileges.GetAsStrings}, + {util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeGroupCollectionReadOnly.String()), p.CollectionReadOnlyPrivileges.GetAsStrings}, + {util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeGroupCollectionReadWrite.String()), p.CollectionReadWritePrivileges.GetAsStrings}, + {util.MetaStore2API(commonpb.ObjectPrivilege_PrivilegeGroupCollectionAdmin.String()), p.CollectionAdminPrivileges.GetAsStrings}, } builtinGroups := make([]*milvuspb.PrivilegeGroupInfo, 0, len(privilegeGroupConfigs)) @@ -245,21 +148,34 @@ func (p *rbacConfig) GetDefaultPrivilegeGroups() []*milvuspb.PrivilegeGroupInfo func (p *rbacConfig) GetDefaultPrivilegeGroup(privName string) *milvuspb.PrivilegeGroupInfo { for _, group := range p.GetDefaultPrivilegeGroups() { - if group.GroupName == privName { + if group.GetGroupName() == privName { return group } } return nil } +func (p *rbacConfig) GetDefaultPrivilegeGroupPrivileges(groupName string) []string { + group := p.GetDefaultPrivilegeGroup(groupName) + if group == nil { + return nil + } + return lo.Map(group.GetPrivileges(), func(priv *milvuspb.PrivilegeEntity, _ int) string { + return priv.GetName() + }) +} + func (p *rbacConfig) GetDefaultPrivilegeGroupNames() []string { - return lo.Keys(builtinPrivilegeGroups) + return lo.Map(p.GetDefaultPrivilegeGroups(), func(group *milvuspb.PrivilegeGroupInfo, _ int) string { + return group.GroupName + }) } func (p *rbacConfig) IsCollectionPrivilegeGroup(privName string) bool { - collectionPrivilegeGroups := lo.PickBy(builtinPrivilegeGroups, func(groupName string, _ []string) bool { - return strings.Contains(groupName, "Collection") - }) - _, exists := collectionPrivilegeGroups[privName] - return exists + for _, groupName := range p.GetDefaultPrivilegeGroupNames() { + if strings.Contains(groupName, milvuspb.PrivilegeLevel_Collection.String()) && groupName == privName { + return true + } + } + return false } diff --git a/tests/integration/rbac/privilege_group_test.go b/tests/integration/rbac/privilege_group_test.go index 966eb6b55b9c5..43b1fae34fad4 100644 --- a/tests/integration/rbac/privilege_group_test.go +++ b/tests/integration/rbac/privilege_group_test.go @@ -268,7 +268,8 @@ func (s *PrivilegeGroupTestSuite) TestGrantV2CustomPrivilegeGroup() { selectResp, _ = s.validateGrants(ctx, role, commonpb.ObjectType_Global.String(), util.AnyWord, util.AnyWord) s.Len(selectResp.GetEntities(), 2) - // add different object type privileges to group1 is not allowed + // add different privilege group level privileges to group1 is not allowed + s.Equal(milvuspb.PrivilegeLevel_Database.String(), util.GetPrivilegeLevel("CreateCollection")) resp, _ = s.Cluster.Proxy.OperatePrivilegeGroup(ctx, &milvuspb.OperatePrivilegeGroupRequest{ GroupName: "group1", Type: milvuspb.OperatePrivilegeGroupType_AddPrivilegesToGroup, diff --git a/tests/integration/rbac/rbac_backup_test.go b/tests/integration/rbac/rbac_backup_test.go index ce526321da654..9ef45878a7f84 100644 --- a/tests/integration/rbac/rbac_backup_test.go +++ b/tests/integration/rbac/rbac_backup_test.go @@ -104,9 +104,9 @@ func (s *RBACBackupTestSuite) TestBackup() { createRole(roleName) // grant collection level search privilege to role test_role - operatePrivilege(roleName, "Search", util.AnyWord, util.AnyWord, milvuspb.OperatePrivilegeType_Grant) + operatePrivilege(roleName, "Search", util.AnyWord, util.DefaultDBName, milvuspb.OperatePrivilegeType_Grant) - // create privielge group test_group + // create privilege group test_group groupName := "test_group" createPrivGroupResp, err := s.Cluster.Proxy.CreatePrivilegeGroup(ctx, &milvuspb.CreatePrivilegeGroupRequest{ GroupName: groupName, @@ -114,6 +114,11 @@ func (s *RBACBackupTestSuite) TestBackup() { s.NoError(err) s.True(merr.Ok(createPrivGroupResp)) + collectionPrivileges := []*milvuspb.PrivilegeEntity{{Name: "Query"}, {Name: "Insert"}} + for _, p := range collectionPrivileges { + s.Equal(milvuspb.PrivilegeLevel_Collection.String(), util.GetPrivilegeLevel(p.Name)) + } + // add query and insert privilege to group test_group addPrivsToGroupResp, err := s.Cluster.Proxy.OperatePrivilegeGroup(ctx, &milvuspb.OperatePrivilegeGroupRequest{ GroupName: groupName, @@ -124,7 +129,7 @@ func (s *RBACBackupTestSuite) TestBackup() { s.True(merr.Ok(addPrivsToGroupResp)) // grant privilege group test_group to role test_role - operatePrivilege(roleName, groupName, util.AnyWord, util.AnyWord, milvuspb.OperatePrivilegeType_Grant) + operatePrivilege(roleName, groupName, util.AnyWord, util.DefaultDBName, milvuspb.OperatePrivilegeType_Grant) userName := "test_user" passwd := "test_passwd" @@ -166,10 +171,10 @@ func (s *RBACBackupTestSuite) TestBackup() { s.False(merr.Ok(restoreRBACResp)) // revoke privilege search from role test_role before dropping the role - operatePrivilege(roleName, "Search", util.AnyWord, util.AnyWord, milvuspb.OperatePrivilegeType_Revoke) + operatePrivilege(roleName, "Search", util.AnyWord, util.DefaultDBName, milvuspb.OperatePrivilegeType_Revoke) // revoke privilege group test_group from role test_role before dropping the role - operatePrivilege(roleName, groupName, util.AnyWord, util.AnyWord, milvuspb.OperatePrivilegeType_Revoke) + operatePrivilege(roleName, groupName, util.AnyWord, util.DefaultDBName, milvuspb.OperatePrivilegeType_Revoke) // drop privilege group test_group dropPrivGroupResp, err := s.Cluster.Proxy.DropPrivilegeGroup(ctx, &milvuspb.DropPrivilegeGroupRequest{ @@ -206,9 +211,9 @@ func (s *RBACBackupTestSuite) TestBackup() { s.Equal(backupRBACResp2.GetRBACMeta().String(), backupRBACResp.GetRBACMeta().String()) // clean rbac meta - operatePrivilege(roleName, "Search", util.AnyWord, util.AnyWord, milvuspb.OperatePrivilegeType_Revoke) + operatePrivilege(roleName, "Search", util.AnyWord, util.DefaultDBName, milvuspb.OperatePrivilegeType_Revoke) - operatePrivilege(roleName, groupName, util.AnyWord, util.AnyWord, milvuspb.OperatePrivilegeType_Revoke) + operatePrivilege(roleName, groupName, util.AnyWord, util.DefaultDBName, milvuspb.OperatePrivilegeType_Revoke) dropPrivGroupResp2, err := s.Cluster.Proxy.DropPrivilegeGroup(ctx, &milvuspb.DropPrivilegeGroupRequest{ GroupName: groupName, diff --git a/tests/python_client/testcases/test_utility.py b/tests/python_client/testcases/test_utility.py index 4608208499d78..9a3c7f3eaa423 100644 --- a/tests/python_client/testcases/test_utility.py +++ b/tests/python_client/testcases/test_utility.py @@ -5384,7 +5384,7 @@ def test_add_privileges_to_group_with_privilege_invalid_value(self, host, port): name = "privilege_group_1" self.utility_wrap.create_privilege_group(privilege_group=name) privilege_name = "invalid_privilege" - error = {"err_code": 1100, "err_msg": f"there is no privilege name or privielge group name [{privilege_name}] " + error = {"err_code": 1100, "err_msg": f"there is no privilege name or privilege group name [{privilege_name}] " f"defined in system to operate: invalid parameter"} self.utility_wrap.add_privileges_to_group(privilege_group="privilege_group_1", privileges=[privilege_name], check_task=CheckTasks.err_res, check_items=error) @@ -5483,7 +5483,7 @@ def test_remove_privileges_to_group_with_privilege_invalid_value(self, host, por name = "privilege_group_1" self.utility_wrap.create_privilege_group(privilege_group=name) privilege_name = "invalid_privilege" - error = {"err_code": 1100, "err_msg": f"there is no privilege name or privielge group name [{privilege_name}] " + error = {"err_code": 1100, "err_msg": f"there is no privilege name or privilege group name [{privilege_name}] " f"defined in system to operate: invalid parameter"} self.utility_wrap.remove_privileges_from_group(privilege_group="privilege_group_1", privileges=[privilege_name], check_task=CheckTasks.err_res, check_items=error) From accc9e7fbfa2bad3ec391abdd4db2bdc3eb2a272 Mon Sep 17 00:00:00 2001 From: smellthemoon <64083300+smellthemoon@users.noreply.github.com> Date: Mon, 13 Jan 2025 16:04:58 +0800 Subject: [PATCH 24/34] fix: fail to get empty index num rows (#39155) #39125 Signed-off-by: lixinguo Co-authored-by: lixinguo --- internal/core/src/index/ScalarIndexSort.cpp | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/internal/core/src/index/ScalarIndexSort.cpp b/internal/core/src/index/ScalarIndexSort.cpp index f53966bf43c42..e43d35902b6f0 100644 --- a/internal/core/src/index/ScalarIndexSort.cpp +++ b/internal/core/src/index/ScalarIndexSort.cpp @@ -176,9 +176,14 @@ ScalarIndexSort::LoadWithoutAssemble(const BinarySet& index_binary, auto index_data = index_binary.GetByName("index_data"); data_.resize(index_size); auto index_num_rows = index_binary.GetByName("index_num_rows"); - memcpy(&total_num_rows_, - index_num_rows->data.get(), - (size_t)index_num_rows->size); + if (index_num_rows) { + memcpy(&total_num_rows_, + index_num_rows->data.get(), + (size_t)index_num_rows->size); + } else { + total_num_rows_ = index_size; + } + idx_to_offsets_.resize(total_num_rows_); valid_bitset_ = TargetBitmap(total_num_rows_, false); memcpy(data_.data(), index_data->data.get(), (size_t)index_data->size); From a00ba861a49d39caea43ae4511e7a2d568033646 Mon Sep 17 00:00:00 2001 From: Bingyi Sun Date: Mon, 13 Jan 2025 16:14:58 +0800 Subject: [PATCH 25/34] fix: Fix in filter search result is empty if pk type is varchar (#39106) https://github.com/milvus-io/milvus/issues/39107 Signed-off-by: sunby --- internal/core/src/segcore/ChunkedSegmentSealedImpl.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/internal/core/src/segcore/ChunkedSegmentSealedImpl.cpp b/internal/core/src/segcore/ChunkedSegmentSealedImpl.cpp index 0610f3adabd10..1a79fc8cff87b 100644 --- a/internal/core/src/segcore/ChunkedSegmentSealedImpl.cpp +++ b/internal/core/src/segcore/ChunkedSegmentSealedImpl.cpp @@ -1215,7 +1215,7 @@ ChunkedSegmentSealedImpl::search_sorted_pk(const PkType& pk, var_column->GetChunk(i)); auto offset = string_chunk->binary_search_string(target); for (; offset != -1 && offset < string_chunk->RowNums() && - var_column->RawAt(offset) == target; + string_chunk->operator[](offset) == target; ++offset) { auto segment_offset = offset + num_rows_until_chunk; if (condition(segment_offset)) { From b3daae8e40f3bb746dba048eb3ae1cb07cc79ae3 Mon Sep 17 00:00:00 2001 From: congqixia Date: Mon, 13 Jan 2025 18:04:58 +0800 Subject: [PATCH 26/34] fix: Make KeyLock obj pool size unlimited (#39182) Related to #39181 Signed-off-by: Congqi Xia --- pkg/util/lock/key_lock.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/util/lock/key_lock.go b/pkg/util/lock/key_lock.go index f09a0069b884e..a85ee02424714 100644 --- a/pkg/util/lock/key_lock.go +++ b/pkg/util/lock/key_lock.go @@ -33,7 +33,7 @@ var ( }) lockerPoolConfig = &pool.ObjectPoolConfig{ LIFO: pool.DefaultLIFO, - MaxTotal: 64, + MaxTotal: -1, MaxIdle: 64, MinIdle: pool.DefaultMinIdle, MinEvictableIdleTime: pool.DefaultMinEvictableIdleTime, From cc8e6ebd5e93a671a765dfb8d96120741672e98b Mon Sep 17 00:00:00 2001 From: sre-ci-robot <56469371+sre-ci-robot@users.noreply.github.com> Date: Mon, 13 Jan 2025 18:37:03 +0800 Subject: [PATCH 27/34] [automated] Bump milvus version to v2.5.3 (#39215) Bump milvus version to v2.5.3 Signed-off-by: sre-ci-robot sre-ci-robot@users.noreply.github.com Co-authored-by: github-actions[bot] <41898282+github-actions[bot]@users.noreply.github.com> --- scripts/standalone_embed.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scripts/standalone_embed.sh b/scripts/standalone_embed.sh index f09117486e671..0c3b3420366f2 100755 --- a/scripts/standalone_embed.sh +++ b/scripts/standalone_embed.sh @@ -47,7 +47,7 @@ EOF --health-start-period=90s \ --health-timeout=20s \ --health-retries=3 \ - milvusdb/milvus:v2.5.2 \ + milvusdb/milvus:v2.5.3 \ milvus run standalone 1> /dev/null } From 702347bbfdd942922c24caf6a4cc7041fa3b5172 Mon Sep 17 00:00:00 2001 From: "yihao.dai" Date: Mon, 13 Jan 2025 21:49:12 +0800 Subject: [PATCH 28/34] fix: Fix incorrect memory estimation for small segments (#38813) Skip estimation index memory logic for segments without index file. issue: https://github.com/milvus-io/milvus/issues/37630 Signed-off-by: bigsheeper --- internal/querynodev2/segments/segment_loader.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/internal/querynodev2/segments/segment_loader.go b/internal/querynodev2/segments/segment_loader.go index f7ae5c0712000..fc44d7db6c8ad 100644 --- a/internal/querynodev2/segments/segment_loader.go +++ b/internal/querynodev2/segments/segment_loader.go @@ -1483,7 +1483,7 @@ func getResourceUsageEstimateOfSegment(schema *schemapb.CollectionSchema, loadIn isVectorType := typeutil.IsVectorType(fieldSchema.DataType) shouldCalculateDataSize := false - if fieldIndexInfo, ok := fieldID2IndexInfo[fieldID]; ok { + if fieldIndexInfo, ok := fieldID2IndexInfo[fieldID]; ok && len(fieldIndexInfo.GetIndexFilePaths()) > 0 { var estimateResult ResourceEstimate err := GetCLoadInfoWithFunc(ctx, fieldSchema, loadInfo, fieldIndexInfo, func(c *LoadIndexInfo) error { GetDynamicPool().Submit(func() (any, error) { From 3447ff73104c35d50cb046a1b194d19f460b9e6a Mon Sep 17 00:00:00 2001 From: Alexander Guzhva Date: Tue, 14 Jan 2025 01:50:58 +0000 Subject: [PATCH 29/34] enhance: [bitset] extend op_find() to be able to search both 0 and 1 (#39176) issue: #39124 `bitset::find_first()` and `bitset::find_next()` now accept one more parameter, which allows to search for `0` bit instead of `1` bit Signed-off-by: Alexandr Guzhva --- internal/core/src/bitset/CMakeLists.txt | 4 +- internal/core/src/bitset/bitset.h | 17 ++-- internal/core/src/bitset/detail/bit_wise.h | 5 +- .../src/bitset/detail/element_vectorized.h | 5 +- .../core/src/bitset/detail/element_wise.h | 93 ++++++++++++++++++- internal/core/src/bitset/detail/proxy.h | 2 + internal/core/unittest/test_bitset.cpp | 56 ++++++----- 7 files changed, 143 insertions(+), 39 deletions(-) diff --git a/internal/core/src/bitset/CMakeLists.txt b/internal/core/src/bitset/CMakeLists.txt index 3f7c6ae24d776..7bec771da9fb1 100644 --- a/internal/core/src/bitset/CMakeLists.txt +++ b/internal/core/src/bitset/CMakeLists.txt @@ -22,8 +22,8 @@ if (${CMAKE_SYSTEM_PROCESSOR} STREQUAL "x86_64") detail/platform/x86/instruction_set.cpp ) - set_source_files_properties(detail/platform/x86/avx512-inst.cpp PROPERTIES COMPILE_FLAGS "-mavx512f -mavx512bw -mavx512vl -mavx512dq") - set_source_files_properties(detail/platform/x86/avx2-inst.cpp PROPERTIES COMPILE_FLAGS "-mavx2 -mavx -mfma") + set_source_files_properties(detail/platform/x86/avx512-inst.cpp PROPERTIES COMPILE_FLAGS "-mavx512f -mavx512bw -mavx512vl -mavx512dq -mavx512cd -mbmi") + set_source_files_properties(detail/platform/x86/avx2-inst.cpp PROPERTIES COMPILE_FLAGS "-mavx2 -mavx -mfma -mbmi") # set_source_files_properties(detail/platform/dynamic.cpp PROPERTIES COMPILE_FLAGS "-mavx512f -mavx512bw -mavx512vl -mavx512dq") # set_source_files_properties(detail/platform/dynamic.cpp PROPERTIES COMPILE_FLAGS "-mavx2 -mavx -mfma") diff --git a/internal/core/src/bitset/bitset.h b/internal/core/src/bitset/bitset.h index 28859fbdf65cf..83ea4b1178c5f 100644 --- a/internal/core/src/bitset/bitset.h +++ b/internal/core/src/bitset/bitset.h @@ -546,23 +546,26 @@ class BitsetBase { return as_derived(); } - // Find the index of the first bit set to true. + // Find the index of the first bit set to either true (default), or false. inline std::optional - find_first() const { + find_first(const bool is_set = true) const { return policy_type::op_find( - this->data(), this->offset(), this->size(), 0); + this->data(), this->offset(), this->size(), 0, is_set); } - // Find the index of the first bit set to true, starting from a given bit index. + // Find the index of the first bit set to either true (default), or false, starting from a given bit index. inline std::optional - find_next(const size_t starting_bit_idx) const { + find_next(const size_t starting_bit_idx, const bool is_set = true) const { const size_t size_v = this->size(); if (starting_bit_idx + 1 >= size_v) { return std::nullopt; } - return policy_type::op_find( - this->data(), this->offset(), this->size(), starting_bit_idx + 1); + return policy_type::op_find(this->data(), + this->offset(), + this->size(), + starting_bit_idx + 1, + is_set); } // Read multiple bits starting from a given bit index. diff --git a/internal/core/src/bitset/detail/bit_wise.h b/internal/core/src/bitset/detail/bit_wise.h index f3d08dc5be5c4..0103a58209e0a 100644 --- a/internal/core/src/bitset/detail/bit_wise.h +++ b/internal/core/src/bitset/detail/bit_wise.h @@ -315,10 +315,11 @@ struct BitWiseBitsetPolicy { op_find(const data_type* const data, const size_t start, const size_t size, - const size_t starting_idx) { + const size_t starting_idx, + const bool is_set) { for (size_t i = starting_idx; i < size; i++) { const auto proxy = get_proxy(data, start + i); - if (proxy) { + if (proxy == is_set) { return i; } } diff --git a/internal/core/src/bitset/detail/element_vectorized.h b/internal/core/src/bitset/detail/element_vectorized.h index 93668904abedd..f490741ac0307 100644 --- a/internal/core/src/bitset/detail/element_vectorized.h +++ b/internal/core/src/bitset/detail/element_vectorized.h @@ -220,9 +220,10 @@ struct VectorizedElementWiseBitsetPolicy { op_find(const data_type* const data, const size_t start, const size_t size, - const size_t starting_idx) { + const size_t starting_idx, + const bool is_set) { return ElementWiseBitsetPolicy::op_find( - data, start, size, starting_idx); + data, start, size, starting_idx, is_set); } // diff --git a/internal/core/src/bitset/detail/element_wise.h b/internal/core/src/bitset/detail/element_wise.h index a738ef42e6a90..abebe32847f8c 100644 --- a/internal/core/src/bitset/detail/element_wise.h +++ b/internal/core/src/bitset/detail/element_wise.h @@ -718,10 +718,10 @@ struct ElementWiseBitsetPolicy { // static inline std::optional - op_find(const data_type* const data, - const size_t start, - const size_t size, - const size_t starting_idx) { + op_find_1(const data_type* const data, + const size_t start, + const size_t size, + const size_t starting_idx) { if (size == 0) { return std::nullopt; } @@ -788,6 +788,91 @@ struct ElementWiseBitsetPolicy { return std::nullopt; } + static inline std::optional + op_find_0(const data_type* const data, + const size_t start, + const size_t size, + const size_t starting_idx) { + if (size == 0) { + return std::nullopt; + } + + // + auto start_element = get_element(start + starting_idx); + const auto end_element = get_element(start + size); + + const auto start_shift = get_shift(start + starting_idx); + const auto end_shift = get_shift(start + size); + + // same element? + if (start_element == end_element) { + const data_type existing_v = ~data[start_element]; + + const data_type existing_mask = get_shift_mask_end(start_shift) & + get_shift_mask_begin(end_shift); + + const data_type value = existing_v & existing_mask; + if (value != 0) { + const auto ctz = CtzHelper::ctz(value); + return size_t(ctz) + start_element * data_bits - start; + } else { + return std::nullopt; + } + } + + // process the first element + if (start_shift != 0) { + const data_type existing_v = ~data[start_element]; + const data_type existing_mask = get_shift_mask_end(start_shift); + + const data_type value = existing_v & existing_mask; + if (value != 0) { + const auto ctz = CtzHelper::ctz(value) + + start_element * data_bits - start; + return size_t(ctz); + } + + start_element += 1; + } + + // process the middle + for (size_t i = start_element; i < end_element; i++) { + const data_type value = ~data[i]; + if (value != 0) { + const auto ctz = CtzHelper::ctz(value); + return size_t(ctz) + i * data_bits - start; + } + } + + // process the last element + if (end_shift != 0) { + const data_type existing_v = ~data[end_element]; + const data_type existing_mask = get_shift_mask_begin(end_shift); + + const data_type value = existing_v & existing_mask; + if (value != 0) { + const auto ctz = CtzHelper::ctz(value); + return size_t(ctz) + end_element * data_bits - start; + } + } + + return std::nullopt; + } + + // + static inline std::optional + op_find(const data_type* const data, + const size_t start, + const size_t size, + const size_t starting_idx, + const bool is_set) { + if (is_set) { + return op_find_1(data, start, size, starting_idx); + } else { + return op_find_0(data, start, size, starting_idx); + } + } + // template static inline void diff --git a/internal/core/src/bitset/detail/proxy.h b/internal/core/src/bitset/detail/proxy.h index b29eaec7bb13c..2b2d6613f7e3b 100644 --- a/internal/core/src/bitset/detail/proxy.h +++ b/internal/core/src/bitset/detail/proxy.h @@ -16,6 +16,8 @@ #pragma once +#include + namespace milvus { namespace bitset { namespace detail { diff --git a/internal/core/unittest/test_bitset.cpp b/internal/core/unittest/test_bitset.cpp index c307fc33df3db..e4decc751caf2 100644 --- a/internal/core/unittest/test_bitset.cpp +++ b/internal/core/unittest/test_bitset.cpp @@ -346,7 +346,7 @@ from_i32(const int32_t i) { // template void -TestFindImpl(BitsetT& bitset, const size_t max_v) { +TestFindImpl(BitsetT& bitset, const size_t max_v, const bool is_set) { const size_t n = bitset.size(); std::default_random_engine rng(123); @@ -361,9 +361,13 @@ TestFindImpl(BitsetT& bitset, const size_t max_v) { } } + if (!is_set) { + bitset.flip(); + } + StopWatch sw; - auto bit_idx = bitset.find_first(); + auto bit_idx = bitset.find_first(is_set); if (!bit_idx.has_value()) { ASSERT_EQ(one_pos.size(), 0); return; @@ -372,7 +376,7 @@ TestFindImpl(BitsetT& bitset, const size_t max_v) { for (size_t i = 0; i < one_pos.size(); i++) { ASSERT_TRUE(bit_idx.has_value()) << n << ", " << max_v; ASSERT_EQ(bit_idx.value(), one_pos[i]) << n << ", " << max_v; - bit_idx = bitset.find_next(bit_idx.value()); + bit_idx = bitset.find_next(bit_idx.value(), is_set); } ASSERT_FALSE(bit_idx.has_value()) @@ -387,32 +391,40 @@ template void TestFindImpl() { for (const size_t n : typical_sizes) { - for (const size_t pr : {1, 100}) { - BitsetT bitset(n); - bitset.reset(); - - if (print_log) { - printf("Testing bitset, n=%zd, pr=%zd\n", n, pr); - } - - TestFindImpl(bitset, pr); - - for (const size_t offset : typical_offsets) { - if (offset >= n) { - continue; - } - + for (const bool is_set : {true, false}) { + for (const size_t pr : {1, 100}) { + BitsetT bitset(n); bitset.reset(); - auto view = bitset.view(offset); if (print_log) { - printf("Testing bitset view, n=%zd, offset=%zd, pr=%zd\n", + printf("Testing bitset, n=%zd, is_set=%d, pr=%zd\n", n, - offset, + (is_set) ? 1 : 0, pr); } - TestFindImpl(view, pr); + TestFindImpl(bitset, pr, is_set); + + for (const size_t offset : typical_offsets) { + if (offset >= n) { + continue; + } + + bitset.reset(); + auto view = bitset.view(offset); + + if (print_log) { + printf( + "Testing bitset view, n=%zd, offset=%zd, " + "is_set=%d, pr=%zd\n", + n, + offset, + (is_set) ? 1 : 0, + pr); + } + + TestFindImpl(view, pr, is_set); + } } } } From 5e38f01e5b672e877ccdfc085b7960a976b1042c Mon Sep 17 00:00:00 2001 From: Buqian Zheng Date: Tue, 14 Jan 2025 10:21:05 +0800 Subject: [PATCH 30/34] enhance: update knowhere version (#39212) Signed-off-by: Buqian Zheng --- internal/core/thirdparty/knowhere/CMakeLists.txt | 2 +- internal/core/unittest/test_indexing.cpp | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/internal/core/thirdparty/knowhere/CMakeLists.txt b/internal/core/thirdparty/knowhere/CMakeLists.txt index f7d697b04683e..6f33ef75a2433 100644 --- a/internal/core/thirdparty/knowhere/CMakeLists.txt +++ b/internal/core/thirdparty/knowhere/CMakeLists.txt @@ -14,7 +14,7 @@ # Update KNOWHERE_VERSION for the first occurrence milvus_add_pkg_config("knowhere") set_property(DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR} PROPERTY INCLUDE_DIRECTORIES "") -set( KNOWHERE_VERSION 9a6a8dfb ) +set( KNOWHERE_VERSION f256dd5 ) set( GIT_REPOSITORY "https://github.com/zilliztech/knowhere.git") message(STATUS "Knowhere repo: ${GIT_REPOSITORY}") message(STATUS "Knowhere version: ${KNOWHERE_VERSION}") diff --git a/internal/core/unittest/test_indexing.cpp b/internal/core/unittest/test_indexing.cpp index bd3678f6f5d13..35656a5d6da2b 100644 --- a/internal/core/unittest/test_indexing.cpp +++ b/internal/core/unittest/test_indexing.cpp @@ -731,6 +731,10 @@ TEST_P(IndexTest, GetVector_EmptySparseVector) { vec_index->Load(milvus::tracer::TraceContext{}, load_conf); EXPECT_EQ(vec_index->Count(), NB); + if (!vec_index->HasRawData()) { + return; + } + auto ids_ds = GenRandomIds(NB); auto sparse_rows = vec_index->GetSparseVector(ids_ds); for (size_t i = 0; i < NB; ++i) { From 3e788f0fbd13bbaf9277975e14fecb95a1aebf7a Mon Sep 17 00:00:00 2001 From: Zhen Ye Date: Tue, 14 Jan 2025 10:33:06 +0800 Subject: [PATCH 31/34] enhance: record memory size (uncompressed) item for index (#38770) issue: #38715 - Current milvus use a serialized index size(compressed) for estimate resource for loading. - Add a new field `MemSize` (before compressing) for index to estimate resource. --------- Signed-off-by: chyezh --- cmd/tools/migration/meta/210_to_220.go | 30 +- internal/core/src/common/protobuf_utils.cpp | 45 + internal/core/src/common/protobuf_utils.h | 42 + internal/core/src/common/protobuf_utils_c.h | 40 + internal/core/src/index/BitmapIndex.cpp | 9 +- internal/core/src/index/BitmapIndex.h | 2 +- internal/core/src/index/HybridScalarIndex.cpp | 6 +- internal/core/src/index/HybridScalarIndex.h | 2 +- internal/core/src/index/Index.h | 3 +- internal/core/src/index/IndexStats.cpp | 81 ++ internal/core/src/index/IndexStats.h | 80 ++ .../core/src/index/InvertedIndexTantivy.cpp | 21 +- .../core/src/index/InvertedIndexTantivy.h | 2 +- internal/core/src/index/ScalarIndexSort.cpp | 10 +- internal/core/src/index/ScalarIndexSort.h | 2 +- internal/core/src/index/StringIndexMarisa.cpp | 10 +- internal/core/src/index/StringIndexMarisa.h | 2 +- internal/core/src/index/TextMatchIndex.cpp | 22 +- internal/core/src/index/TextMatchIndex.h | 3 +- internal/core/src/index/VectorDiskIndex.cpp | 9 +- internal/core/src/index/VectorDiskIndex.h | 2 +- internal/core/src/index/VectorMemIndex.cpp | 10 +- internal/core/src/index/VectorMemIndex.h | 2 +- .../core/src/indexbuilder/IndexCreatorBase.h | 3 +- .../src/indexbuilder/ScalarIndexCreator.cpp | 2 +- .../src/indexbuilder/ScalarIndexCreator.h | 2 +- .../core/src/indexbuilder/VecIndexCreator.cpp | 2 +- .../core/src/indexbuilder/VecIndexCreator.h | 2 +- internal/core/src/indexbuilder/index_c.cpp | 16 +- internal/core/src/indexbuilder/index_c.h | 5 +- internal/core/src/segcore/Types.h | 5 +- .../core/src/storage/DiskFileManagerImpl.cpp | 2 + .../core/src/storage/DiskFileManagerImpl.h | 7 + .../core/src/storage/MemFileManagerImpl.cpp | 1 + .../core/src/storage/MemFileManagerImpl.h | 7 + .../core/unittest/test_array_bitmap_index.cpp | 10 +- internal/core/unittest/test_bitmap_index.cpp | 11 +- internal/core/unittest/test_hybrid_index.cpp | 10 +- internal/core/unittest/test_indexing.cpp | 103 +- .../core/unittest/test_inverted_index.cpp | 20 +- internal/core/unittest/test_utils.cpp | 6 + internal/core/unittest/test_utils/DataGen.h | 8 +- internal/datacoord/compaction_trigger_test.go | 90 +- internal/datacoord/garbage_collector_test.go | 356 +++---- internal/datacoord/index_meta.go | 15 +- internal/datacoord/index_meta_test.go | 288 +++--- internal/datacoord/index_service.go | 3 +- internal/datacoord/index_service_test.go | 600 +++++------ internal/datacoord/metrics_info_test.go | 28 +- internal/datacoord/server_test.go | 28 +- internal/datacoord/services_test.go | 28 +- internal/datacoord/task_scheduler_test.go | 672 ++++++------- internal/indexnode/indexnode_service.go | 4 + internal/indexnode/task_index.go | 16 +- internal/indexnode/taskinfo_ops.go | 23 +- .../metastore/kv/datacoord/kv_catalog_test.go | 56 +- internal/metastore/model/segment_index.go | 17 +- .../metastore/model/segment_index_test.go | 28 +- .../querycoordv2/meta/coordinator_broker.go | 2 +- internal/querynodev2/segments/segment.go | 3 +- internal/util/indexcgowrapper/index.go | 61 +- internal/util/segcore/cgo_util.go | 9 + internal/util/segcore/cgo_util_test.go | 29 + internal/util/segcore/cgo_util_test_only.go | 31 + pkg/proto/cgo_msg.proto | 10 + pkg/proto/cgopb/cgo_msg.pb.go | 177 +++- pkg/proto/index_coord.proto | 4 +- pkg/proto/indexpb/index_coord.pb.go | 943 +++++++++--------- pkg/proto/worker.proto | 1 + pkg/proto/workerpb/worker.pb.go | 342 ++++--- 70 files changed, 2548 insertions(+), 1973 deletions(-) create mode 100644 internal/core/src/common/protobuf_utils.cpp create mode 100644 internal/core/src/common/protobuf_utils_c.h create mode 100644 internal/core/src/index/IndexStats.cpp create mode 100644 internal/core/src/index/IndexStats.h create mode 100644 internal/util/segcore/cgo_util_test_only.go diff --git a/cmd/tools/migration/meta/210_to_220.go b/cmd/tools/migration/meta/210_to_220.go index 14b59a992a21e..ac1cf6f5f99a5 100644 --- a/cmd/tools/migration/meta/210_to_220.go +++ b/cmd/tools/migration/meta/210_to_220.go @@ -251,21 +251,21 @@ func combineToSegmentIndexesMeta220(segmentIndexes SegmentIndexesMeta210, indexB } segmentIndexModel := &model.SegmentIndex{ - SegmentID: segID, - CollectionID: record.GetCollectionID(), - PartitionID: record.GetPartitionID(), - NumRows: buildMeta.GetReq().GetNumRows(), - IndexID: indexID, - BuildID: record.GetBuildID(), - NodeID: buildMeta.GetNodeID(), - IndexVersion: buildMeta.GetIndexVersion(), - IndexState: buildMeta.GetState(), - FailReason: buildMeta.GetFailReason(), - IsDeleted: buildMeta.GetMarkDeleted(), - CreatedUTCTime: record.GetCreateTime(), - IndexFileKeys: fileKeys, - IndexSize: buildMeta.GetSerializeSize(), - WriteHandoff: buildMeta.GetState() == commonpb.IndexState_Finished, + SegmentID: segID, + CollectionID: record.GetCollectionID(), + PartitionID: record.GetPartitionID(), + NumRows: buildMeta.GetReq().GetNumRows(), + IndexID: indexID, + BuildID: record.GetBuildID(), + NodeID: buildMeta.GetNodeID(), + IndexVersion: buildMeta.GetIndexVersion(), + IndexState: buildMeta.GetState(), + FailReason: buildMeta.GetFailReason(), + IsDeleted: buildMeta.GetMarkDeleted(), + CreatedUTCTime: record.GetCreateTime(), + IndexFileKeys: fileKeys, + IndexSerializedSize: buildMeta.GetSerializeSize(), + WriteHandoff: buildMeta.GetState() == commonpb.IndexState_Finished, } segmentIndexModels.AddRecord(segID, indexID, segmentIndexModel) } diff --git a/internal/core/src/common/protobuf_utils.cpp b/internal/core/src/common/protobuf_utils.cpp new file mode 100644 index 0000000000000..14972b0964614 --- /dev/null +++ b/internal/core/src/common/protobuf_utils.cpp @@ -0,0 +1,45 @@ +// Copyright (C) 2019-2020 Zilliz. All rights reserved. +// +// Licensed 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 + +#include "common/protobuf_utils.h" +#include "common/protobuf_utils_c.h" + +// Make a static_assert to ensure that the size and alignment of the C++ and C +static_assert( + sizeof(milvus::ProtoLayout) == sizeof(ProtoLayout), + "Size of milvus::ProtoLayout is not equal to size of ProtoLayoutInterface"); + +// Make a static_assert to ensure that the size and alignment of the C++ and C +static_assert(alignof(milvus::ProtoLayout) == alignof(ProtoLayout), + "Alignment of milvus::ProtoLayout is not equal to alignment of " + "ProtoLayoutInterface"); + +ProtoLayoutInterface +CreateProtoLayout() { + auto ptr = new milvus::ProtoLayout(); + return reinterpret_cast(ptr); +} + +void +ReleaseProtoLayout(ProtoLayoutInterface proto) { + delete reinterpret_cast(proto); +} + +namespace milvus { +ProtoLayout::ProtoLayout() : blob_(nullptr), size_(0) { +} + +ProtoLayout::~ProtoLayout() { + if (blob_ != nullptr) { + delete[] static_cast(blob_); + } +} +} // namespace milvus diff --git a/internal/core/src/common/protobuf_utils.h b/internal/core/src/common/protobuf_utils.h index 0ce6f8f5739bb..924a7b90c73a4 100644 --- a/internal/core/src/common/protobuf_utils.h +++ b/internal/core/src/common/protobuf_utils.h @@ -38,4 +38,46 @@ RepeatedKeyValToMap( } return mapping; } + +class ProtoLayout; +using ProtoLayoutPtr = std::unique_ptr; + +// ProtoLayout is a c++ type for esaier resource management at C-side. +// It's always keep same memory layout with ProtoLayout at C side for cgo call. +class ProtoLayout { + public: + ProtoLayout(); + + ProtoLayout(const ProtoLayout&) = delete; + + ProtoLayout(ProtoLayout&&) = delete; + + ProtoLayout& + operator=(const ProtoLayout&) = delete; + + ProtoLayout& + operator=(ProtoLayout&&) = delete; + + ~ProtoLayout(); + + // Serialize the proto into bytes and hold it in the layout. + // Return false if failure. + template + bool + SerializeAndHoldProto(T& proto) { + if (blob_ != nullptr || size_ != 0) { + throw std::runtime_error( + "ProtoLayout should always be empty " + "before calling SerializeAndHoldProto"); + } + size_ = proto.ByteSizeLong(); + blob_ = new uint8_t[size_]; + return proto.SerializeToArray(blob_, size_); + } + + private: + void* blob_; + size_t size_; +}; + } //namespace milvus diff --git a/internal/core/src/common/protobuf_utils_c.h b/internal/core/src/common/protobuf_utils_c.h new file mode 100644 index 0000000000000..48af575281c8d --- /dev/null +++ b/internal/core/src/common/protobuf_utils_c.h @@ -0,0 +1,40 @@ +// Copyright (C) 2019-2020 Zilliz. All rights reserved. +// +// Licensed 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 + +#pragma once +#ifdef __cplusplus +extern "C" { +#endif + +// ProtoLayout is a common ffi type for cgo call with serialized protobuf message. +// It's always keep same memory layout with milvus::ProtoLayout at C++ side. +typedef struct ProtoLayout { + void* blob; + size_t size; +} ProtoLayout; + +// ProtoLayoutInterface is the pointer alias for ProtoLayout. +// It should always created by CreateProtoLayout and released by ReleaseProtoLayout. +typedef struct ProtoLayout* ProtoLayoutInterface; + +// CreateProtoLayout is used to create an empty ProtoLayout. +// When you want to create a ProtoLayout at go-side, and return some data from C-side. +// You should use this API. +ProtoLayoutInterface +CreateProtoLayout(); + +void +ReleaseProtoLayout(ProtoLayoutInterface proto); + +#ifdef __cplusplus +} + +#endif diff --git a/internal/core/src/index/BitmapIndex.cpp b/internal/core/src/index/BitmapIndex.cpp index ff8d07d7f80e9..716faeb32e840 100644 --- a/internal/core/src/index/BitmapIndex.cpp +++ b/internal/core/src/index/BitmapIndex.cpp @@ -286,18 +286,15 @@ BitmapIndex::Serialize(const Config& config) { } template -BinarySet +IndexStatsPtr BitmapIndex::Upload(const Config& config) { auto binary_set = Serialize(config); file_manager_->AddFile(binary_set); auto remote_path_to_size = file_manager_->GetRemotePathsToFileSize(); - BinarySet ret; - for (auto& file : remote_path_to_size) { - ret.Append(file.first, nullptr, file.second); - } - return ret; + return IndexStats::NewFromSizeMap(file_manager_->GetAddedTotalMemSize(), + remote_path_to_size); } template diff --git a/internal/core/src/index/BitmapIndex.h b/internal/core/src/index/BitmapIndex.h index 6dcd1733b38d9..f14652278598e 100644 --- a/internal/core/src/index/BitmapIndex.h +++ b/internal/core/src/index/BitmapIndex.h @@ -114,7 +114,7 @@ class BitmapIndex : public ScalarIndex { return Count(); } - BinarySet + IndexStatsPtr Upload(const Config& config = {}) override; const bool diff --git a/internal/core/src/index/HybridScalarIndex.cpp b/internal/core/src/index/HybridScalarIndex.cpp index a70a70ed6f03e..304fea67065fd 100644 --- a/internal/core/src/index/HybridScalarIndex.cpp +++ b/internal/core/src/index/HybridScalarIndex.cpp @@ -298,7 +298,7 @@ HybridScalarIndex::SerializeIndexType() { } template -BinarySet +IndexStatsPtr HybridScalarIndex::Upload(const Config& config) { auto internal_index = GetInternalIndex(); auto index_ret = internal_index->Upload(config); @@ -306,9 +306,9 @@ HybridScalarIndex::Upload(const Config& config) { auto index_type_ret = SerializeIndexType(); for (auto& [key, value] : index_type_ret.binary_map_) { - index_ret.Append(key, value); + index_ret->AppendSerializedIndexFileInfo( + SerializedIndexFileInfo(key, value->size)); } - return index_ret; } diff --git a/internal/core/src/index/HybridScalarIndex.h b/internal/core/src/index/HybridScalarIndex.h index 8b6e484c2a71f..301f54fea0ccf 100644 --- a/internal/core/src/index/HybridScalarIndex.h +++ b/internal/core/src/index/HybridScalarIndex.h @@ -153,7 +153,7 @@ class HybridScalarIndex : public ScalarIndex { return internal_index_->HasRawData(); } - BinarySet + IndexStatsPtr Upload(const Config& config = {}) override; private: diff --git a/internal/core/src/index/Index.h b/internal/core/src/index/Index.h index bb136a10757b1..137a4feb63c6c 100644 --- a/internal/core/src/index/Index.h +++ b/internal/core/src/index/Index.h @@ -26,6 +26,7 @@ #include "common/Tracer.h" #include "common/Types.h" #include "index/Meta.h" +#include "index/IndexStats.h" namespace milvus::index { @@ -57,7 +58,7 @@ class IndexBase { virtual int64_t Count() = 0; - virtual BinarySet + virtual IndexStatsPtr Upload(const Config& config = {}) = 0; virtual const bool diff --git a/internal/core/src/index/IndexStats.cpp b/internal/core/src/index/IndexStats.cpp new file mode 100644 index 0000000000000..9dc229c0f0104 --- /dev/null +++ b/internal/core/src/index/IndexStats.cpp @@ -0,0 +1,81 @@ +// Copyright (C) 2019-2020 Zilliz. All rights reserved. +// +// Licensed 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 + +#include "index/IndexStats.h" + +namespace milvus::index { + +IndexStatsPtr +IndexStats::NewFromSizeMap(int64_t mem_size, + std::map& index_size_map) { + std::vector serialized_index_infos; + serialized_index_infos.reserve(index_size_map.size()); + for (auto& file : index_size_map) { + serialized_index_infos.emplace_back(file.first, file.second); + } + return IndexStats::New(mem_size, std::move(serialized_index_infos)); +} + +IndexStatsPtr +IndexStats::New(int64_t mem_size, + std::vector&& serialized_index_infos) { + return std::unique_ptr( + new IndexStats(mem_size, std::move(serialized_index_infos))); +} + +IndexStats::IndexStats( + int64_t mem_size, + std::vector&& serialized_index_infos) + : mem_size_(mem_size), serialized_index_infos_(serialized_index_infos) { +} + +void +IndexStats::AppendSerializedIndexFileInfo(SerializedIndexFileInfo&& info) { + serialized_index_infos_.push_back(std::move(info)); +} + +void +IndexStats::SerializeAt(milvus::ProtoLayout* layout) { + milvus::proto::cgo::IndexStats result; + result.set_mem_size(mem_size_); + for (auto& info : serialized_index_infos_) { + auto serialized_info = result.add_serialized_index_infos(); + serialized_info->set_file_name(info.file_name); + serialized_info->set_file_size(info.file_size); + } + AssertInfo(layout->SerializeAndHoldProto(result), + "marshal IndexStats failed"); +} + +std::vector +IndexStats::GetIndexFiles() const { + std::vector files; + for (auto& info : serialized_index_infos_) { + files.push_back(info.file_name); + } + return files; +} + +int64_t +IndexStats::GetMemSize() const { + return mem_size_; +} + +int64_t +IndexStats::GetSerializedSize() const { + int64_t size = 0; + for (auto& info : serialized_index_infos_) { + size += info.file_size; + } + return size; +} + +} // namespace milvus::index \ No newline at end of file diff --git a/internal/core/src/index/IndexStats.h b/internal/core/src/index/IndexStats.h new file mode 100644 index 0000000000000..4f1f116231819 --- /dev/null +++ b/internal/core/src/index/IndexStats.h @@ -0,0 +1,80 @@ +// Copyright (C) 2019-2020 Zilliz. All rights reserved. +// +// Licensed 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 + +#pragma once + +#include +#include +#include "common/protobuf_utils.h" +#include "pb/cgo_msg.pb.h" + +namespace milvus::index { + +class SerializedIndexFileInfo { + public: + SerializedIndexFileInfo(const std::string& file_name, int64_t file_size) + : file_name(file_name), file_size(file_size) { + } + + std::string file_name; + int64_t file_size; +}; + +class IndexStats; + +using IndexStatsPtr = std::unique_ptr; + +class IndexStats { + public: + static IndexStatsPtr + NewFromSizeMap(int64_t mem_size, + std::map& index_size_map); + + // Create a new IndexStats instance. + static IndexStatsPtr + New(int64_t mem_size, + std::vector&& serialized_index_infos); + + IndexStats(const IndexStats&) = delete; + + IndexStats(IndexStats&&) = delete; + + IndexStats& + operator=(const IndexStats&) = delete; + + IndexStats& + operator=(IndexStats&&) = delete; + + // Append a new serialized index file info into the result. + void + AppendSerializedIndexFileInfo(SerializedIndexFileInfo&& info); + + // Serialize the result into the target proto layout. + void + SerializeAt(milvus::ProtoLayout* layout); + + std::vector + GetIndexFiles() const; + + int64_t + GetMemSize() const; + + int64_t + GetSerializedSize() const; + + private: + IndexStats(int64_t mem_size, + std::vector&& serialized_index_infos); + + int64_t mem_size_; + std::vector serialized_index_infos_; +}; +} // namespace milvus::index diff --git a/internal/core/src/index/InvertedIndexTantivy.cpp b/internal/core/src/index/InvertedIndexTantivy.cpp index 4204fd05d8b3c..5d4dcf65a960a 100644 --- a/internal/core/src/index/InvertedIndexTantivy.cpp +++ b/internal/core/src/index/InvertedIndexTantivy.cpp @@ -136,7 +136,7 @@ InvertedIndexTantivy::Serialize(const Config& config) { } template -BinarySet +IndexStatsPtr InvertedIndexTantivy::Upload(const Config& config) { finish(); @@ -156,20 +156,25 @@ InvertedIndexTantivy::Upload(const Config& config) { } } - BinarySet ret; - auto remote_paths_to_size = disk_file_manager_->GetRemotePathsToFileSize(); - for (auto& file : remote_paths_to_size) { - ret.Append(file.first, nullptr, file.second); - } + auto binary_set = Serialize(config); mem_file_manager_->AddFile(binary_set); auto remote_mem_path_to_size = mem_file_manager_->GetRemotePathsToFileSize(); + + std::vector index_files; + index_files.reserve(remote_paths_to_size.size() + + remote_mem_path_to_size.size()); + for (auto& file : remote_paths_to_size) { + index_files.emplace_back(file.first, file.second); + } for (auto& file : remote_mem_path_to_size) { - ret.Append(file.first, nullptr, file.second); + index_files.emplace_back(file.first, file.second); } - return ret; + return IndexStats::New(mem_file_manager_->GetAddedTotalMemSize() + + disk_file_manager_->GetAddedTotalFileSize(), + std::move(index_files)); } template diff --git a/internal/core/src/index/InvertedIndexTantivy.h b/internal/core/src/index/InvertedIndexTantivy.h index e4bcccb0c5b72..4d61cb37b6550 100644 --- a/internal/core/src/index/InvertedIndexTantivy.h +++ b/internal/core/src/index/InvertedIndexTantivy.h @@ -89,7 +89,7 @@ class InvertedIndexTantivy : public ScalarIndex { BinarySet Serialize(const Config& config) override; - BinarySet + IndexStatsPtr Upload(const Config& config = {}) override; /* diff --git a/internal/core/src/index/ScalarIndexSort.cpp b/internal/core/src/index/ScalarIndexSort.cpp index e43d35902b6f0..1fd10d211869e 100644 --- a/internal/core/src/index/ScalarIndexSort.cpp +++ b/internal/core/src/index/ScalarIndexSort.cpp @@ -151,18 +151,14 @@ ScalarIndexSort::Serialize(const Config& config) { } template -BinarySet +IndexStatsPtr ScalarIndexSort::Upload(const Config& config) { auto binary_set = Serialize(config); file_manager_->AddFile(binary_set); auto remote_paths_to_size = file_manager_->GetRemotePathsToFileSize(); - BinarySet ret; - for (auto& file : remote_paths_to_size) { - ret.Append(file.first, nullptr, file.second); - } - - return ret; + return IndexStats::NewFromSizeMap(file_manager_->GetAddedTotalMemSize(), + remote_paths_to_size); } template diff --git a/internal/core/src/index/ScalarIndexSort.h b/internal/core/src/index/ScalarIndexSort.h index 1370b9dff89d6..0133fd711390e 100644 --- a/internal/core/src/index/ScalarIndexSort.h +++ b/internal/core/src/index/ScalarIndexSort.h @@ -90,7 +90,7 @@ class ScalarIndexSort : public ScalarIndex { return (int64_t)data_.size(); } - BinarySet + IndexStatsPtr Upload(const Config& config = {}) override; const bool diff --git a/internal/core/src/index/StringIndexMarisa.cpp b/internal/core/src/index/StringIndexMarisa.cpp index 289ba2409da86..3182a8bda5b1d 100644 --- a/internal/core/src/index/StringIndexMarisa.cpp +++ b/internal/core/src/index/StringIndexMarisa.cpp @@ -174,18 +174,14 @@ StringIndexMarisa::Serialize(const Config& config) { return res_set; } -BinarySet +IndexStatsPtr StringIndexMarisa::Upload(const Config& config) { auto binary_set = Serialize(config); file_manager_->AddFile(binary_set); auto remote_paths_to_size = file_manager_->GetRemotePathsToFileSize(); - BinarySet ret; - for (auto& file : remote_paths_to_size) { - ret.Append(file.first, nullptr, file.second); - } - - return ret; + return IndexStats::NewFromSizeMap(file_manager_->GetAddedTotalMemSize(), + remote_paths_to_size); } void diff --git a/internal/core/src/index/StringIndexMarisa.h b/internal/core/src/index/StringIndexMarisa.h index f3dff120897f0..8781255cd5405 100644 --- a/internal/core/src/index/StringIndexMarisa.h +++ b/internal/core/src/index/StringIndexMarisa.h @@ -92,7 +92,7 @@ class StringIndexMarisa : public StringIndex { std::optional Reverse_Lookup(size_t offset) const override; - BinarySet + IndexStatsPtr Upload(const Config& config = {}) override; const bool diff --git a/internal/core/src/index/TextMatchIndex.cpp b/internal/core/src/index/TextMatchIndex.cpp index 0422c3bf2a7fb..394bf78d582c5 100644 --- a/internal/core/src/index/TextMatchIndex.cpp +++ b/internal/core/src/index/TextMatchIndex.cpp @@ -78,7 +78,7 @@ TextMatchIndex::TextMatchIndex(const storage::FileManagerContext& ctx) d_type_ = TantivyDataType::Text; } -BinarySet +IndexStatsPtr TextMatchIndex::Upload(const Config& config) { finish(); @@ -98,21 +98,25 @@ TextMatchIndex::Upload(const Config& config) { } } - BinarySet ret; - auto remote_paths_to_size = disk_file_manager_->GetRemotePathsToFileSize(); - for (auto& file : remote_paths_to_size) { - ret.Append(file.first, nullptr, file.second); - } + auto binary_set = Serialize(config); mem_file_manager_->AddFile(binary_set); auto remote_mem_path_to_size = mem_file_manager_->GetRemotePathsToFileSize(); + + std::vector index_files; + index_files.reserve(remote_paths_to_size.size() + + remote_mem_path_to_size.size()); + for (auto& file : remote_paths_to_size) { + index_files.emplace_back(file.first, file.second); + } for (auto& file : remote_mem_path_to_size) { - ret.Append(file.first, nullptr, file.second); + index_files.emplace_back(file.first, file.second); } - - return ret; + return IndexStats::New(mem_file_manager_->GetAddedTotalMemSize() + + disk_file_manager_->GetAddedTotalFileSize(), + std::move(index_files)); } void diff --git a/internal/core/src/index/TextMatchIndex.h b/internal/core/src/index/TextMatchIndex.h index afdfef59ade19..edeee8b0be0f0 100644 --- a/internal/core/src/index/TextMatchIndex.h +++ b/internal/core/src/index/TextMatchIndex.h @@ -15,6 +15,7 @@ #include #include "index/InvertedIndexTantivy.h" +#include "index/IndexStats.h" namespace milvus::index { @@ -39,7 +40,7 @@ class TextMatchIndex : public InvertedIndexTantivy { explicit TextMatchIndex(const storage::FileManagerContext& ctx); public: - BinarySet + IndexStatsPtr Upload(const Config& config) override; void diff --git a/internal/core/src/index/VectorDiskIndex.cpp b/internal/core/src/index/VectorDiskIndex.cpp index e4be33267c153..2821d07c347b3 100644 --- a/internal/core/src/index/VectorDiskIndex.cpp +++ b/internal/core/src/index/VectorDiskIndex.cpp @@ -115,7 +115,7 @@ VectorDiskAnnIndex::Load(milvus::tracer::TraceContext ctx, } template -BinarySet +IndexStatsPtr VectorDiskAnnIndex::Upload(const Config& config) { BinarySet ret; auto stat = index_.Serialize(ret); @@ -124,11 +124,8 @@ VectorDiskAnnIndex::Upload(const Config& config) { "failed to serialize index, " + KnowhereStatusString(stat)); } auto remote_paths_to_size = file_manager_->GetRemotePathsToFileSize(); - for (auto& file : remote_paths_to_size) { - ret.Append(file.first, nullptr, file.second); - } - - return ret; + return IndexStats::NewFromSizeMap(file_manager_->GetAddedTotalFileSize(), + remote_paths_to_size); } template diff --git a/internal/core/src/index/VectorDiskIndex.h b/internal/core/src/index/VectorDiskIndex.h index d079bab4e51a9..f9164cd74a5a4 100644 --- a/internal/core/src/index/VectorDiskIndex.h +++ b/internal/core/src/index/VectorDiskIndex.h @@ -46,7 +46,7 @@ class VectorDiskAnnIndex : public VectorIndex { return binary_set; } - BinarySet + IndexStatsPtr Upload(const Config& config = {}) override; int64_t diff --git a/internal/core/src/index/VectorMemIndex.cpp b/internal/core/src/index/VectorMemIndex.cpp index 1e4ff425d6f4a..adfc2b5a930de 100644 --- a/internal/core/src/index/VectorMemIndex.cpp +++ b/internal/core/src/index/VectorMemIndex.cpp @@ -92,18 +92,14 @@ VectorMemIndex::VectorIterators(const milvus::DatasetPtr dataset, } template -BinarySet +IndexStatsPtr VectorMemIndex::Upload(const Config& config) { auto binary_set = Serialize(config); file_manager_->AddFile(binary_set); auto remote_paths_to_size = file_manager_->GetRemotePathsToFileSize(); - BinarySet ret; - for (auto& file : remote_paths_to_size) { - ret.Append(file.first, nullptr, file.second); - } - - return ret; + return IndexStats::NewFromSizeMap(file_manager_->GetAddedTotalMemSize(), + remote_paths_to_size); } template diff --git a/internal/core/src/index/VectorMemIndex.h b/internal/core/src/index/VectorMemIndex.h index 637c96879a751..35bbcd84c56b8 100644 --- a/internal/core/src/index/VectorMemIndex.h +++ b/internal/core/src/index/VectorMemIndex.h @@ -78,7 +78,7 @@ class VectorMemIndex : public VectorIndex { std::unique_ptr[]> GetSparseVector(const DatasetPtr dataset) const override; - BinarySet + IndexStatsPtr Upload(const Config& config = {}) override; knowhere::expected> diff --git a/internal/core/src/indexbuilder/IndexCreatorBase.h b/internal/core/src/indexbuilder/IndexCreatorBase.h index b6a2ac44b2779..cfe74095eb851 100644 --- a/internal/core/src/indexbuilder/IndexCreatorBase.h +++ b/internal/core/src/indexbuilder/IndexCreatorBase.h @@ -13,6 +13,7 @@ #include #include "common/Types.h" +#include "index/Index.h" #include "storage/FileManager.h" namespace milvus::indexbuilder { @@ -33,7 +34,7 @@ class IndexCreatorBase { virtual void Load(const milvus::BinarySet&) = 0; - virtual BinarySet + virtual index::IndexStatsPtr Upload() = 0; }; diff --git a/internal/core/src/indexbuilder/ScalarIndexCreator.cpp b/internal/core/src/indexbuilder/ScalarIndexCreator.cpp index 855be1476017f..c67cab550b4e1 100644 --- a/internal/core/src/indexbuilder/ScalarIndexCreator.cpp +++ b/internal/core/src/indexbuilder/ScalarIndexCreator.cpp @@ -63,7 +63,7 @@ ScalarIndexCreator::index_type() { return index_type_; } -BinarySet +index::IndexStatsPtr ScalarIndexCreator::Upload() { return index_->Upload(); } diff --git a/internal/core/src/indexbuilder/ScalarIndexCreator.h b/internal/core/src/indexbuilder/ScalarIndexCreator.h index 2ac34050f0323..3d32fc78e35cc 100644 --- a/internal/core/src/indexbuilder/ScalarIndexCreator.h +++ b/internal/core/src/indexbuilder/ScalarIndexCreator.h @@ -38,7 +38,7 @@ class ScalarIndexCreator : public IndexCreatorBase { void Load(const milvus::BinarySet&) override; - BinarySet + index::IndexStatsPtr Upload() override; private: diff --git a/internal/core/src/indexbuilder/VecIndexCreator.cpp b/internal/core/src/indexbuilder/VecIndexCreator.cpp index 789f10caa42dd..68afdff4ed8d7 100644 --- a/internal/core/src/indexbuilder/VecIndexCreator.cpp +++ b/internal/core/src/indexbuilder/VecIndexCreator.cpp @@ -84,7 +84,7 @@ VecIndexCreator::Query(const milvus::DatasetPtr& dataset, return search_result; } -BinarySet +index::IndexStatsPtr VecIndexCreator::Upload() { return index_->Upload(); } diff --git a/internal/core/src/indexbuilder/VecIndexCreator.h b/internal/core/src/indexbuilder/VecIndexCreator.h index d8d605f321dea..d9cb2ac054228 100644 --- a/internal/core/src/indexbuilder/VecIndexCreator.h +++ b/internal/core/src/indexbuilder/VecIndexCreator.h @@ -58,7 +58,7 @@ class VecIndexCreator : public IndexCreatorBase { const SearchInfo& search_info, const BitsetView& bitset); - BinarySet + index::IndexStatsPtr Upload() override; public: diff --git a/internal/core/src/indexbuilder/index_c.cpp b/internal/core/src/indexbuilder/index_c.cpp index df1cc9ae59c1a..65108a7b5b4fe 100644 --- a/internal/core/src/indexbuilder/index_c.cpp +++ b/internal/core/src/indexbuilder/index_c.cpp @@ -238,7 +238,7 @@ CreateIndex(CIndex* res_index, } CStatus -BuildTextIndex(CBinarySet* c_binary_set, +BuildTextIndex(ProtoLayoutInterface result, const uint8_t* serialized_build_index_info, const uint64_t len) { try { @@ -286,9 +286,9 @@ BuildTextIndex(CBinarySet* c_binary_set, "milvus_tokenizer", field_schema.get_analyzer_params().c_str()); index->Build(config); - auto binary = - std::make_unique(index->Upload(config)); - *c_binary_set = binary.release(); + auto create_index_result = index->Upload(config); + create_index_result->SerializeAt( + reinterpret_cast(result)); auto status = CStatus(); status.error_code = Success; status.error_msg = ""; @@ -776,7 +776,7 @@ AppendIndexStorageInfo(CBuildIndexInfo c_build_index_info, } CStatus -SerializeIndexAndUpLoad(CIndex index, CBinarySet* c_binary_set) { +SerializeIndexAndUpLoad(CIndex index, ProtoLayoutInterface result) { auto status = CStatus(); try { AssertInfo( @@ -784,9 +784,9 @@ SerializeIndexAndUpLoad(CIndex index, CBinarySet* c_binary_set) { "failed to serialize index to binary set, passed index was null"); auto real_index = reinterpret_cast(index); - auto binary = - std::make_unique(real_index->Upload()); - *c_binary_set = binary.release(); + auto create_index_result = real_index->Upload(); + create_index_result->SerializeAt( + reinterpret_cast(result)); status.error_code = Success; status.error_msg = ""; } catch (std::exception& e) { diff --git a/internal/core/src/indexbuilder/index_c.h b/internal/core/src/indexbuilder/index_c.h index 6d26adc3442d9..2af8e6d58944e 100644 --- a/internal/core/src/indexbuilder/index_c.h +++ b/internal/core/src/indexbuilder/index_c.h @@ -17,6 +17,7 @@ extern "C" { #include #include "common/type_c.h" +#include "common/protobuf_utils_c.h" #include "common/binary_set_c.h" #include "indexbuilder/type_c.h" @@ -36,7 +37,7 @@ CStatus DeleteIndex(CIndex index); CStatus -BuildTextIndex(CBinarySet* c_binary_set, +BuildTextIndex(ProtoLayoutInterface c_binary_set, const uint8_t* serialized_build_index_info, const uint64_t len); @@ -131,7 +132,7 @@ AppendOptionalFieldDataPath(CBuildIndexInfo c_build_index_info, const char* c_file_path); CStatus -SerializeIndexAndUpLoad(CIndex index, CBinarySet* c_binary_set); +SerializeIndexAndUpLoad(CIndex index, ProtoLayoutInterface result); CStatus AppendIndexStorageInfo(CBuildIndexInfo c_build_index_info, diff --git a/internal/core/src/segcore/Types.h b/internal/core/src/segcore/Types.h index 4f006f4a0d52c..c1cf8e524a6ad 100644 --- a/internal/core/src/segcore/Types.h +++ b/internal/core/src/segcore/Types.h @@ -47,7 +47,10 @@ struct LoadIndexInfo { int64_t index_store_version; IndexVersion index_engine_version; proto::schema::FieldSchema schema; - int64_t index_size; + int64_t index_size; // It's the size of index file before compressing + // (aka. the filesize before loading operation at knowhere), + // because the uncompressed-index-file-size may not be stored at previous milvus. + // so the size may be not accurate (generated by the compressed-index-file-size multiplied with a compress-ratio) }; } // namespace milvus::segcore diff --git a/internal/core/src/storage/DiskFileManagerImpl.cpp b/internal/core/src/storage/DiskFileManagerImpl.cpp index e54fa6d748825..c021ff0aa1377 100644 --- a/internal/core/src/storage/DiskFileManagerImpl.cpp +++ b/internal/core/src/storage/DiskFileManagerImpl.cpp @@ -94,6 +94,7 @@ DiskFileManagerImpl::AddFile(const std::string& file) noexcept { auto fileName = GetFileName(file); auto fileSize = local_chunk_manager->Size(file); + added_total_file_size_ += fileSize; std::vector batch_remote_files; std::vector remote_file_sizes; @@ -146,6 +147,7 @@ DiskFileManagerImpl::AddTextLog(const std::string& file) noexcept { auto fileName = GetFileName(file); auto fileSize = local_chunk_manager->Size(file); + added_total_file_size_ += fileSize; std::vector batch_remote_files; std::vector remote_file_sizes; diff --git a/internal/core/src/storage/DiskFileManagerImpl.h b/internal/core/src/storage/DiskFileManagerImpl.h index b2c87b1ff78db..32de1aa363b68 100644 --- a/internal/core/src/storage/DiskFileManagerImpl.h +++ b/internal/core/src/storage/DiskFileManagerImpl.h @@ -110,6 +110,11 @@ class DiskFileManagerImpl : public FileManagerImpl { return GetRemoteIndexObjectPrefix(); } + size_t + GetAddedTotalFileSize() const { + return added_total_file_size_; + } + private: int64_t GetIndexBuildId() { @@ -131,6 +136,8 @@ class DiskFileManagerImpl : public FileManagerImpl { // remote file path std::map remote_paths_to_size_; + + size_t added_total_file_size_ = 0; }; using DiskANNFileManagerImplPtr = std::shared_ptr; diff --git a/internal/core/src/storage/MemFileManagerImpl.cpp b/internal/core/src/storage/MemFileManagerImpl.cpp index a920708bf59f5..84a1c14e48fe7 100644 --- a/internal/core/src/storage/MemFileManagerImpl.cpp +++ b/internal/core/src/storage/MemFileManagerImpl.cpp @@ -73,6 +73,7 @@ MemFileManagerImpl::AddFile(const BinarySet& binary_set) { slice_sizes.emplace_back(iter->second->size); slice_names.emplace_back(remotePrefix + "/" + iter->first); batch_size += iter->second->size; + added_total_mem_size_ += iter->second->size; } if (data_slices.size() > 0) { diff --git a/internal/core/src/storage/MemFileManagerImpl.h b/internal/core/src/storage/MemFileManagerImpl.h index 8fffc1b387449..6b09b6254f24e 100644 --- a/internal/core/src/storage/MemFileManagerImpl.h +++ b/internal/core/src/storage/MemFileManagerImpl.h @@ -64,9 +64,16 @@ class MemFileManagerImpl : public FileManagerImpl { return remote_paths_to_size_; } + size_t + GetAddedTotalMemSize() const { + return added_total_mem_size_; + } + private: // remote file path std::map remote_paths_to_size_; + + size_t added_total_mem_size_ = 0; }; using MemFileManagerImplPtr = std::shared_ptr; diff --git a/internal/core/unittest/test_array_bitmap_index.cpp b/internal/core/unittest/test_array_bitmap_index.cpp index b6cad45ce8bdc..66205b113939f 100644 --- a/internal/core/unittest/test_array_bitmap_index.cpp +++ b/internal/core/unittest/test_array_bitmap_index.cpp @@ -236,10 +236,12 @@ class ArrayBitmapIndexTest : public testing::Test { DataType::ARRAY, config, ctx); build_index->Build(); - auto binary_set = build_index->Upload(); - for (const auto& [key, _] : binary_set.binary_map_) { - index_files.push_back(key); - } + auto create_index_result = build_index->Upload(); + auto memSize = create_index_result->GetMemSize(); + auto serializedSize = create_index_result->GetSerializedSize(); + ASSERT_GT(memSize, 0); + ASSERT_GT(serializedSize, 0); + index_files = create_index_result->GetIndexFiles(); } index::CreateIndexInfo index_info{}; diff --git a/internal/core/unittest/test_bitmap_index.cpp b/internal/core/unittest/test_bitmap_index.cpp index 50bc13aceec83..078546ae86b9e 100644 --- a/internal/core/unittest/test_bitmap_index.cpp +++ b/internal/core/unittest/test_bitmap_index.cpp @@ -134,7 +134,6 @@ class BitmapIndexTest : public testing::Test { log_path, serialized_bytes.data(), serialized_bytes.size()); storage::FileManagerContext ctx(field_meta, index_meta, chunk_manager_); - std::vector index_files; Config config; config["index_type"] = milvus::index::BITMAP_INDEX_TYPE; @@ -145,10 +144,12 @@ class BitmapIndexTest : public testing::Test { type_, config, ctx); build_index->Build(); - auto binary_set = build_index->Upload(); - for (const auto& [key, _] : binary_set.binary_map_) { - index_files.push_back(key); - } + auto create_index_result = build_index->Upload(); + auto memSize = create_index_result->GetMemSize(); + auto serializedSize = create_index_result->GetSerializedSize(); + ASSERT_GT(memSize, 0); + ASSERT_GT(serializedSize, 0); + auto index_files = create_index_result->GetIndexFiles(); index::CreateIndexInfo index_info{}; index_info.index_type = milvus::index::BITMAP_INDEX_TYPE; diff --git a/internal/core/unittest/test_hybrid_index.cpp b/internal/core/unittest/test_hybrid_index.cpp index a4a6661c20914..fb95b2c13f14a 100644 --- a/internal/core/unittest/test_hybrid_index.cpp +++ b/internal/core/unittest/test_hybrid_index.cpp @@ -149,10 +149,12 @@ class HybridIndexTestV1 : public testing::Test { type_, config, ctx); build_index->Build(); - auto binary_set = build_index->Upload(); - for (const auto& [key, _] : binary_set.binary_map_) { - index_files.push_back(key); - } + auto create_index_result = build_index->Upload(); + auto memSize = create_index_result->GetMemSize(); + auto serializedSize = create_index_result->GetSerializedSize(); + ASSERT_GT(memSize, 0); + ASSERT_GT(serializedSize, 0); + index_files = create_index_result->GetIndexFiles(); } index::CreateIndexInfo index_info{}; diff --git a/internal/core/unittest/test_indexing.cpp b/internal/core/unittest/test_indexing.cpp index 35656a5d6da2b..7b4bb72aefe1b 100644 --- a/internal/core/unittest/test_indexing.cpp +++ b/internal/core/unittest/test_indexing.cpp @@ -486,17 +486,18 @@ TEST_P(IndexTest, BuildAndQuery) { milvus::index::IndexBasePtr new_index; milvus::index::VectorIndex* vec_index = nullptr; - auto binary_set = index->Upload(); + auto create_index_result = index->Upload(); index.reset(); new_index = milvus::index::IndexFactory::GetInstance().CreateIndex( create_index_info, file_manager_context); vec_index = dynamic_cast(new_index.get()); - std::vector index_files; - for (auto& binary : binary_set.binary_map_) { - index_files.emplace_back(binary.first); - } + auto index_files = create_index_result->GetIndexFiles(); + auto memSize = create_index_result->GetMemSize(); + auto serializedSize = create_index_result->GetSerializedSize(); + ASSERT_GT(memSize, 0); + ASSERT_GT(serializedSize, 0); load_conf = generate_load_conf(index_type, metric_type, 0); load_conf["index_files"] = index_files; ASSERT_NO_THROW(vec_index->Load(milvus::tracer::TraceContext{}, load_conf)); @@ -551,7 +552,7 @@ TEST_P(IndexTest, Mmap) { milvus::index::IndexBasePtr new_index; milvus::index::VectorIndex* vec_index = nullptr; - auto binary_set = index->Upload(); + auto create_index_result = index->Upload(); index.reset(); new_index = milvus::index::IndexFactory::GetInstance().CreateIndex( @@ -561,10 +562,11 @@ TEST_P(IndexTest, Mmap) { } vec_index = dynamic_cast(new_index.get()); - std::vector index_files; - for (auto& binary : binary_set.binary_map_) { - index_files.emplace_back(binary.first); - } + auto index_files = create_index_result->GetIndexFiles(); + auto memSize = create_index_result->GetMemSize(); + auto serializedSize = create_index_result->GetSerializedSize(); + ASSERT_GT(memSize, 0); + ASSERT_GT(serializedSize, 0); load_conf = generate_load_conf(index_type, metric_type, 0); load_conf["index_files"] = index_files; load_conf["mmap_filepath"] = "mmap/test_index_mmap_" + index_type; @@ -610,24 +612,20 @@ TEST_P(IndexTest, GetVector) { milvus::index::IndexBasePtr new_index; milvus::index::VectorIndex* vec_index = nullptr; - auto binary_set = index->Upload(); + auto create_index_result = index->Upload(); index.reset(); - std::vector index_files; - for (auto& binary : binary_set.binary_map_) { - index_files.emplace_back(binary.first); - } + auto index_files = create_index_result->GetIndexFiles(); + auto memSize = create_index_result->GetMemSize(); + auto serializedSize = create_index_result->GetSerializedSize(); + ASSERT_GT(memSize, 0); + ASSERT_GT(serializedSize, 0); new_index = milvus::index::IndexFactory::GetInstance().CreateIndex( create_index_info, file_manager_context); load_conf = generate_load_conf(index_type, metric_type, 0); load_conf["index_files"] = index_files; vec_index = dynamic_cast(new_index.get()); - if (index_type == knowhere::IndexEnum::INDEX_DISKANN) { - vec_index->Load(binary_set, load_conf); - EXPECT_EQ(vec_index->Count(), NB); - } else { - vec_index->Load(milvus::tracer::TraceContext{}, load_conf); - } + vec_index->Load(milvus::tracer::TraceContext{}, load_conf); if (!is_sparse) { EXPECT_EQ(vec_index->GetDim(), DIM); } @@ -716,12 +714,13 @@ TEST_P(IndexTest, GetVector_EmptySparseVector) { milvus::index::IndexBasePtr new_index; milvus::index::VectorIndex* vec_index = nullptr; - auto binary_set = index->Upload(); + auto create_index_result = index->Upload(); index.reset(); - std::vector index_files; - for (auto& binary : binary_set.binary_map_) { - index_files.emplace_back(binary.first); - } + auto index_files = create_index_result->GetIndexFiles(); + auto memSize = create_index_result->GetMemSize(); + auto serializedSize = create_index_result->GetSerializedSize(); + ASSERT_GT(memSize, 0); + ASSERT_GT(serializedSize, 0); new_index = milvus::index::IndexFactory::GetInstance().CreateIndex( create_index_info, file_manager_context); load_conf = generate_load_conf(index_type, metric_type, 0); @@ -797,16 +796,17 @@ TEST(Indexing, SearchDiskAnnWithInvalidParam) { ASSERT_NO_THROW(index->BuildWithDataset(xb_dataset, build_conf)); // serialize and load disk index, disk index can only be search after loading for now - auto binary_set = index->Upload(); + auto create_index_result = index->Upload(); + auto memSize = create_index_result->GetMemSize(); + auto serializedSize = create_index_result->GetSerializedSize(); + ASSERT_GT(memSize, 0); + ASSERT_GT(serializedSize, 0); + auto index_files = create_index_result->GetIndexFiles(); index.reset(); auto new_index = milvus::index::IndexFactory::GetInstance().CreateIndex( create_index_info, file_manager_context); auto vec_index = dynamic_cast(new_index.get()); - std::vector index_files; - for (auto& binary : binary_set.binary_map_) { - index_files.emplace_back(binary.first); - } auto load_conf = generate_load_conf(index_type, metric_type, NB); load_conf["index_files"] = index_files; vec_index->Load(milvus::tracer::TraceContext{}, load_conf); @@ -880,16 +880,17 @@ TEST(Indexing, SearchDiskAnnWithFloat16) { ASSERT_NO_THROW(index->BuildWithDataset(xb_dataset, build_conf)); // serialize and load disk index, disk index can only be search after loading for now - auto binary_set = index->Upload(); + auto create_index_result = index->Upload(); + auto memSize = create_index_result->GetMemSize(); + auto serializedSize = create_index_result->GetSerializedSize(); + ASSERT_GT(memSize, 0); + ASSERT_GT(serializedSize, 0); + auto index_files = create_index_result->GetIndexFiles(); index.reset(); auto new_index = milvus::index::IndexFactory::GetInstance().CreateIndex( create_index_info, file_manager_context); auto vec_index = dynamic_cast(new_index.get()); - std::vector index_files; - for (auto& binary : binary_set.binary_map_) { - index_files.emplace_back(binary.first); - } auto load_conf = generate_load_conf(index_type, metric_type, NB); load_conf["index_files"] = index_files; vec_index->Load(milvus::tracer::TraceContext{}, load_conf); @@ -962,16 +963,17 @@ TEST(Indexing, SearchDiskAnnWithBFloat16) { ASSERT_NO_THROW(index->BuildWithDataset(xb_dataset, build_conf)); // serialize and load disk index, disk index can only be search after loading for now - auto binary_set = index->Upload(); + auto create_index_result = index->Upload(); + auto memSize = create_index_result->GetMemSize(); + auto serializedSize = create_index_result->GetSerializedSize(); + ASSERT_GT(memSize, 0); + ASSERT_GT(serializedSize, 0); + auto index_files = create_index_result->GetIndexFiles(); index.reset(); auto new_index = milvus::index::IndexFactory::GetInstance().CreateIndex( create_index_info, file_manager_context); auto vec_index = dynamic_cast(new_index.get()); - std::vector index_files; - for (auto& binary : binary_set.binary_map_) { - index_files.emplace_back(binary.first); - } auto load_conf = generate_load_conf(index_type, metric_type, NB); load_conf["index_files"] = index_files; vec_index->Load(milvus::tracer::TraceContext{}, load_conf); @@ -992,4 +994,21 @@ TEST(Indexing, SearchDiskAnnWithBFloat16) { SearchResult result; EXPECT_NO_THROW(vec_index->Query(xq_dataset, search_info, nullptr, result)); } -#endif \ No newline at end of file +#endif + +TEST(Indexing, IndexStats) { + using milvus::index::IndexStats; + using milvus::index::SerializedIndexFileInfo; + auto sized_map = + std::map{{"file1", 100}, {"file2", 200}}; + auto result = IndexStats::NewFromSizeMap(16, sized_map); + result->AppendSerializedIndexFileInfo( + SerializedIndexFileInfo{"file3", 300}); + auto files = result->GetIndexFiles(); + ASSERT_EQ(files.size(), 3); + ASSERT_EQ(files[0], "file1"); + ASSERT_EQ(files[1], "file2"); + ASSERT_EQ(files[2], "file3"); + ASSERT_EQ(result->GetMemSize(), 16); + ASSERT_EQ(result->GetSerializedSize(), 600); +} \ No newline at end of file diff --git a/internal/core/unittest/test_inverted_index.cpp b/internal/core/unittest/test_inverted_index.cpp index 6574174dc2da7..8da7f2c078405 100644 --- a/internal/core/unittest/test_inverted_index.cpp +++ b/internal/core/unittest/test_inverted_index.cpp @@ -193,10 +193,12 @@ test_run() { dtype, config, ctx); index->Build(); - auto bs = index->Upload(); - for (const auto& [key, _] : bs.binary_map_) { - index_files.push_back(key); - } + auto create_index_result = index->Upload(); + auto memSize = create_index_result->GetMemSize(); + auto serializedSize = create_index_result->GetSerializedSize(); + ASSERT_GT(memSize, 0); + ASSERT_GT(serializedSize, 0); + index_files = create_index_result->GetIndexFiles(); } { @@ -466,10 +468,12 @@ test_string() { dtype, config, ctx); index->Build(); - auto bs = index->Upload(); - for (const auto& [key, _] : bs.binary_map_) { - index_files.push_back(key); - } + auto create_index_result = index->Upload(); + auto memSize = create_index_result->GetMemSize(); + auto serializedSize = create_index_result->GetSerializedSize(); + ASSERT_GT(memSize, 0); + ASSERT_GT(serializedSize, 0); + index_files = create_index_result->GetIndexFiles(); } { diff --git a/internal/core/unittest/test_utils.cpp b/internal/core/unittest/test_utils.cpp index 7861dd9baab58..17e6510286c78 100644 --- a/internal/core/unittest/test_utils.cpp +++ b/internal/core/unittest/test_utils.cpp @@ -199,3 +199,9 @@ TEST(Util, dis_closer) { EXPECT_FALSE(milvus::query::dis_closer(0.1, 0.2, "IP")); EXPECT_FALSE(milvus::query::dis_closer(0.1, 0.1, "IP")); } + +TEST(Util, ProtoLayout) { + milvus::ProtoLayout layout; + milvus::proto::cgo::IndexStats result; + EXPECT_TRUE(layout.SerializeAndHoldProto(result)); +} \ No newline at end of file diff --git a/internal/core/unittest/test_utils/DataGen.h b/internal/core/unittest/test_utils/DataGen.h index 7f904f6e44659..6f943a315eb08 100644 --- a/internal/core/unittest/test_utils/DataGen.h +++ b/internal/core/unittest/test_utils/DataGen.h @@ -1165,12 +1165,8 @@ GenVecIndexing(int64_t N, knowhere::Version::GetCurrentVersion().VersionNumber(), file_manager_context); indexing->BuildWithDataset(database, conf); - auto binary_set = indexing->Upload(); - - std::vector index_files; - for (auto& binary : binary_set.binary_map_) { - index_files.emplace_back(binary.first); - } + auto create_index_result = indexing->Upload(); + auto index_files = create_index_result->GetIndexFiles(); conf["index_files"] = index_files; // we need a load stage to use index as the producation does // knowhere would do some data preparation in this stage diff --git a/internal/datacoord/compaction_trigger_test.go b/internal/datacoord/compaction_trigger_test.go index 53ecf91b5a5fc..c93491e339405 100644 --- a/internal/datacoord/compaction_trigger_test.go +++ b/internal/datacoord/compaction_trigger_test.go @@ -303,59 +303,59 @@ func Test_compactionTrigger_force(t *testing.T) { segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{ 1: { indexID: { - SegmentID: 1, - CollectionID: 2, - PartitionID: 1, - NumRows: 100, - IndexID: indexID, - BuildID: 1, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 0, - IndexFileKeys: nil, - IndexSize: 0, - WriteHandoff: false, + SegmentID: 1, + CollectionID: 2, + PartitionID: 1, + NumRows: 100, + IndexID: indexID, + BuildID: 1, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 0, + IndexFileKeys: nil, + IndexSerializedSize: 0, + WriteHandoff: false, }, }, 2: { indexID: { - SegmentID: 2, - CollectionID: 2, - PartitionID: 1, - NumRows: 100, - IndexID: indexID, - BuildID: 2, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 0, - IndexFileKeys: nil, - IndexSize: 0, - WriteHandoff: false, + SegmentID: 2, + CollectionID: 2, + PartitionID: 1, + NumRows: 100, + IndexID: indexID, + BuildID: 2, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 0, + IndexFileKeys: nil, + IndexSerializedSize: 0, + WriteHandoff: false, }, }, 3: { indexID: { - SegmentID: 3, - CollectionID: 1111, - PartitionID: 1, - NumRows: 100, - IndexID: indexID, - BuildID: 3, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 0, - IndexFileKeys: nil, - IndexSize: 0, - WriteHandoff: false, + SegmentID: 3, + CollectionID: 1111, + PartitionID: 1, + NumRows: 100, + IndexID: indexID, + BuildID: 3, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 0, + IndexFileKeys: nil, + IndexSerializedSize: 0, + WriteHandoff: false, }, }, }, diff --git a/internal/datacoord/garbage_collector_test.go b/internal/datacoord/garbage_collector_test.go index 5ac9f666c7c05..2e30c13117e75 100644 --- a/internal/datacoord/garbage_collector_test.go +++ b/internal/datacoord/garbage_collector_test.go @@ -487,40 +487,40 @@ func createMetaForRecycleUnusedSegIndexes(catalog metastore.DataCoordCatalog) *m segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{ segID: { indexID: { - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 1026, - IndexID: indexID, - BuildID: buildID, - NodeID: 1, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 10, - IndexFileKeys: []string{"file1", "file2"}, - IndexSize: 0, - WriteHandoff: false, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 1026, + IndexID: indexID, + BuildID: buildID, + NodeID: 1, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 10, + IndexFileKeys: []string{"file1", "file2"}, + IndexSerializedSize: 0, + WriteHandoff: false, }, }, segID + 1: { indexID: { - SegmentID: segID + 1, - CollectionID: collID, - PartitionID: partID, - NumRows: 1026, - IndexID: indexID, - BuildID: buildID + 1, - NodeID: 1, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 10, - IndexFileKeys: []string{"file1", "file2"}, - IndexSize: 0, - WriteHandoff: false, + SegmentID: segID + 1, + CollectionID: collID, + PartitionID: partID, + NumRows: 1026, + IndexID: indexID, + BuildID: buildID + 1, + NodeID: 1, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 10, + IndexFileKeys: []string{"file1", "file2"}, + IndexSerializedSize: 0, + WriteHandoff: false, }, }, }, @@ -532,37 +532,37 @@ func createMetaForRecycleUnusedSegIndexes(catalog metastore.DataCoordCatalog) *m } meta.indexMeta.segmentBuildInfo.Add(&model.SegmentIndex{ - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 1026, - IndexID: indexID, - BuildID: buildID, - NodeID: 1, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 10, - IndexFileKeys: []string{"file1", "file2"}, - IndexSize: 0, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 1026, + IndexID: indexID, + BuildID: buildID, + NodeID: 1, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 10, + IndexFileKeys: []string{"file1", "file2"}, + IndexSerializedSize: 0, }) meta.indexMeta.segmentBuildInfo.Add(&model.SegmentIndex{ - SegmentID: segID + 1, - CollectionID: collID, - PartitionID: partID, - NumRows: 1026, - IndexID: indexID, - BuildID: buildID + 1, - NodeID: 1, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 10, - IndexFileKeys: []string{"file1", "file2"}, - IndexSize: 0, + SegmentID: segID + 1, + CollectionID: collID, + PartitionID: partID, + NumRows: 1026, + IndexID: indexID, + BuildID: buildID + 1, + NodeID: 1, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 10, + IndexFileKeys: []string{"file1", "file2"}, + IndexSerializedSize: 0, }) for id, segment := range segments { @@ -652,40 +652,40 @@ func createMetaTableForRecycleUnusedIndexFiles(catalog *datacoord.Catalog) *meta segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{ segID: { indexID: { - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 1026, - IndexID: indexID, - BuildID: buildID, - NodeID: 1, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 10, - IndexFileKeys: []string{"file1", "file2"}, - IndexSize: 0, - WriteHandoff: false, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 1026, + IndexID: indexID, + BuildID: buildID, + NodeID: 1, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 10, + IndexFileKeys: []string{"file1", "file2"}, + IndexSerializedSize: 0, + WriteHandoff: false, }, }, segID + 1: { indexID: { - SegmentID: segID + 1, - CollectionID: collID, - PartitionID: partID, - NumRows: 1026, - IndexID: indexID, - BuildID: buildID + 1, - NodeID: 1, - IndexVersion: 1, - IndexState: commonpb.IndexState_InProgress, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 10, - IndexFileKeys: nil, - IndexSize: 0, - WriteHandoff: false, + SegmentID: segID + 1, + CollectionID: collID, + PartitionID: partID, + NumRows: 1026, + IndexID: indexID, + BuildID: buildID + 1, + NodeID: 1, + IndexVersion: 1, + IndexState: commonpb.IndexState_InProgress, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 10, + IndexFileKeys: nil, + IndexSerializedSize: 0, + WriteHandoff: false, }, }, }, @@ -710,38 +710,38 @@ func createMetaTableForRecycleUnusedIndexFiles(catalog *datacoord.Catalog) *meta }, } meta.indexMeta.segmentBuildInfo.Add(&model.SegmentIndex{ - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 1026, - IndexID: indexID, - BuildID: buildID, - NodeID: 1, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 10, - IndexFileKeys: []string{"file1", "file2"}, - IndexSize: 0, - WriteHandoff: false, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 1026, + IndexID: indexID, + BuildID: buildID, + NodeID: 1, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 10, + IndexFileKeys: []string{"file1", "file2"}, + IndexSerializedSize: 0, + WriteHandoff: false, }) meta.indexMeta.segmentBuildInfo.Add(&model.SegmentIndex{ - SegmentID: segID + 1, - CollectionID: collID, - PartitionID: partID, - NumRows: 1026, - IndexID: indexID, - BuildID: buildID + 1, - NodeID: 1, - IndexVersion: 1, - IndexState: commonpb.IndexState_InProgress, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 10, - IndexFileKeys: nil, - IndexSize: 0, - WriteHandoff: false, + SegmentID: segID + 1, + CollectionID: collID, + PartitionID: partID, + NumRows: 1026, + IndexID: indexID, + BuildID: buildID + 1, + NodeID: 1, + IndexVersion: 1, + IndexState: commonpb.IndexState_InProgress, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 10, + IndexFileKeys: nil, + IndexSerializedSize: 0, + WriteHandoff: false, }) for id, segment := range segments { meta.segments.SetSegment(id, segment) @@ -1052,40 +1052,40 @@ func TestGarbageCollector_clearETCD(t *testing.T) { segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{ segID: { indexID: { - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 5000, - IndexID: indexID, - BuildID: buildID, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 0, - IndexFileKeys: []string{"file1", "file2"}, - IndexSize: 1024, - WriteHandoff: false, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 5000, + IndexID: indexID, + BuildID: buildID, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 0, + IndexFileKeys: []string{"file1", "file2"}, + IndexSerializedSize: 1024, + WriteHandoff: false, }, }, segID + 1: { indexID: { - SegmentID: segID + 1, - CollectionID: collID, - PartitionID: partID, - NumRows: 5000, - IndexID: indexID, - BuildID: buildID + 1, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 0, - IndexFileKeys: []string{"file3", "file4"}, - IndexSize: 1024, - WriteHandoff: false, + SegmentID: segID + 1, + CollectionID: collID, + PartitionID: partID, + NumRows: 5000, + IndexID: indexID, + BuildID: buildID + 1, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 0, + IndexFileKeys: []string{"file3", "file4"}, + IndexSerializedSize: 1024, + WriteHandoff: false, }, }, }, @@ -1138,39 +1138,39 @@ func TestGarbageCollector_clearETCD(t *testing.T) { } m.indexMeta.segmentBuildInfo.Add(&model.SegmentIndex{ - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 5000, - IndexID: indexID, - BuildID: buildID, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 0, - IndexFileKeys: []string{"file1", "file2"}, - IndexSize: 1024, - WriteHandoff: false, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 5000, + IndexID: indexID, + BuildID: buildID, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 0, + IndexFileKeys: []string{"file1", "file2"}, + IndexSerializedSize: 1024, + WriteHandoff: false, }) m.indexMeta.segmentBuildInfo.Add(&model.SegmentIndex{ - SegmentID: segID + 1, - CollectionID: collID, - PartitionID: partID, - NumRows: 5000, - IndexID: indexID, - BuildID: buildID + 1, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 0, - IndexFileKeys: []string{"file3", "file4"}, - IndexSize: 1024, - WriteHandoff: false, + SegmentID: segID + 1, + CollectionID: collID, + PartitionID: partID, + NumRows: 5000, + IndexID: indexID, + BuildID: buildID + 1, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 0, + IndexFileKeys: []string{"file3", "file4"}, + IndexSerializedSize: 1024, + WriteHandoff: false, }) for id, segment := range segments { diff --git a/internal/datacoord/index_meta.go b/internal/datacoord/index_meta.go index f8bc0e0d70bc7..41057cd95a779 100644 --- a/internal/datacoord/index_meta.go +++ b/internal/datacoord/index_meta.go @@ -45,6 +45,7 @@ import ( "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/indexparams" "github.com/milvus-io/milvus/pkg/util/metricsinfo" + "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/timerecord" "github.com/milvus-io/milvus/pkg/util/tsoutil" "github.com/milvus-io/milvus/pkg/util/typeutil" @@ -74,7 +75,7 @@ func newIndexTaskStats(s *model.SegmentIndex) *metricsinfo.IndexTaskStats { BuildID: s.BuildID, IndexState: s.IndexState.String(), FailReason: s.FailReason, - IndexSize: s.IndexSize, + IndexSize: s.IndexMemSize, IndexVersion: s.IndexVersion, CreatedUTCTime: typeutil.TimestampToString(s.CreatedUTCTime * 1000), FinishedUTCTime: typeutil.TimestampToString(s.FinishedUTCTime * 1000), @@ -154,6 +155,9 @@ func (m *indexMeta) reloadFromKV() error { return err } for _, segIdx := range segmentIndexes { + if segIdx.IndexMemSize == 0 { + segIdx.IndexMemSize = segIdx.IndexSerializedSize * paramtable.Get().DataCoordCfg.IndexMemSizeEstimateMultiplier.GetAsUint64() + } m.updateSegmentIndex(segIdx) metrics.FlushedSegmentFileNum.WithLabelValues(metrics.IndexFileLabel).Observe(float64(len(segIdx.IndexFileKeys))) } @@ -788,7 +792,8 @@ func (m *indexMeta) FinishTask(taskInfo *workerpb.IndexTaskInfo) error { segIdx.IndexState = taskInfo.GetState() segIdx.IndexFileKeys = common.CloneStringList(taskInfo.GetIndexFileKeys()) segIdx.FailReason = taskInfo.GetFailReason() - segIdx.IndexSize = taskInfo.GetSerializedSize() + segIdx.IndexSerializedSize = taskInfo.GetSerializedSize() + segIdx.IndexMemSize = taskInfo.GetMemSize() segIdx.CurrentIndexVersion = taskInfo.GetCurrentIndexVersion() segIdx.FinishedUTCTime = uint64(time.Now().Unix()) return m.alterSegmentIndexes([]*model.SegmentIndex{segIdx}) @@ -885,8 +890,8 @@ func (m *indexMeta) SetStoredIndexFileSizeMetric(collections map[UniqueID]*colle coll, ok := collections[segmentIdx.CollectionID] if ok { metrics.DataCoordStoredIndexFilesSize.WithLabelValues(coll.DatabaseName, coll.Schema.GetName(), - fmt.Sprint(segmentIdx.CollectionID)).Add(float64(segmentIdx.IndexSize)) - total += segmentIdx.IndexSize + fmt.Sprint(segmentIdx.CollectionID)).Add(float64(segmentIdx.IndexSerializedSize)) + total += segmentIdx.IndexSerializedSize } } return total @@ -1135,7 +1140,7 @@ func (m *indexMeta) GetSegmentIndexedFields(collectionID UniqueID, segmentID Uni IndexFieldID: index.IndexID, IndexID: index.IndexID, BuildID: buildID, - IndexSize: int64(si.IndexSize), + IndexSize: int64(si.IndexSerializedSize), }) } } diff --git a/internal/datacoord/index_meta_test.go b/internal/datacoord/index_meta_test.go index fd3f76db286eb..55a7c0dacddb1 100644 --- a/internal/datacoord/index_meta_test.go +++ b/internal/datacoord/index_meta_test.go @@ -517,20 +517,20 @@ func TestMeta_AddSegmentIndex(t *testing.T) { } segmentIndex := &model.SegmentIndex{ - SegmentID: 1, - CollectionID: 2, - PartitionID: 3, - NumRows: 10240, - IndexID: 4, - BuildID: 5, - NodeID: 6, - IndexVersion: 0, - IndexState: 0, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 12, - IndexFileKeys: nil, - IndexSize: 0, + SegmentID: 1, + CollectionID: 2, + PartitionID: 3, + NumRows: 10240, + IndexID: 4, + BuildID: 5, + NodeID: 6, + IndexVersion: 0, + IndexState: 0, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 12, + IndexFileKeys: nil, + IndexSerializedSize: 0, } t.Run("save meta fail", func(t *testing.T) { @@ -664,20 +664,20 @@ func TestMeta_GetSegmentIndexState(t *testing.T) { t.Run("unissued", func(t *testing.T) { m.updateSegmentIndex(&model.SegmentIndex{ - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 10250, - IndexID: indexID, - BuildID: buildID, - NodeID: 1, - IndexVersion: 0, - IndexState: commonpb.IndexState_Unissued, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 12, - IndexFileKeys: nil, - IndexSize: 0, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 10250, + IndexID: indexID, + BuildID: buildID, + NodeID: 1, + IndexVersion: 0, + IndexState: commonpb.IndexState_Unissued, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 12, + IndexFileKeys: nil, + IndexSerializedSize: 0, }) state := m.GetSegmentIndexState(collID, segID, indexID) @@ -686,20 +686,20 @@ func TestMeta_GetSegmentIndexState(t *testing.T) { t.Run("finish", func(t *testing.T) { m.updateSegmentIndex(&model.SegmentIndex{ - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 10250, - IndexID: indexID, - BuildID: buildID, - NodeID: 1, - IndexVersion: 0, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 12, - IndexFileKeys: nil, - IndexSize: 0, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 10250, + IndexID: indexID, + BuildID: buildID, + NodeID: 1, + IndexVersion: 0, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 12, + IndexFileKeys: nil, + IndexSerializedSize: 0, }) state := m.GetSegmentIndexState(collID, segID, indexID) @@ -734,20 +734,20 @@ func TestMeta_GetIndexedSegment(t *testing.T) { m.segmentIndexes = map[UniqueID]map[UniqueID]*model.SegmentIndex{ segID: { indexID: { - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 1025, - IndexID: indexID, - BuildID: buildID, - NodeID: nodeID, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 10, - IndexFileKeys: nil, - IndexSize: 0, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 1025, + IndexID: indexID, + BuildID: buildID, + NodeID: nodeID, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 10, + IndexFileKeys: nil, + IndexSerializedSize: 0, }, }, } @@ -770,20 +770,20 @@ func TestMeta_GetIndexedSegment(t *testing.T) { } m.segmentBuildInfo.Add(&model.SegmentIndex{ - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 1025, - IndexID: indexID, - BuildID: buildID, - NodeID: nodeID, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 10, - IndexFileKeys: nil, - IndexSize: 0, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 1025, + IndexID: indexID, + BuildID: buildID, + NodeID: nodeID, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 10, + IndexFileKeys: nil, + IndexSerializedSize: 0, }) t.Run("success", func(t *testing.T) { @@ -1092,20 +1092,20 @@ func TestMeta_GetIndexParams(t *testing.T) { func TestMeta_GetIndexJob(t *testing.T) { m := newSegmentIndexMeta(nil) m.segmentBuildInfo.Add(&model.SegmentIndex{ - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 1025, - IndexID: indexID, - BuildID: buildID, - NodeID: 1, - IndexVersion: 1, - IndexState: commonpb.IndexState_Unissued, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 0, - IndexFileKeys: nil, - IndexSize: 0, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 1025, + IndexID: indexID, + BuildID: buildID, + NodeID: 1, + IndexVersion: 1, + IndexState: commonpb.IndexState_Unissued, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 0, + IndexFileKeys: nil, + IndexSerializedSize: 0, }) t.Run("exist", func(t *testing.T) { @@ -1180,20 +1180,20 @@ func updateSegmentIndexMeta(t *testing.T) *indexMeta { indexBuildInfo := newSegmentIndexBuildInfo() indexBuildInfo.Add(&model.SegmentIndex{ - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 1025, - IndexID: indexID, - BuildID: buildID, - NodeID: 0, - IndexVersion: 0, - IndexState: commonpb.IndexState_Unissued, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 0, - IndexFileKeys: nil, - IndexSize: 0, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 1025, + IndexID: indexID, + BuildID: buildID, + NodeID: 0, + IndexVersion: 0, + IndexState: commonpb.IndexState_Unissued, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 0, + IndexFileKeys: nil, + IndexSerializedSize: 0, }) return &indexMeta{ @@ -1201,20 +1201,20 @@ func updateSegmentIndexMeta(t *testing.T) *indexMeta { segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{ segID: { indexID: { - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 1025, - IndexID: indexID, - BuildID: buildID, - NodeID: 0, - IndexVersion: 0, - IndexState: commonpb.IndexState_Unissued, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 0, - IndexFileKeys: nil, - IndexSize: 0, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 1025, + IndexID: indexID, + BuildID: buildID, + NodeID: 0, + IndexVersion: 0, + IndexState: commonpb.IndexState_Unissued, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 0, + IndexFileKeys: nil, + IndexSerializedSize: 0, }, }, }, @@ -1521,26 +1521,26 @@ func TestIndexMeta_GetUnindexedSegments(t *testing.T) { func TestBuildIndexTaskStatsJSON(t *testing.T) { im := &indexMeta{segmentBuildInfo: newSegmentIndexBuildInfo()} si1 := &model.SegmentIndex{ - BuildID: 1, - CollectionID: 100, - SegmentID: 1000, - IndexID: 10, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IndexSize: 1024, - IndexVersion: 1, - CreatedUTCTime: uint64(time.Now().Unix()), + BuildID: 1, + CollectionID: 100, + SegmentID: 1000, + IndexID: 10, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IndexSerializedSize: 1024, + IndexVersion: 1, + CreatedUTCTime: uint64(time.Now().Unix()), } si2 := &model.SegmentIndex{ - BuildID: 2, - CollectionID: 101, - SegmentID: 1001, - IndexID: 11, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IndexSize: 2048, - IndexVersion: 1, - CreatedUTCTime: uint64(time.Now().Unix()), + BuildID: 2, + CollectionID: 101, + SegmentID: 1001, + IndexID: 11, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IndexSerializedSize: 2048, + IndexVersion: 1, + CreatedUTCTime: uint64(time.Now().Unix()), } actualJSON := im.TaskStatsJSON() @@ -1638,20 +1638,20 @@ func TestMeta_GetSegmentIndexStatus(t *testing.T) { m.segmentIndexes = map[UniqueID]map[UniqueID]*model.SegmentIndex{ segID: { indexID: { - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 10250, - IndexID: indexID, - BuildID: buildID, - NodeID: 1, - IndexVersion: 0, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 12, - IndexFileKeys: nil, - IndexSize: 0, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 10250, + IndexID: indexID, + BuildID: buildID, + NodeID: 1, + IndexVersion: 0, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 12, + IndexFileKeys: nil, + IndexSerializedSize: 0, }, }, segID + 1: {}, diff --git a/internal/datacoord/index_service.go b/internal/datacoord/index_service.go index 37432322bda59..81d9cdd8caa8c 100644 --- a/internal/datacoord/index_service.go +++ b/internal/datacoord/index_service.go @@ -932,7 +932,8 @@ func (s *Server) GetIndexInfos(ctx context.Context, req *indexpb.GetIndexInfoReq IndexName: s.meta.indexMeta.GetIndexNameByID(segIdx.CollectionID, segIdx.IndexID), IndexParams: indexParams, IndexFilePaths: indexFilePaths, - SerializedSize: segIdx.IndexSize, + SerializedSize: segIdx.IndexSerializedSize, + MemSize: segIdx.IndexMemSize, IndexVersion: segIdx.IndexVersion, NumRows: segIdx.NumRows, CurrentIndexVersion: segIdx.CurrentIndexVersion, diff --git a/internal/datacoord/index_service_test.go b/internal/datacoord/index_service_test.go index e120483065ab9..040fcfe124372 100644 --- a/internal/datacoord/index_service_test.go +++ b/internal/datacoord/index_service_test.go @@ -400,89 +400,89 @@ func TestServer_AlterIndex(t *testing.T) { segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{ segID: { indexID: { - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 10000, - IndexID: indexID, - BuildID: buildID, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: createTS, - IndexFileKeys: nil, - IndexSize: 0, - WriteHandoff: false, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 10000, + IndexID: indexID, + BuildID: buildID, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: createTS, + IndexFileKeys: nil, + IndexSerializedSize: 0, + WriteHandoff: false, }, indexID + 1: { - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 10000, - IndexID: indexID + 1, - BuildID: buildID + 1, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: createTS, - IndexFileKeys: nil, - IndexSize: 0, - WriteHandoff: false, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 10000, + IndexID: indexID + 1, + BuildID: buildID + 1, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: createTS, + IndexFileKeys: nil, + IndexSerializedSize: 0, + WriteHandoff: false, }, indexID + 3: { - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 10000, - IndexID: indexID + 3, - BuildID: buildID + 3, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_InProgress, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: createTS, - IndexFileKeys: nil, - IndexSize: 0, - WriteHandoff: false, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 10000, + IndexID: indexID + 3, + BuildID: buildID + 3, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_InProgress, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: createTS, + IndexFileKeys: nil, + IndexSerializedSize: 0, + WriteHandoff: false, }, indexID + 4: { - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 10000, - IndexID: indexID + 4, - BuildID: buildID + 4, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Failed, - FailReason: "mock failed", - IsDeleted: false, - CreatedUTCTime: createTS, - IndexFileKeys: nil, - IndexSize: 0, - WriteHandoff: false, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 10000, + IndexID: indexID + 4, + BuildID: buildID + 4, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Failed, + FailReason: "mock failed", + IsDeleted: false, + CreatedUTCTime: createTS, + IndexFileKeys: nil, + IndexSerializedSize: 0, + WriteHandoff: false, }, indexID + 5: { - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 10000, - IndexID: indexID + 5, - BuildID: buildID + 5, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Unissued, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: createTS, - IndexFileKeys: nil, - IndexSize: 0, - WriteHandoff: false, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 10000, + IndexID: indexID + 5, + BuildID: buildID + 5, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Unissued, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: createTS, + IndexFileKeys: nil, + IndexSerializedSize: 0, + WriteHandoff: false, }, }, segID - 1: { @@ -843,21 +843,21 @@ func TestServer_GetIndexState(t *testing.T) { segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{ segID: { indexID: { - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 3000, - IndexID: indexID, - BuildID: buildID, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_IndexStateNone, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 0, - IndexFileKeys: nil, - IndexSize: 0, - WriteHandoff: false, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 3000, + IndexID: indexID, + BuildID: buildID, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_IndexStateNone, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 0, + IndexFileKeys: nil, + IndexSerializedSize: 0, + WriteHandoff: false, }, }, }, @@ -968,21 +968,21 @@ func TestServer_GetSegmentIndexState(t *testing.T) { }, } s.meta.indexMeta.updateSegmentIndex(&model.SegmentIndex{ - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 10250, - IndexID: indexID, - BuildID: 10, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_InProgress, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: createTS, - IndexFileKeys: []string{"file1", "file2"}, - IndexSize: 1025, - WriteHandoff: false, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 10250, + IndexID: indexID, + BuildID: 10, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_InProgress, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: createTS, + IndexFileKeys: []string{"file1", "file2"}, + IndexSerializedSize: 1025, + WriteHandoff: false, }) s.meta.segments.SetSegment(segID, &SegmentInfo{ SegmentInfo: &datapb.SegmentInfo{ @@ -1005,21 +1005,21 @@ func TestServer_GetSegmentIndexState(t *testing.T) { t.Run("finish", func(t *testing.T) { s.meta.indexMeta.updateSegmentIndex(&model.SegmentIndex{ - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 10250, - IndexID: indexID, - BuildID: 10, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: createTS, - IndexFileKeys: []string{"file1", "file2"}, - IndexSize: 1025, - WriteHandoff: false, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 10250, + IndexID: indexID, + BuildID: 10, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: createTS, + IndexFileKeys: []string{"file1", "file2"}, + IndexSerializedSize: 1025, + WriteHandoff: false, }) resp, err := s.GetSegmentIndexState(ctx, req) assert.NoError(t, err) @@ -1127,21 +1127,21 @@ func TestServer_GetIndexBuildProgress(t *testing.T) { t.Run("finish", func(t *testing.T) { s.meta.indexMeta.updateSegmentIndex(&model.SegmentIndex{ - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 10250, - IndexID: indexID, - BuildID: 10, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: createTS, - IndexFileKeys: []string{"file1", "file2"}, - IndexSize: 0, - WriteHandoff: false, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 10250, + IndexID: indexID, + BuildID: 10, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: createTS, + IndexFileKeys: []string{"file1", "file2"}, + IndexSerializedSize: 0, + WriteHandoff: false, }) s.meta.segments = NewSegmentsInfo() s.meta.segments.SetSegment(segID, &SegmentInfo{ @@ -1389,89 +1389,89 @@ func TestServer_DescribeIndex(t *testing.T) { segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{ segID: { indexID: { - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 10000, - IndexID: indexID, - BuildID: buildID, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: createTS, - IndexFileKeys: nil, - IndexSize: 0, - WriteHandoff: false, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 10000, + IndexID: indexID, + BuildID: buildID, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: createTS, + IndexFileKeys: nil, + IndexSerializedSize: 0, + WriteHandoff: false, }, indexID + 1: { - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 10000, - IndexID: indexID + 1, - BuildID: buildID + 1, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: createTS, - IndexFileKeys: nil, - IndexSize: 0, - WriteHandoff: false, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 10000, + IndexID: indexID + 1, + BuildID: buildID + 1, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: createTS, + IndexFileKeys: nil, + IndexSerializedSize: 0, + WriteHandoff: false, }, indexID + 3: { - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 10000, - IndexID: indexID + 3, - BuildID: buildID + 3, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_InProgress, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: createTS, - IndexFileKeys: nil, - IndexSize: 0, - WriteHandoff: false, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 10000, + IndexID: indexID + 3, + BuildID: buildID + 3, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_InProgress, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: createTS, + IndexFileKeys: nil, + IndexSerializedSize: 0, + WriteHandoff: false, }, indexID + 4: { - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 10000, - IndexID: indexID + 4, - BuildID: buildID + 4, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Failed, - FailReason: "mock failed", - IsDeleted: false, - CreatedUTCTime: createTS, - IndexFileKeys: nil, - IndexSize: 0, - WriteHandoff: false, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 10000, + IndexID: indexID + 4, + BuildID: buildID + 4, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Failed, + FailReason: "mock failed", + IsDeleted: false, + CreatedUTCTime: createTS, + IndexFileKeys: nil, + IndexSerializedSize: 0, + WriteHandoff: false, }, indexID + 5: { - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 10000, - IndexID: indexID + 5, - BuildID: buildID + 5, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Unissued, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: createTS, - IndexFileKeys: nil, - IndexSize: 0, - WriteHandoff: false, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 10000, + IndexID: indexID + 5, + BuildID: buildID + 5, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Unissued, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: createTS, + IndexFileKeys: nil, + IndexSerializedSize: 0, + WriteHandoff: false, }, }, segID - 1: { @@ -1894,89 +1894,89 @@ func TestServer_GetIndexStatistics(t *testing.T) { segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{ segID: { indexID: { - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 10000, - IndexID: indexID, - BuildID: buildID, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: createTS, - IndexFileKeys: nil, - IndexSize: 0, - WriteHandoff: false, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 10000, + IndexID: indexID, + BuildID: buildID, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: createTS, + IndexFileKeys: nil, + IndexSerializedSize: 0, + WriteHandoff: false, }, indexID + 1: { - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 10000, - IndexID: indexID + 1, - BuildID: buildID + 1, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: createTS, - IndexFileKeys: nil, - IndexSize: 0, - WriteHandoff: false, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 10000, + IndexID: indexID + 1, + BuildID: buildID + 1, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: createTS, + IndexFileKeys: nil, + IndexSerializedSize: 0, + WriteHandoff: false, }, indexID + 3: { - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 10000, - IndexID: indexID + 3, - BuildID: buildID + 3, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_InProgress, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: createTS, - IndexFileKeys: nil, - IndexSize: 0, - WriteHandoff: false, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 10000, + IndexID: indexID + 3, + BuildID: buildID + 3, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_InProgress, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: createTS, + IndexFileKeys: nil, + IndexSerializedSize: 0, + WriteHandoff: false, }, indexID + 4: { - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 10000, - IndexID: indexID + 4, - BuildID: buildID + 4, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Failed, - FailReason: "mock failed", - IsDeleted: false, - CreatedUTCTime: createTS, - IndexFileKeys: nil, - IndexSize: 0, - WriteHandoff: false, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 10000, + IndexID: indexID + 4, + BuildID: buildID + 4, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Failed, + FailReason: "mock failed", + IsDeleted: false, + CreatedUTCTime: createTS, + IndexFileKeys: nil, + IndexSerializedSize: 0, + WriteHandoff: false, }, indexID + 5: { - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 10000, - IndexID: indexID + 5, - BuildID: buildID + 5, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Unissued, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: createTS, - IndexFileKeys: nil, - IndexSize: 0, - WriteHandoff: false, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 10000, + IndexID: indexID + 5, + BuildID: buildID + 5, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Unissued, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: createTS, + IndexFileKeys: nil, + IndexSerializedSize: 0, + WriteHandoff: false, }, }, }, @@ -2286,21 +2286,21 @@ func TestServer_GetIndexInfos(t *testing.T) { segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{ segID: { indexID: { - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 10000, - IndexID: indexID, - BuildID: buildID, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: createTS, - IndexFileKeys: nil, - IndexSize: 0, - WriteHandoff: false, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 10000, + IndexID: indexID, + BuildID: buildID, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: createTS, + IndexFileKeys: nil, + IndexSerializedSize: 0, + WriteHandoff: false, }, }, }, diff --git a/internal/datacoord/metrics_info_test.go b/internal/datacoord/metrics_info_test.go index 2438df1cd240d..b4577948e999e 100644 --- a/internal/datacoord/metrics_info_test.go +++ b/internal/datacoord/metrics_info_test.go @@ -706,20 +706,20 @@ func TestServer_getSegmentsJSON(t *testing.T) { segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{ 1000: { 10: &model.SegmentIndex{ - SegmentID: 1000, - CollectionID: 1, - PartitionID: 2, - NumRows: 10250, - IndexID: 10, - BuildID: 10000, - NodeID: 1, - IndexVersion: 0, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 12, - IndexFileKeys: nil, - IndexSize: 0, + SegmentID: 1000, + CollectionID: 1, + PartitionID: 2, + NumRows: 10250, + IndexID: 10, + BuildID: 10000, + NodeID: 1, + IndexVersion: 0, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 12, + IndexFileKeys: nil, + IndexSerializedSize: 0, }, }, }, diff --git a/internal/datacoord/server_test.go b/internal/datacoord/server_test.go index ece0e6539750b..ec1fc9396b4fe 100644 --- a/internal/datacoord/server_test.go +++ b/internal/datacoord/server_test.go @@ -1633,20 +1633,20 @@ func TestGetRecoveryInfo(t *testing.T) { }) assert.NoError(t, err) svr.meta.indexMeta.updateSegmentIndex(&model.SegmentIndex{ - SegmentID: seg4.ID, - CollectionID: 0, - PartitionID: 0, - NumRows: 100, - IndexID: 0, - BuildID: 0, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 0, - IndexFileKeys: nil, - IndexSize: 0, + SegmentID: seg4.ID, + CollectionID: 0, + PartitionID: 0, + NumRows: 100, + IndexID: 0, + BuildID: 0, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 0, + IndexFileKeys: nil, + IndexSerializedSize: 0, }) req := &datapb.GetRecoveryInfoRequest{ diff --git a/internal/datacoord/services_test.go b/internal/datacoord/services_test.go index 3203aa00699d5..9ee282c7b9aa5 100644 --- a/internal/datacoord/services_test.go +++ b/internal/datacoord/services_test.go @@ -1258,20 +1258,20 @@ func TestGetRecoveryInfoV2(t *testing.T) { }) assert.NoError(t, err) svr.meta.indexMeta.updateSegmentIndex(&model.SegmentIndex{ - SegmentID: seg4.ID, - CollectionID: 0, - PartitionID: 0, - NumRows: 100, - IndexID: 0, - BuildID: 0, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 0, - IndexFileKeys: nil, - IndexSize: 0, + SegmentID: seg4.ID, + CollectionID: 0, + PartitionID: 0, + NumRows: 100, + IndexID: 0, + BuildID: 0, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 0, + IndexFileKeys: nil, + IndexSerializedSize: 0, }) ch := &channelMeta{Name: "vchan1", CollectionID: 0} diff --git a/internal/datacoord/task_scheduler_test.go b/internal/datacoord/task_scheduler_test.go index 143a932caf343..d7122c807803a 100644 --- a/internal/datacoord/task_scheduler_test.go +++ b/internal/datacoord/task_scheduler_test.go @@ -58,190 +58,190 @@ var ( func createIndexMeta(catalog metastore.DataCoordCatalog) *indexMeta { indexBuildInfo := newSegmentIndexBuildInfo() indexBuildInfo.Add(&model.SegmentIndex{ - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 1025, - IndexID: indexID, - BuildID: buildID, - NodeID: 0, - IndexVersion: 0, - IndexState: commonpb.IndexState_Unissued, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 0, - IndexFileKeys: nil, - IndexSize: 1, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 1025, + IndexID: indexID, + BuildID: buildID, + NodeID: 0, + IndexVersion: 0, + IndexState: commonpb.IndexState_Unissued, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 0, + IndexFileKeys: nil, + IndexSerializedSize: 1, }) indexBuildInfo.Add(&model.SegmentIndex{ - SegmentID: segID + 1, - CollectionID: collID, - PartitionID: partID, - NumRows: 1026, - IndexID: indexID, - BuildID: buildID + 1, - NodeID: nodeID, - IndexVersion: 1, - IndexState: commonpb.IndexState_InProgress, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 1111, - IndexFileKeys: nil, - IndexSize: 1, + SegmentID: segID + 1, + CollectionID: collID, + PartitionID: partID, + NumRows: 1026, + IndexID: indexID, + BuildID: buildID + 1, + NodeID: nodeID, + IndexVersion: 1, + IndexState: commonpb.IndexState_InProgress, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 1111, + IndexFileKeys: nil, + IndexSerializedSize: 1, }) indexBuildInfo.Add(&model.SegmentIndex{ - SegmentID: segID + 2, - CollectionID: collID, - PartitionID: partID, - NumRows: 1026, - IndexID: indexID, - BuildID: buildID + 2, - NodeID: nodeID, - IndexVersion: 1, - IndexState: commonpb.IndexState_InProgress, - FailReason: "", - IsDeleted: true, - CreatedUTCTime: 1111, - IndexFileKeys: nil, - IndexSize: 1, + SegmentID: segID + 2, + CollectionID: collID, + PartitionID: partID, + NumRows: 1026, + IndexID: indexID, + BuildID: buildID + 2, + NodeID: nodeID, + IndexVersion: 1, + IndexState: commonpb.IndexState_InProgress, + FailReason: "", + IsDeleted: true, + CreatedUTCTime: 1111, + IndexFileKeys: nil, + IndexSerializedSize: 1, }) indexBuildInfo.Add(&model.SegmentIndex{ - SegmentID: segID + 3, - CollectionID: collID, - PartitionID: partID, - NumRows: 500, - IndexID: indexID, - BuildID: buildID + 3, - NodeID: 0, - IndexVersion: 0, - IndexState: commonpb.IndexState_Unissued, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 1111, - IndexFileKeys: nil, - IndexSize: 1, + SegmentID: segID + 3, + CollectionID: collID, + PartitionID: partID, + NumRows: 500, + IndexID: indexID, + BuildID: buildID + 3, + NodeID: 0, + IndexVersion: 0, + IndexState: commonpb.IndexState_Unissued, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 1111, + IndexFileKeys: nil, + IndexSerializedSize: 1, }) indexBuildInfo.Add(&model.SegmentIndex{ - SegmentID: segID + 4, - CollectionID: collID, - PartitionID: partID, - NumRows: 1026, - IndexID: indexID, - BuildID: buildID + 4, - NodeID: nodeID, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 1111, - IndexFileKeys: nil, - IndexSize: 1, + SegmentID: segID + 4, + CollectionID: collID, + PartitionID: partID, + NumRows: 1026, + IndexID: indexID, + BuildID: buildID + 4, + NodeID: nodeID, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 1111, + IndexFileKeys: nil, + IndexSerializedSize: 1, }) indexBuildInfo.Add(&model.SegmentIndex{ - SegmentID: segID + 5, - CollectionID: collID, - PartitionID: partID, - NumRows: 1026, - IndexID: indexID, - BuildID: buildID + 5, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 1111, - IndexFileKeys: nil, - IndexSize: 1, + SegmentID: segID + 5, + CollectionID: collID, + PartitionID: partID, + NumRows: 1026, + IndexID: indexID, + BuildID: buildID + 5, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 1111, + IndexFileKeys: nil, + IndexSerializedSize: 1, }) indexBuildInfo.Add(&model.SegmentIndex{ - SegmentID: segID + 6, - CollectionID: collID, - PartitionID: partID, - NumRows: 1026, - IndexID: indexID, - BuildID: buildID + 6, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 1111, - IndexFileKeys: nil, - IndexSize: 1, + SegmentID: segID + 6, + CollectionID: collID, + PartitionID: partID, + NumRows: 1026, + IndexID: indexID, + BuildID: buildID + 6, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 1111, + IndexFileKeys: nil, + IndexSerializedSize: 1, }) indexBuildInfo.Add(&model.SegmentIndex{ - SegmentID: segID + 7, - CollectionID: collID, - PartitionID: partID, - NumRows: 1026, - IndexID: indexID, - BuildID: buildID + 7, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Failed, - FailReason: "error", - IsDeleted: false, - CreatedUTCTime: 1111, - IndexFileKeys: nil, - IndexSize: 1, + SegmentID: segID + 7, + CollectionID: collID, + PartitionID: partID, + NumRows: 1026, + IndexID: indexID, + BuildID: buildID + 7, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Failed, + FailReason: "error", + IsDeleted: false, + CreatedUTCTime: 1111, + IndexFileKeys: nil, + IndexSerializedSize: 1, }) indexBuildInfo.Add(&model.SegmentIndex{ - SegmentID: segID + 8, - CollectionID: collID, - PartitionID: partID, - NumRows: 1026, - IndexID: indexID, - BuildID: buildID + 8, - NodeID: nodeID + 1, - IndexVersion: 1, - IndexState: commonpb.IndexState_InProgress, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 1111, - IndexFileKeys: nil, - IndexSize: 1, + SegmentID: segID + 8, + CollectionID: collID, + PartitionID: partID, + NumRows: 1026, + IndexID: indexID, + BuildID: buildID + 8, + NodeID: nodeID + 1, + IndexVersion: 1, + IndexState: commonpb.IndexState_InProgress, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 1111, + IndexFileKeys: nil, + IndexSerializedSize: 1, }) indexBuildInfo.Add(&model.SegmentIndex{ - SegmentID: segID + 9, - CollectionID: collID, - PartitionID: partID, - NumRows: 500, - IndexID: indexID, - BuildID: buildID + 9, - NodeID: 0, - IndexVersion: 0, - IndexState: commonpb.IndexState_Unissued, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 1111, - IndexFileKeys: nil, - IndexSize: 1, + SegmentID: segID + 9, + CollectionID: collID, + PartitionID: partID, + NumRows: 500, + IndexID: indexID, + BuildID: buildID + 9, + NodeID: 0, + IndexVersion: 0, + IndexState: commonpb.IndexState_Unissued, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 1111, + IndexFileKeys: nil, + IndexSerializedSize: 1, }) indexBuildInfo.Add(&model.SegmentIndex{ - SegmentID: segID + 10, - CollectionID: collID, - PartitionID: partID, - NumRows: 500, - IndexID: indexID, - BuildID: buildID + 10, - NodeID: nodeID, - IndexVersion: 0, - IndexState: commonpb.IndexState_Unissued, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 1111, - IndexFileKeys: nil, - IndexSize: 1, + SegmentID: segID + 10, + CollectionID: collID, + PartitionID: partID, + NumRows: 500, + IndexID: indexID, + BuildID: buildID + 10, + NodeID: nodeID, + IndexVersion: 0, + IndexState: commonpb.IndexState_Unissued, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 1111, + IndexFileKeys: nil, + IndexSerializedSize: 1, }) return &indexMeta{ @@ -278,200 +278,200 @@ func createIndexMeta(catalog metastore.DataCoordCatalog) *indexMeta { segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{ segID: { indexID: { - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: 1025, - IndexID: indexID, - BuildID: buildID, - NodeID: 0, - IndexVersion: 0, - IndexState: commonpb.IndexState_Unissued, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 0, - IndexFileKeys: nil, - IndexSize: 1, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: 1025, + IndexID: indexID, + BuildID: buildID, + NodeID: 0, + IndexVersion: 0, + IndexState: commonpb.IndexState_Unissued, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 0, + IndexFileKeys: nil, + IndexSerializedSize: 1, }, }, segID + 1: { indexID: { - SegmentID: segID + 1, - CollectionID: collID, - PartitionID: partID, - NumRows: 1026, - IndexID: indexID, - BuildID: buildID + 1, - NodeID: nodeID, - IndexVersion: 1, - IndexState: commonpb.IndexState_InProgress, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 1111, - IndexFileKeys: nil, - IndexSize: 1, + SegmentID: segID + 1, + CollectionID: collID, + PartitionID: partID, + NumRows: 1026, + IndexID: indexID, + BuildID: buildID + 1, + NodeID: nodeID, + IndexVersion: 1, + IndexState: commonpb.IndexState_InProgress, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 1111, + IndexFileKeys: nil, + IndexSerializedSize: 1, }, }, segID + 2: { indexID: { - SegmentID: segID + 2, - CollectionID: collID, - PartitionID: partID, - NumRows: 1026, - IndexID: indexID, - BuildID: buildID + 2, - NodeID: nodeID, - IndexVersion: 1, - IndexState: commonpb.IndexState_InProgress, - FailReason: "", - IsDeleted: true, - CreatedUTCTime: 1111, - IndexFileKeys: nil, - IndexSize: 1, + SegmentID: segID + 2, + CollectionID: collID, + PartitionID: partID, + NumRows: 1026, + IndexID: indexID, + BuildID: buildID + 2, + NodeID: nodeID, + IndexVersion: 1, + IndexState: commonpb.IndexState_InProgress, + FailReason: "", + IsDeleted: true, + CreatedUTCTime: 1111, + IndexFileKeys: nil, + IndexSerializedSize: 1, }, }, segID + 3: { indexID: { - SegmentID: segID + 3, - CollectionID: collID, - PartitionID: partID, - NumRows: 500, - IndexID: indexID, - BuildID: buildID + 3, - NodeID: 0, - IndexVersion: 0, - IndexState: commonpb.IndexState_Unissued, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 1111, - IndexFileKeys: nil, - IndexSize: 1, + SegmentID: segID + 3, + CollectionID: collID, + PartitionID: partID, + NumRows: 500, + IndexID: indexID, + BuildID: buildID + 3, + NodeID: 0, + IndexVersion: 0, + IndexState: commonpb.IndexState_Unissued, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 1111, + IndexFileKeys: nil, + IndexSerializedSize: 1, }, }, segID + 4: { indexID: { - SegmentID: segID + 4, - CollectionID: collID, - PartitionID: partID, - NumRows: 1026, - IndexID: indexID, - BuildID: buildID + 4, - NodeID: nodeID, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 1111, - IndexFileKeys: nil, - IndexSize: 1, + SegmentID: segID + 4, + CollectionID: collID, + PartitionID: partID, + NumRows: 1026, + IndexID: indexID, + BuildID: buildID + 4, + NodeID: nodeID, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 1111, + IndexFileKeys: nil, + IndexSerializedSize: 1, }, }, segID + 5: { indexID: { - SegmentID: segID + 5, - CollectionID: collID, - PartitionID: partID, - NumRows: 1026, - IndexID: indexID, - BuildID: buildID + 5, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 1111, - IndexFileKeys: nil, - IndexSize: 1, + SegmentID: segID + 5, + CollectionID: collID, + PartitionID: partID, + NumRows: 1026, + IndexID: indexID, + BuildID: buildID + 5, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 1111, + IndexFileKeys: nil, + IndexSerializedSize: 1, }, }, segID + 6: { indexID: { - SegmentID: segID + 6, - CollectionID: collID, - PartitionID: partID, - NumRows: 1026, - IndexID: indexID, - BuildID: buildID + 6, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 1111, - IndexFileKeys: nil, - IndexSize: 1, + SegmentID: segID + 6, + CollectionID: collID, + PartitionID: partID, + NumRows: 1026, + IndexID: indexID, + BuildID: buildID + 6, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 1111, + IndexFileKeys: nil, + IndexSerializedSize: 1, }, }, segID + 7: { indexID: { - SegmentID: segID + 7, - CollectionID: collID, - PartitionID: partID, - NumRows: 1026, - IndexID: indexID, - BuildID: buildID + 7, - NodeID: 0, - IndexVersion: 1, - IndexState: commonpb.IndexState_Failed, - FailReason: "error", - IsDeleted: false, - CreatedUTCTime: 1111, - IndexFileKeys: nil, - IndexSize: 1, + SegmentID: segID + 7, + CollectionID: collID, + PartitionID: partID, + NumRows: 1026, + IndexID: indexID, + BuildID: buildID + 7, + NodeID: 0, + IndexVersion: 1, + IndexState: commonpb.IndexState_Failed, + FailReason: "error", + IsDeleted: false, + CreatedUTCTime: 1111, + IndexFileKeys: nil, + IndexSerializedSize: 1, }, }, segID + 8: { indexID: { - SegmentID: segID + 8, - CollectionID: collID, - PartitionID: partID, - NumRows: 1026, - IndexID: indexID, - BuildID: buildID + 8, - NodeID: nodeID + 1, - IndexVersion: 1, - IndexState: commonpb.IndexState_InProgress, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 1111, - IndexFileKeys: nil, - IndexSize: 1, + SegmentID: segID + 8, + CollectionID: collID, + PartitionID: partID, + NumRows: 1026, + IndexID: indexID, + BuildID: buildID + 8, + NodeID: nodeID + 1, + IndexVersion: 1, + IndexState: commonpb.IndexState_InProgress, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 1111, + IndexFileKeys: nil, + IndexSerializedSize: 1, }, }, segID + 9: { indexID: { - SegmentID: segID + 9, - CollectionID: collID, - PartitionID: partID, - NumRows: 500, - IndexID: indexID, - BuildID: buildID + 9, - NodeID: 0, - IndexVersion: 0, - IndexState: commonpb.IndexState_Unissued, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 1111, - IndexFileKeys: nil, - IndexSize: 1, + SegmentID: segID + 9, + CollectionID: collID, + PartitionID: partID, + NumRows: 500, + IndexID: indexID, + BuildID: buildID + 9, + NodeID: 0, + IndexVersion: 0, + IndexState: commonpb.IndexState_Unissued, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 1111, + IndexFileKeys: nil, + IndexSerializedSize: 1, }, }, segID + 10: { indexID: { - SegmentID: segID + 10, - CollectionID: collID, - PartitionID: partID, - NumRows: 500, - IndexID: indexID, - BuildID: buildID + 10, - NodeID: nodeID, - IndexVersion: 0, - IndexState: commonpb.IndexState_Unissued, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 1111, - IndexFileKeys: nil, - IndexSize: 1, + SegmentID: segID + 10, + CollectionID: collID, + PartitionID: partID, + NumRows: 500, + IndexID: indexID, + BuildID: buildID + 10, + NodeID: nodeID, + IndexVersion: 0, + IndexState: commonpb.IndexState_Unissued, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 1111, + IndexFileKeys: nil, + IndexSerializedSize: 1, }, }, }, @@ -1541,20 +1541,20 @@ func (s *taskSchedulerSuite) Test_indexTaskWithMvOptionalScalarField() { segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{ segID: { indexID: { - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: minNumberOfRowsToBuild, - IndexID: indexID, - BuildID: buildID, - NodeID: 0, - IndexVersion: 0, - IndexState: commonpb.IndexState_Unissued, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 0, - IndexFileKeys: nil, - IndexSize: 0, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: minNumberOfRowsToBuild, + IndexID: indexID, + BuildID: buildID, + NodeID: 0, + IndexVersion: 0, + IndexState: commonpb.IndexState_Unissued, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 0, + IndexFileKeys: nil, + IndexSerializedSize: 0, }, }, }, @@ -1583,20 +1583,20 @@ func (s *taskSchedulerSuite) Test_indexTaskWithMvOptionalScalarField() { } mt.indexMeta.segmentBuildInfo.Add(&model.SegmentIndex{ - SegmentID: segID, - CollectionID: collID, - PartitionID: partID, - NumRows: minNumberOfRowsToBuild, - IndexID: indexID, - BuildID: buildID, - NodeID: 0, - IndexVersion: 0, - IndexState: commonpb.IndexState_Unissued, - FailReason: "", - IsDeleted: false, - CreatedUTCTime: 0, - IndexFileKeys: nil, - IndexSize: 0, + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + NumRows: minNumberOfRowsToBuild, + IndexID: indexID, + BuildID: buildID, + NodeID: 0, + IndexVersion: 0, + IndexState: commonpb.IndexState_Unissued, + FailReason: "", + IsDeleted: false, + CreatedUTCTime: 0, + IndexFileKeys: nil, + IndexSerializedSize: 0, }) cm := mocks.NewChunkManager(s.T()) cm.EXPECT().RootPath().Return("ut-index") diff --git a/internal/indexnode/indexnode_service.go b/internal/indexnode/indexnode_service.go index 98b5b8e08fdf7..4146a4efa7e31 100644 --- a/internal/indexnode/indexnode_service.go +++ b/internal/indexnode/indexnode_service.go @@ -132,6 +132,7 @@ func (i *IndexNode) QueryJobs(ctx context.Context, req *workerpb.QueryJobsReques state: info.state, fileKeys: common.CloneStringList(info.fileKeys), serializedSize: info.serializedSize, + memSize: info.memSize, failReason: info.failReason, currentIndexVersion: info.currentIndexVersion, indexStoreVersion: info.indexStoreVersion, @@ -154,6 +155,7 @@ func (i *IndexNode) QueryJobs(ctx context.Context, req *workerpb.QueryJobsReques ret.IndexInfos[i].State = info.state ret.IndexInfos[i].IndexFileKeys = info.fileKeys ret.IndexInfos[i].SerializedSize = info.serializedSize + ret.IndexInfos[i].MemSize = info.memSize ret.IndexInfos[i].FailReason = info.failReason ret.IndexInfos[i].CurrentIndexVersion = info.currentIndexVersion ret.IndexInfos[i].IndexStoreVersion = info.indexStoreVersion @@ -444,6 +446,7 @@ func (i *IndexNode) QueryJobsV2(ctx context.Context, req *workerpb.QueryJobsV2Re state: info.state, fileKeys: common.CloneStringList(info.fileKeys), serializedSize: info.serializedSize, + memSize: info.memSize, failReason: info.failReason, currentIndexVersion: info.currentIndexVersion, indexStoreVersion: info.indexStoreVersion, @@ -462,6 +465,7 @@ func (i *IndexNode) QueryJobsV2(ctx context.Context, req *workerpb.QueryJobsV2Re results[i].State = info.state results[i].IndexFileKeys = info.fileKeys results[i].SerializedSize = info.serializedSize + results[i].MemSize = info.memSize results[i].FailReason = info.failReason results[i].CurrentIndexVersion = info.currentIndexVersion results[i].IndexStoreVersion = info.indexStoreVersion diff --git a/internal/indexnode/task_index.go b/internal/indexnode/task_index.go index 4a316f3c909cb..d57507b06ffaa 100644 --- a/internal/indexnode/task_index.go +++ b/internal/indexnode/task_index.go @@ -332,7 +332,7 @@ func (it *indexBuildTask) PostExecute(ctx context.Context) error { log.Warn("IndexNode indexBuildTask Execute CIndexDelete failed", zap.Error(err)) } } - indexFilePath2Size, err := it.index.UpLoad() + indexStats, err := it.index.UpLoad() if err != nil { log.Warn("failed to upload index", zap.Error(err)) gcIndex() @@ -347,19 +347,21 @@ func (it *indexBuildTask) PostExecute(ctx context.Context) error { // use serialized size before encoding var serializedSize uint64 saveFileKeys := make([]string, 0) - for filePath, fileSize := range indexFilePath2Size { - serializedSize += uint64(fileSize) - parts := strings.Split(filePath, "/") + for _, indexInfo := range indexStats.GetSerializedIndexInfos() { + serializedSize += uint64(indexInfo.FileSize) + parts := strings.Split(indexInfo.FileName, "/") fileKey := parts[len(parts)-1] saveFileKeys = append(saveFileKeys, fileKey) } - it.node.storeIndexFilesAndStatistic(it.req.GetClusterID(), it.req.GetBuildID(), saveFileKeys, serializedSize, it.req.GetCurrentIndexVersion()) - log.Debug("save index files done", zap.Strings("IndexFiles", saveFileKeys)) + it.node.storeIndexFilesAndStatistic(it.req.GetClusterID(), it.req.GetBuildID(), saveFileKeys, serializedSize, uint64(indexStats.MemSize), it.req.GetCurrentIndexVersion()) saveIndexFileDur := it.tr.RecordSpan() metrics.IndexNodeSaveIndexFileLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(saveIndexFileDur.Seconds()) it.tr.Elapse("index building all done") - log.Info("Successfully save index files") + log.Info("Successfully save index files", + zap.Uint64("serializedSize", serializedSize), + zap.Int64("memSize", indexStats.MemSize), + zap.Strings("indexFiles", saveFileKeys)) return nil } diff --git a/internal/indexnode/taskinfo_ops.go b/internal/indexnode/taskinfo_ops.go index 122a58b011c59..d9760e043c948 100644 --- a/internal/indexnode/taskinfo_ops.go +++ b/internal/indexnode/taskinfo_ops.go @@ -34,6 +34,7 @@ type indexTaskInfo struct { state commonpb.IndexState fileKeys []string serializedSize uint64 + memSize uint64 failReason string currentIndexVersion int32 indexStoreVersion int64 @@ -90,6 +91,7 @@ func (i *IndexNode) storeIndexFilesAndStatistic( buildID UniqueID, fileKeys []string, serializedSize uint64, + memSize uint64, currentIndexVersion int32, ) { key := taskKey{ClusterID: ClusterID, TaskID: buildID} @@ -98,31 +100,12 @@ func (i *IndexNode) storeIndexFilesAndStatistic( if info, ok := i.indexTasks[key]; ok { info.fileKeys = common.CloneStringList(fileKeys) info.serializedSize = serializedSize + info.memSize = memSize info.currentIndexVersion = currentIndexVersion return } } -func (i *IndexNode) storeIndexFilesAndStatisticV2( - ClusterID string, - buildID UniqueID, - fileKeys []string, - serializedSize uint64, - currentIndexVersion int32, - indexStoreVersion int64, -) { - key := taskKey{ClusterID: ClusterID, TaskID: buildID} - i.stateLock.Lock() - defer i.stateLock.Unlock() - if info, ok := i.indexTasks[key]; ok { - info.fileKeys = common.CloneStringList(fileKeys) - info.serializedSize = serializedSize - info.currentIndexVersion = currentIndexVersion - info.indexStoreVersion = indexStoreVersion - return - } -} - func (i *IndexNode) deleteIndexTaskInfos(ctx context.Context, keys []taskKey) []*indexTaskInfo { i.stateLock.Lock() defer i.stateLock.Unlock() diff --git a/internal/metastore/kv/datacoord/kv_catalog_test.go b/internal/metastore/kv/datacoord/kv_catalog_test.go index e12a2d2da2ccc..edf2349f483e2 100644 --- a/internal/metastore/kv/datacoord/kv_catalog_test.go +++ b/internal/metastore/kv/datacoord/kv_catalog_test.go @@ -1013,20 +1013,20 @@ func TestCatalog_DropIndex(t *testing.T) { func TestCatalog_CreateSegmentIndex(t *testing.T) { segIdx := &model.SegmentIndex{ - SegmentID: 1, - CollectionID: 2, - PartitionID: 3, - NumRows: 1024, - IndexID: 4, - BuildID: 5, - NodeID: 6, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IndexVersion: 0, - IsDeleted: false, - CreatedUTCTime: 0, - IndexFileKeys: nil, - IndexSize: 0, + SegmentID: 1, + CollectionID: 2, + PartitionID: 3, + NumRows: 1024, + IndexID: 4, + BuildID: 5, + NodeID: 6, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IndexVersion: 0, + IsDeleted: false, + CreatedUTCTime: 0, + IndexFileKeys: nil, + IndexSerializedSize: 0, } t.Run("success", func(t *testing.T) { @@ -1113,20 +1113,20 @@ func TestCatalog_ListSegmentIndexes(t *testing.T) { func TestCatalog_AlterSegmentIndexes(t *testing.T) { segIdx := &model.SegmentIndex{ - SegmentID: 0, - CollectionID: 0, - PartitionID: 0, - NumRows: 0, - IndexID: 0, - BuildID: 0, - NodeID: 0, - IndexState: 0, - FailReason: "", - IndexVersion: 0, - IsDeleted: false, - CreatedUTCTime: 0, - IndexFileKeys: nil, - IndexSize: 0, + SegmentID: 0, + CollectionID: 0, + PartitionID: 0, + NumRows: 0, + IndexID: 0, + BuildID: 0, + NodeID: 0, + IndexState: 0, + FailReason: "", + IndexVersion: 0, + IsDeleted: false, + CreatedUTCTime: 0, + IndexFileKeys: nil, + IndexSerializedSize: 0, } t.Run("add", func(t *testing.T) { diff --git a/internal/metastore/model/segment_index.go b/internal/metastore/model/segment_index.go index 0fe7d2a8828a4..63eeed2121860 100644 --- a/internal/metastore/model/segment_index.go +++ b/internal/metastore/model/segment_index.go @@ -20,7 +20,12 @@ type SegmentIndex struct { IsDeleted bool CreatedUTCTime uint64 IndexFileKeys []string - IndexSize uint64 + // The byte size of serialized index data at oos. (compressed) + IndexSerializedSize uint64 + // The byte size of index data in memory. (uncompressed) + // The IndexMemSize may not be stored at old milvus implementation, so it may be not accurate. + // (generated by the IndexSerializedSize multiplied with a configured compress-ratio). + IndexMemSize uint64 // deprecated WriteHandoff bool CurrentIndexVersion int32 @@ -32,7 +37,6 @@ func UnmarshalSegmentIndexModel(segIndex *indexpb.SegmentIndex) *SegmentIndex { if segIndex == nil { return nil } - return &SegmentIndex{ SegmentID: segIndex.SegmentID, CollectionID: segIndex.CollectionID, @@ -47,7 +51,8 @@ func UnmarshalSegmentIndexModel(segIndex *indexpb.SegmentIndex) *SegmentIndex { IsDeleted: segIndex.Deleted, CreatedUTCTime: segIndex.CreateTime, IndexFileKeys: common.CloneStringList(segIndex.IndexFileKeys), - IndexSize: segIndex.SerializeSize, + IndexSerializedSize: segIndex.SerializeSize, + IndexMemSize: segIndex.MemSize, WriteHandoff: segIndex.WriteHandoff, CurrentIndexVersion: segIndex.GetCurrentIndexVersion(), FinishedUTCTime: segIndex.FinishedTime, @@ -73,7 +78,8 @@ func MarshalSegmentIndexModel(segIdx *SegmentIndex) *indexpb.SegmentIndex { IndexFileKeys: common.CloneStringList(segIdx.IndexFileKeys), Deleted: segIdx.IsDeleted, CreateTime: segIdx.CreatedUTCTime, - SerializeSize: segIdx.IndexSize, + SerializeSize: segIdx.IndexSerializedSize, + MemSize: segIdx.IndexMemSize, WriteHandoff: segIdx.WriteHandoff, CurrentIndexVersion: segIdx.CurrentIndexVersion, FinishedTime: segIdx.FinishedUTCTime, @@ -95,7 +101,8 @@ func CloneSegmentIndex(segIndex *SegmentIndex) *SegmentIndex { IsDeleted: segIndex.IsDeleted, CreatedUTCTime: segIndex.CreatedUTCTime, IndexFileKeys: common.CloneStringList(segIndex.IndexFileKeys), - IndexSize: segIndex.IndexSize, + IndexSerializedSize: segIndex.IndexSerializedSize, + IndexMemSize: segIndex.IndexMemSize, WriteHandoff: segIndex.WriteHandoff, CurrentIndexVersion: segIndex.CurrentIndexVersion, FinishedUTCTime: segIndex.FinishedUTCTime, diff --git a/internal/metastore/model/segment_index_test.go b/internal/metastore/model/segment_index_test.go index d038e14804a1c..8161dde7373f8 100644 --- a/internal/metastore/model/segment_index_test.go +++ b/internal/metastore/model/segment_index_test.go @@ -31,20 +31,20 @@ var ( } indexModel2 = &SegmentIndex{ - CollectionID: colID, - PartitionID: partID, - SegmentID: segmentID, - NumRows: 1025, - IndexID: indexID, - BuildID: buildID, - NodeID: 0, - IndexState: commonpb.IndexState_Finished, - FailReason: "", - IndexVersion: 0, - IsDeleted: false, - CreatedUTCTime: 1, - IndexFileKeys: nil, - IndexSize: 0, + CollectionID: colID, + PartitionID: partID, + SegmentID: segmentID, + NumRows: 1025, + IndexID: indexID, + BuildID: buildID, + NodeID: 0, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IndexVersion: 0, + IsDeleted: false, + CreatedUTCTime: 1, + IndexFileKeys: nil, + IndexSerializedSize: 0, } ) diff --git a/internal/querycoordv2/meta/coordinator_broker.go b/internal/querycoordv2/meta/coordinator_broker.go index b9ceab2ea46f6..b0524ae65f30f 100644 --- a/internal/querycoordv2/meta/coordinator_broker.go +++ b/internal/querycoordv2/meta/coordinator_broker.go @@ -356,7 +356,7 @@ func (broker *CoordinatorBroker) GetIndexInfo(ctx context.Context, collectionID BuildID: info.GetBuildID(), IndexParams: info.GetIndexParams(), IndexFilePaths: info.GetIndexFilePaths(), - IndexSize: int64(info.GetSerializedSize()), + IndexSize: int64(info.GetMemSize()), IndexVersion: info.GetIndexVersion(), NumRows: info.GetNumRows(), CurrentIndexVersion: info.GetCurrentIndexVersion(), diff --git a/internal/querynodev2/segments/segment.go b/internal/querynodev2/segments/segment.go index e2699608d8fc5..fd9c51157a288 100644 --- a/internal/querynodev2/segments/segment.go +++ b/internal/querynodev2/segments/segment.go @@ -965,8 +965,7 @@ func GetCLoadInfoWithFunc(ctx context.Context, IndexFiles: indexInfo.GetIndexFilePaths(), IndexEngineVersion: indexInfo.GetCurrentIndexVersion(), IndexStoreVersion: indexInfo.GetIndexStoreVersion(), - // TODO: For quickly fixing, we add the multiplier here, but those logic should be put at the datacoord after we add the mem size for each index. - IndexFileSize: int64(paramtable.Get().DataCoordCfg.IndexMemSizeEstimateMultiplier.GetAsFloat() * float64(indexInfo.GetIndexSize())), + IndexFileSize: indexInfo.GetIndexSize(), } // 2. diff --git a/internal/util/indexcgowrapper/index.go b/internal/util/indexcgowrapper/index.go index d126ab79bd72b..9c90662092944 100644 --- a/internal/util/indexcgowrapper/index.go +++ b/internal/util/indexcgowrapper/index.go @@ -5,6 +5,7 @@ package indexcgowrapper #include // free #include "indexbuilder/index_c.h" +#include "common/type_c.h" */ import "C" @@ -22,7 +23,9 @@ 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/storage" + "github.com/milvus-io/milvus/internal/util/segcore" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/cgopb" "github.com/milvus-io/milvus/pkg/proto/indexcgopb" ) @@ -40,7 +43,7 @@ type CodecIndex interface { Load([]*Blob) error Delete() error CleanLocalData() error - UpLoad() (map[string]int64, error) + UpLoad() (*cgopb.IndexStats, error) } var _ CodecIndex = (*CgoIndex)(nil) @@ -135,31 +138,21 @@ func CreateTextIndex(ctx context.Context, buildIndexInfo *indexcgopb.BuildIndexI zap.Error(err)) return nil, err } - var cBinarySet C.CBinarySet - status := C.BuildTextIndex(&cBinarySet, (*C.uint8_t)(unsafe.Pointer(&buildIndexInfoBlob[0])), (C.uint64_t)(len(buildIndexInfoBlob))) + result := C.CreateProtoLayout() + defer C.ReleaseProtoLayout(result) + status := C.BuildTextIndex(result, (*C.uint8_t)(unsafe.Pointer(&buildIndexInfoBlob[0])), (C.uint64_t)(len(buildIndexInfoBlob))) if err := HandleCStatus(&status, "failed to build text index"); err != nil { return nil, err } - - defer func() { - if cBinarySet != nil { - C.DeleteBinarySet(cBinarySet) - } - }() - - res := make(map[string]int64) - indexFilePaths, err := GetBinarySetKeys(cBinarySet) - if err != nil { + var indexStats cgopb.IndexStats + if err := segcore.UnmarshalProtoLayout(result, &indexStats); err != nil { return nil, err } - for _, path := range indexFilePaths { - size, err := GetBinarySetSize(cBinarySet, path) - if err != nil { - return nil, err - } - res[path] = size - } + res := make(map[string]int64) + for _, indexInfo := range indexStats.GetSerializedIndexInfos() { + res[indexInfo.FileName] = indexInfo.FileSize + } return res, nil } @@ -405,31 +398,17 @@ func (index *CgoIndex) CleanLocalData() error { return HandleCStatus(&status, "failed to clean cached data on disk") } -func (index *CgoIndex) UpLoad() (map[string]int64, error) { - var cBinarySet C.CBinarySet - - status := C.SerializeIndexAndUpLoad(index.indexPtr, &cBinarySet) - defer func() { - if cBinarySet != nil { - C.DeleteBinarySet(cBinarySet) - } - }() +func (index *CgoIndex) UpLoad() (*cgopb.IndexStats, error) { + result := C.CreateProtoLayout() + defer C.ReleaseProtoLayout(result) + status := C.SerializeIndexAndUpLoad(index.indexPtr, result) if err := HandleCStatus(&status, "failed to serialize index and upload index"); err != nil { return nil, err } - res := make(map[string]int64) - indexFilePaths, err := GetBinarySetKeys(cBinarySet) - if err != nil { + var indexStats cgopb.IndexStats + if err := segcore.UnmarshalProtoLayout(result, &indexStats); err != nil { return nil, err } - for _, path := range indexFilePaths { - size, err := GetBinarySetSize(cBinarySet, path) - if err != nil { - return nil, err - } - res[path] = size - } - - return res, nil + return &indexStats, nil } diff --git a/internal/util/segcore/cgo_util.go b/internal/util/segcore/cgo_util.go index a34c08657df13..c7cff7d847f98 100644 --- a/internal/util/segcore/cgo_util.go +++ b/internal/util/segcore/cgo_util.go @@ -21,6 +21,7 @@ package segcore #include "segcore/collection_c.h" #include "common/type_c.h" +#include "common/protobuf_utils_c.h" #include "segcore/segment_c.h" #include "storage/storage_c.h" */ @@ -29,6 +30,7 @@ import "C" import ( "context" "math" + "reflect" "unsafe" "google.golang.org/protobuf/proto" @@ -50,6 +52,13 @@ func ConsumeCStatusIntoError(status *C.CStatus) error { return merr.SegcoreError(int32(errorCode), errorMsg) } +func UnmarshalProtoLayout(protoLayout any, msg proto.Message) error { + layout := unsafe.Pointer(reflect.ValueOf(protoLayout).Pointer()) + cProtoLayout := (*C.ProtoLayout)(layout) + blob := (*(*[math.MaxInt32]byte)(cProtoLayout.blob))[:int(cProtoLayout.size):int(cProtoLayout.size)] + return proto.Unmarshal(blob, msg) +} + // unmarshalCProto unmarshal the proto from C memory func unmarshalCProto(cRes *C.CProto, msg proto.Message) error { blob := (*(*[math.MaxInt32]byte)(cRes.proto_blob))[:int(cRes.proto_size):int(cRes.proto_size)] diff --git a/internal/util/segcore/cgo_util_test.go b/internal/util/segcore/cgo_util_test.go index 3a517ede69d56..e2b8e997ab122 100644 --- a/internal/util/segcore/cgo_util_test.go +++ b/internal/util/segcore/cgo_util_test.go @@ -2,9 +2,13 @@ package segcore import ( "context" + "runtime" "testing" "github.com/stretchr/testify/assert" + "google.golang.org/protobuf/proto" + + "github.com/milvus-io/milvus/pkg/proto/cgopb" ) func TestConsumeCStatusIntoError(t *testing.T) { @@ -17,3 +21,28 @@ func TestGetLocalUsedSize(t *testing.T) { assert.NoError(t, err) assert.NotNil(t, size) } + +func TestProtoLayout(t *testing.T) { + layout := CreateProtoLayout() + testProto := cgopb.IndexStats{ + MemSize: 1024, + SerializedIndexInfos: []*cgopb.SerializedIndexFileInfo{ + { + FileName: "test", + FileSize: 768, + }, + }, + } + msg, err := proto.Marshal(&testProto) + defer runtime.KeepAlive(msg) + assert.NoError(t, err) + SetProtoLayout(layout, msg) + + resultProto := cgopb.IndexStats{} + UnmarshalProtoLayout(layout, &resultProto) + + assert.True(t, proto.Equal(&testProto, &resultProto)) + layout.blob = nil + layout.size = 0 + ReleaseProtoLayout(layout) +} diff --git a/internal/util/segcore/cgo_util_test_only.go b/internal/util/segcore/cgo_util_test_only.go new file mode 100644 index 0000000000000..e67667c60f6b3 --- /dev/null +++ b/internal/util/segcore/cgo_util_test_only.go @@ -0,0 +1,31 @@ +//go:build test +// +build test + +package segcore + +/* +#cgo pkg-config: milvus_core + +#include "common/protobuf_utils_c.h" +*/ +import "C" + +import ( + "reflect" + "unsafe" +) + +func CreateProtoLayout() *C.ProtoLayout { + ptr := C.CreateProtoLayout() + layout := unsafe.Pointer(reflect.ValueOf(ptr).Pointer()) + return (*C.ProtoLayout)(layout) +} + +func SetProtoLayout(protoLayout *C.ProtoLayout, slice []byte) { + protoLayout.blob = unsafe.Pointer(&slice[0]) + protoLayout.size = C.size_t(len(slice)) +} + +func ReleaseProtoLayout(protoLayout *C.ProtoLayout) { + C.ReleaseProtoLayout((C.ProtoLayoutInterface)(unsafe.Pointer(reflect.ValueOf(protoLayout).Pointer()))) +} diff --git a/pkg/proto/cgo_msg.proto b/pkg/proto/cgo_msg.proto index 7b7a516d9da96..6c286bba0fb17 100644 --- a/pkg/proto/cgo_msg.proto +++ b/pkg/proto/cgo_msg.proto @@ -22,3 +22,13 @@ message LoadIndexInfo { int32 index_engine_version = 15; int64 index_file_size = 16; } + +message IndexStats { + int64 mem_size = 1; + repeated SerializedIndexFileInfo serialized_index_infos = 3; +} + +message SerializedIndexFileInfo { + string file_name = 1; + int64 file_size = 2; +} diff --git a/pkg/proto/cgopb/cgo_msg.pb.go b/pkg/proto/cgopb/cgo_msg.pb.go index 91c1105e30aa8..0018995d86ced 100644 --- a/pkg/proto/cgopb/cgo_msg.pb.go +++ b/pkg/proto/cgopb/cgo_msg.pb.go @@ -180,6 +180,116 @@ func (x *LoadIndexInfo) GetIndexFileSize() int64 { return 0 } +type IndexStats struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + MemSize int64 `protobuf:"varint,1,opt,name=mem_size,json=memSize,proto3" json:"mem_size,omitempty"` + SerializedIndexInfos []*SerializedIndexFileInfo `protobuf:"bytes,3,rep,name=serialized_index_infos,json=serializedIndexInfos,proto3" json:"serialized_index_infos,omitempty"` +} + +func (x *IndexStats) Reset() { + *x = IndexStats{} + if protoimpl.UnsafeEnabled { + mi := &file_cgo_msg_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *IndexStats) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*IndexStats) ProtoMessage() {} + +func (x *IndexStats) ProtoReflect() protoreflect.Message { + mi := &file_cgo_msg_proto_msgTypes[1] + 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 IndexStats.ProtoReflect.Descriptor instead. +func (*IndexStats) Descriptor() ([]byte, []int) { + return file_cgo_msg_proto_rawDescGZIP(), []int{1} +} + +func (x *IndexStats) GetMemSize() int64 { + if x != nil { + return x.MemSize + } + return 0 +} + +func (x *IndexStats) GetSerializedIndexInfos() []*SerializedIndexFileInfo { + if x != nil { + return x.SerializedIndexInfos + } + return nil +} + +type SerializedIndexFileInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + FileName string `protobuf:"bytes,1,opt,name=file_name,json=fileName,proto3" json:"file_name,omitempty"` + FileSize int64 `protobuf:"varint,2,opt,name=file_size,json=fileSize,proto3" json:"file_size,omitempty"` +} + +func (x *SerializedIndexFileInfo) Reset() { + *x = SerializedIndexFileInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_cgo_msg_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SerializedIndexFileInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SerializedIndexFileInfo) ProtoMessage() {} + +func (x *SerializedIndexFileInfo) ProtoReflect() protoreflect.Message { + mi := &file_cgo_msg_proto_msgTypes[2] + 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 SerializedIndexFileInfo.ProtoReflect.Descriptor instead. +func (*SerializedIndexFileInfo) Descriptor() ([]byte, []int) { + return file_cgo_msg_proto_rawDescGZIP(), []int{2} +} + +func (x *SerializedIndexFileInfo) GetFileName() string { + if x != nil { + return x.FileName + } + return "" +} + +func (x *SerializedIndexFileInfo) GetFileSize() int64 { + if x != nil { + return x.FileSize + } + return 0 +} + var File_cgo_msg_proto protoreflect.FileDescriptor var file_cgo_msg_proto_rawDesc = []byte{ @@ -228,7 +338,21 @@ var file_cgo_msg_proto_rawDesc = []byte{ 0x78, 0x50, 0x61, 0x72, 0x61, 0x6d, 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, 0x42, 0x2d, 0x5a, 0x2b, 0x67, 0x69, 0x74, 0x68, + 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x88, 0x01, 0x0a, 0x0a, 0x49, 0x6e, 0x64, + 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x6d, 0x65, 0x6d, 0x5f, 0x73, + 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6d, 0x65, 0x6d, 0x53, 0x69, + 0x7a, 0x65, 0x12, 0x5f, 0x0a, 0x16, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, + 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x03, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x63, 0x67, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x46, 0x69, 0x6c, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x14, 0x73, + 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, + 0x66, 0x6f, 0x73, 0x22, 0x53, 0x0a, 0x17, 0x53, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, + 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x46, 0x69, 0x6c, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1b, + 0x0a, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x08, 0x66, 0x69, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x66, + 0x69, 0x6c, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, + 0x66, 0x69, 0x6c, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x42, 0x2d, 0x5a, 0x2b, 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, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x67, 0x6f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, @@ -246,20 +370,23 @@ func file_cgo_msg_proto_rawDescGZIP() []byte { return file_cgo_msg_proto_rawDescData } -var file_cgo_msg_proto_msgTypes = make([]protoimpl.MessageInfo, 2) +var file_cgo_msg_proto_msgTypes = make([]protoimpl.MessageInfo, 4) var file_cgo_msg_proto_goTypes = []interface{}{ - (*LoadIndexInfo)(nil), // 0: milvus.proto.cgo.LoadIndexInfo - nil, // 1: milvus.proto.cgo.LoadIndexInfo.IndexParamsEntry - (*schemapb.FieldSchema)(nil), // 2: milvus.proto.schema.FieldSchema + (*LoadIndexInfo)(nil), // 0: milvus.proto.cgo.LoadIndexInfo + (*IndexStats)(nil), // 1: milvus.proto.cgo.IndexStats + (*SerializedIndexFileInfo)(nil), // 2: milvus.proto.cgo.SerializedIndexFileInfo + nil, // 3: milvus.proto.cgo.LoadIndexInfo.IndexParamsEntry + (*schemapb.FieldSchema)(nil), // 4: milvus.proto.schema.FieldSchema } var file_cgo_msg_proto_depIdxs = []int32{ - 2, // 0: milvus.proto.cgo.LoadIndexInfo.field:type_name -> milvus.proto.schema.FieldSchema - 1, // 1: milvus.proto.cgo.LoadIndexInfo.index_params:type_name -> milvus.proto.cgo.LoadIndexInfo.IndexParamsEntry - 2, // [2:2] is the sub-list for method output_type - 2, // [2:2] is the sub-list for method input_type - 2, // [2:2] is the sub-list for extension type_name - 2, // [2:2] is the sub-list for extension extendee - 0, // [0:2] is the sub-list for field type_name + 4, // 0: milvus.proto.cgo.LoadIndexInfo.field:type_name -> milvus.proto.schema.FieldSchema + 3, // 1: milvus.proto.cgo.LoadIndexInfo.index_params:type_name -> milvus.proto.cgo.LoadIndexInfo.IndexParamsEntry + 2, // 2: milvus.proto.cgo.IndexStats.serialized_index_infos:type_name -> milvus.proto.cgo.SerializedIndexFileInfo + 3, // [3:3] is the sub-list for method output_type + 3, // [3:3] is the sub-list for method input_type + 3, // [3:3] is the sub-list for extension type_name + 3, // [3:3] is the sub-list for extension extendee + 0, // [0:3] is the sub-list for field type_name } func init() { file_cgo_msg_proto_init() } @@ -280,6 +407,30 @@ func file_cgo_msg_proto_init() { return nil } } + file_cgo_msg_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*IndexStats); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_cgo_msg_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SerializedIndexFileInfo); 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{ @@ -287,7 +438,7 @@ func file_cgo_msg_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_cgo_msg_proto_rawDesc, NumEnums: 0, - NumMessages: 2, + NumMessages: 4, NumExtensions: 0, NumServices: 0, }, diff --git a/pkg/proto/index_coord.proto b/pkg/proto/index_coord.proto index e13ffd46ac1e5..967624d6f5517 100644 --- a/pkg/proto/index_coord.proto +++ b/pkg/proto/index_coord.proto @@ -76,11 +76,12 @@ message SegmentIndex { repeated string index_file_keys = 11; bool deleted = 12; uint64 create_time = 13; - uint64 serialize_size = 14; + uint64 serialize_size = 14; // the total size of index file at oos. (may be compress) bool write_handoff = 15; int32 current_index_version = 16; int64 index_store_version = 17; uint64 finished_time = 18; + uint64 mem_size = 19; // the total size of index file at local (disk or memory) before loading by knowhere. (not compress) } message RegisterNodeRequest { @@ -160,6 +161,7 @@ message IndexFilePathInfo { int64 index_version = 9; int64 num_rows = 10; int32 current_index_version = 11; + uint64 mem_size = 12; } message SegmentInfo { diff --git a/pkg/proto/indexpb/index_coord.pb.go b/pkg/proto/indexpb/index_coord.pb.go index 7a206f4a8423e..a35ed846aa1b4 100644 --- a/pkg/proto/indexpb/index_coord.pb.go +++ b/pkg/proto/indexpb/index_coord.pb.go @@ -413,11 +413,12 @@ type SegmentIndex struct { IndexFileKeys []string `protobuf:"bytes,11,rep,name=index_file_keys,json=indexFileKeys,proto3" json:"index_file_keys,omitempty"` Deleted bool `protobuf:"varint,12,opt,name=deleted,proto3" json:"deleted,omitempty"` CreateTime uint64 `protobuf:"varint,13,opt,name=create_time,json=createTime,proto3" json:"create_time,omitempty"` - SerializeSize uint64 `protobuf:"varint,14,opt,name=serialize_size,json=serializeSize,proto3" json:"serialize_size,omitempty"` + SerializeSize uint64 `protobuf:"varint,14,opt,name=serialize_size,json=serializeSize,proto3" json:"serialize_size,omitempty"` // the total size of index file at oos. (may be compress) WriteHandoff bool `protobuf:"varint,15,opt,name=write_handoff,json=writeHandoff,proto3" json:"write_handoff,omitempty"` CurrentIndexVersion int32 `protobuf:"varint,16,opt,name=current_index_version,json=currentIndexVersion,proto3" json:"current_index_version,omitempty"` IndexStoreVersion int64 `protobuf:"varint,17,opt,name=index_store_version,json=indexStoreVersion,proto3" json:"index_store_version,omitempty"` FinishedTime uint64 `protobuf:"varint,18,opt,name=finished_time,json=finishedTime,proto3" json:"finished_time,omitempty"` + MemSize uint64 `protobuf:"varint,19,opt,name=mem_size,json=memSize,proto3" json:"mem_size,omitempty"` // the total size of index file at local (disk or memory) before loading by knowhere. (not compress) } func (x *SegmentIndex) Reset() { @@ -578,6 +579,13 @@ func (x *SegmentIndex) GetFinishedTime() uint64 { return 0 } +func (x *SegmentIndex) GetMemSize() uint64 { + if x != nil { + return x.MemSize + } + return 0 +} + type RegisterNodeRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1264,6 +1272,7 @@ type IndexFilePathInfo struct { IndexVersion int64 `protobuf:"varint,9,opt,name=index_version,json=indexVersion,proto3" json:"index_version,omitempty"` NumRows int64 `protobuf:"varint,10,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"` CurrentIndexVersion int32 `protobuf:"varint,11,opt,name=current_index_version,json=currentIndexVersion,proto3" json:"current_index_version,omitempty"` + MemSize uint64 `protobuf:"varint,12,opt,name=mem_size,json=memSize,proto3" json:"mem_size,omitempty"` } func (x *IndexFilePathInfo) Reset() { @@ -1375,6 +1384,13 @@ func (x *IndexFilePathInfo) GetCurrentIndexVersion() int32 { return 0 } +func (x *IndexFilePathInfo) GetMemSize() uint64 { + if x != nil { + return x.MemSize + } + return 0 +} + type SegmentInfo struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -2806,7 +2822,7 @@ var file_index_coord_proto_rawDesc = []byte{ 0x0a, 0x07, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x63, - 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x22, 0x8e, 0x05, 0x0a, 0x0c, 0x53, 0x65, + 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x22, 0xa9, 0x05, 0x0a, 0x0c, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, @@ -2847,283 +2863,196 @@ var file_index_coord_proto_rawDesc = []byte{ 0x28, 0x03, 0x52, 0x11, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x6f, 0x72, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0d, 0x66, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x12, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0c, 0x66, 0x69, - 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x22, 0x97, 0x01, 0x0a, 0x13, 0x52, - 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, 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, 0x36, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, - 0x02, 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, 0x41, 0x64, 0x64, 0x72, - 0x65, 0x73, 0x73, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x16, 0x0a, 0x06, - 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, - 0x64, 0x65, 0x49, 0x44, 0x22, 0x8f, 0x01, 0x0a, 0x14, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, - 0x72, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, - 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, - 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x12, 0x42, 0x0a, 0x0b, 0x69, 0x6e, 0x69, 0x74, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, - 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, - 0x49, 0x6e, 0x69, 0x74, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x52, 0x0a, 0x69, 0x6e, 0x69, 0x74, - 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x22, 0x59, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, - 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x22, - 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, - 0x65, 0x22, 0xa4, 0x01, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, - 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, - 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, + 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x6d, 0x65, + 0x6d, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x13, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x6d, 0x65, + 0x6d, 0x53, 0x69, 0x7a, 0x65, 0x22, 0x97, 0x01, 0x0a, 0x13, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, + 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, 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, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, - 0x12, 0x35, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, - 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, - 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, - 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, 0x5f, - 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, - 0x69, 0x6c, 0x52, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x22, 0x80, 0x01, 0x0a, 0x1b, 0x47, 0x65, 0x74, - 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, - 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, - 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, - 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x73, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, - 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x22, 0xa8, 0x01, 0x0a, 0x11, - 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, - 0x35, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, - 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, - 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72, - 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, 0x69, - 0x6c, 0x52, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, - 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, - 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x92, 0x01, 0x0a, 0x1c, 0x47, 0x65, 0x74, 0x53, 0x65, - 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, - 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, - 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x3d, 0x0a, 0x06, - 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, - 0x78, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, - 0x61, 0x74, 0x65, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x22, 0xdc, 0x03, 0x0a, 0x12, - 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, - 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, - 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x12, - 0x42, 0x0a, 0x0b, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x04, + 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, + 0x36, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x02, 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, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x52, 0x07, + 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, + 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x22, + 0x8f, 0x01, 0x0a, 0x14, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x42, 0x0a, + 0x0b, 0x69, 0x6e, 0x69, 0x74, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x50, + 0x61, 0x72, 0x61, 0x6d, 0x73, 0x52, 0x0a, 0x69, 0x6e, 0x69, 0x74, 0x50, 0x61, 0x72, 0x61, 0x6d, + 0x73, 0x22, 0x59, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1d, 0x0a, + 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xa4, 0x01, 0x0a, + 0x15, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x35, 0x0a, 0x05, 0x73, + 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, + 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72, 0x65, 0x61, 0x73, 0x6f, + 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x61, + 0x73, 0x6f, 0x6e, 0x22, 0x80, 0x01, 0x0a, 0x1b, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x44, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x22, 0xa8, 0x01, 0x0a, 0x11, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1c, 0x0a, 0x09, + 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x35, 0x0a, 0x05, 0x73, 0x74, + 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, + 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x61, 0x73, + 0x6f, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, + 0x65, 0x22, 0x92, 0x01, 0x0a, 0x1c, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, + 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x65, + 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x06, + 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x22, 0xdc, 0x03, 0x0a, 0x12, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x22, 0x0a, + 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x44, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x42, 0x0a, 0x0b, 0x74, 0x79, + 0x70, 0x65, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, + 0x69, 0x72, 0x52, 0x0a, 0x74, 0x79, 0x70, 0x65, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x44, + 0x0a, 0x0c, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, - 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0a, 0x74, 0x79, 0x70, 0x65, 0x50, 0x61, 0x72, - 0x61, 0x6d, 0x73, 0x12, 0x44, 0x0a, 0x0c, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x70, 0x61, 0x72, - 0x61, 0x6d, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, - 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0b, 0x69, 0x6e, - 0x64, 0x65, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, - 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x74, 0x69, - 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x22, 0x0a, 0x0d, 0x69, 0x73, 0x5f, 0x61, 0x75, - 0x74, 0x6f, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, - 0x69, 0x73, 0x41, 0x75, 0x74, 0x6f, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x4d, 0x0a, 0x11, 0x75, - 0x73, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, - 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, - 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0f, 0x75, 0x73, 0x65, 0x72, 0x49, - 0x6e, 0x64, 0x65, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x4e, 0x0a, 0x24, 0x75, 0x73, - 0x65, 0x72, 0x5f, 0x61, 0x75, 0x74, 0x6f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6d, 0x65, 0x74, - 0x72, 0x69, 0x63, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x69, 0x66, 0x69, - 0x65, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x08, 0x52, 0x20, 0x75, 0x73, 0x65, 0x72, 0x41, 0x75, - 0x74, 0x6f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x54, 0x79, 0x70, - 0x65, 0x53, 0x70, 0x65, 0x63, 0x69, 0x66, 0x69, 0x65, 0x64, 0x22, 0xb2, 0x01, 0x0a, 0x11, 0x41, - 0x6c, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, - 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, - 0x61, 0x6d, 0x65, 0x12, 0x39, 0x0a, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x03, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, - 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x1f, - 0x0a, 0x0b, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x04, 0x20, - 0x03, 0x28, 0x09, 0x52, 0x0a, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x22, - 0x78, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, - 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x65, - 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, - 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, - 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, - 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xab, 0x03, 0x0a, 0x11, 0x49, 0x6e, - 0x64, 0x65, 0x78, 0x46, 0x69, 0x6c, 0x65, 0x50, 0x61, 0x74, 0x68, 0x49, 0x6e, 0x66, 0x6f, 0x12, - 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x18, 0x0a, - 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, - 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x69, 0x6e, 0x64, 0x65, 0x78, - 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, - 0x44, 0x12, 0x18, 0x0a, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x03, 0x52, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x69, - 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x44, 0x0a, 0x0c, 0x69, 0x6e, - 0x64, 0x65, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, - 0x61, 0x69, 0x72, 0x52, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, - 0x12, 0x28, 0x0a, 0x10, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x70, - 0x61, 0x74, 0x68, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x69, 0x6e, 0x64, 0x65, - 0x78, 0x46, 0x69, 0x6c, 0x65, 0x50, 0x61, 0x74, 0x68, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x65, - 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x08, 0x20, - 0x01, 0x28, 0x04, 0x52, 0x0e, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x53, - 0x69, 0x7a, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x76, 0x65, 0x72, - 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x69, 0x6e, 0x64, 0x65, - 0x78, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, - 0x72, 0x6f, 0x77, 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, - 0x6f, 0x77, 0x73, 0x12, 0x32, 0x0a, 0x15, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x69, - 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x0b, 0x20, 0x01, - 0x28, 0x05, 0x52, 0x13, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, - 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0xba, 0x01, 0x0a, 0x0b, 0x53, 0x65, 0x67, 0x6d, - 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, - 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x73, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, - 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x21, 0x0a, 0x0c, 0x65, 0x6e, 0x61, - 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x0b, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x46, 0x0a, 0x0b, - 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x46, 0x69, 0x6c, 0x65, - 0x50, 0x61, 0x74, 0x68, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, - 0x6e, 0x66, 0x6f, 0x73, 0x22, 0x8a, 0x02, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, - 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, - 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, - 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x12, 0x5c, 0x0a, 0x0c, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x6e, - 0x66, 0x6f, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, - 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x52, 0x0b, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, - 0x1a, 0x5f, 0x0a, 0x10, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x45, - 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x35, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, - 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, - 0x01, 0x22, 0x94, 0x01, 0x0a, 0x10, 0x44, 0x72, 0x6f, 0x70, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, + 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x50, 0x61, + 0x72, 0x61, 0x6d, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, + 0x6d, 0x70, 0x12, 0x22, 0x0a, 0x0d, 0x69, 0x73, 0x5f, 0x61, 0x75, 0x74, 0x6f, 0x5f, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x69, 0x73, 0x41, 0x75, 0x74, + 0x6f, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x4d, 0x0a, 0x11, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, + 0x50, 0x61, 0x69, 0x72, 0x52, 0x0f, 0x75, 0x73, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x50, + 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x4e, 0x0a, 0x24, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x61, 0x75, + 0x74, 0x6f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x5f, 0x74, + 0x79, 0x70, 0x65, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x69, 0x66, 0x69, 0x65, 0x64, 0x18, 0x09, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x20, 0x75, 0x73, 0x65, 0x72, 0x41, 0x75, 0x74, 0x6f, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x54, 0x79, 0x70, 0x65, 0x53, 0x70, 0x65, 0x63, + 0x69, 0x66, 0x69, 0x65, 0x64, 0x22, 0xb2, 0x01, 0x0a, 0x11, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x49, + 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x22, 0x0a, 0x0c, 0x63, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, + 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x39, + 0x0a, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, + 0x72, 0x52, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x64, 0x65, 0x6c, + 0x65, 0x74, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, + 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x22, 0x78, 0x0a, 0x13, 0x47, 0x65, + 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xc6, 0x03, 0x0a, 0x11, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x46, 0x69, + 0x6c, 0x65, 0x50, 0x61, 0x74, 0x68, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x65, 0x6c, + 0x64, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, + 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x07, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, + 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x62, + 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x44, 0x0a, 0x0c, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x70, + 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0b, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x28, 0x0a, 0x10, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x73, 0x18, + 0x07, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x46, 0x69, 0x6c, 0x65, + 0x50, 0x61, 0x74, 0x68, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, + 0x7a, 0x65, 0x64, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0e, + 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x23, + 0x0a, 0x0d, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, + 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x56, 0x65, 0x72, 0x73, + 0x69, 0x6f, 0x6e, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, + 0x0a, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x32, + 0x0a, 0x15, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, + 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x05, 0x52, 0x13, 0x63, + 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x56, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x12, 0x19, 0x0a, 0x08, 0x6d, 0x65, 0x6d, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x0c, + 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x6d, 0x65, 0x6d, 0x53, 0x69, 0x7a, 0x65, 0x22, 0xba, 0x01, + 0x0a, 0x0b, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x22, 0x0a, + 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, + 0x21, 0x0a, 0x0c, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x64, + 0x65, 0x78, 0x12, 0x46, 0x0a, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x69, 0x6e, 0x66, 0x6f, + 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, 0x6e, 0x64, + 0x65, 0x78, 0x46, 0x69, 0x6c, 0x65, 0x50, 0x61, 0x74, 0x68, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x22, 0x8a, 0x02, 0x0a, 0x14, 0x47, + 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x5c, 0x0a, 0x0c, 0x73, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x39, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x49, 0x6e, 0x66, 0x6f, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0b, 0x73, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0x5f, 0x0a, 0x10, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x35, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x94, 0x01, 0x0a, 0x10, 0x44, 0x72, 0x6f, 0x70, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x22, 0x0a, 0x0c, + 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, + 0x12, 0x22, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x44, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, + 0x61, 0x6d, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x64, 0x72, 0x6f, 0x70, 0x5f, 0x61, 0x6c, 0x6c, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x64, 0x72, 0x6f, 0x70, 0x41, 0x6c, 0x6c, 0x22, 0x77, + 0x0a, 0x14, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, - 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x70, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, - 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x12, 0x1d, - 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x19, 0x0a, - 0x08, 0x64, 0x72, 0x6f, 0x70, 0x5f, 0x61, 0x6c, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x07, 0x64, 0x72, 0x6f, 0x70, 0x41, 0x6c, 0x6c, 0x22, 0x77, 0x0a, 0x14, 0x44, 0x65, 0x73, 0x63, - 0x72, 0x69, 0x62, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, - 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, - 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, - 0x70, 0x22, 0x8c, 0x01, 0x0a, 0x15, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x49, 0x6e, - 0x64, 0x65, 0x78, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, - 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, - 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, - 0x12, 0x3e, 0x0a, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, - 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, - 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x73, - 0x22, 0x61, 0x0a, 0x1c, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, - 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, - 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, - 0x61, 0x6d, 0x65, 0x22, 0xc4, 0x01, 0x0a, 0x1d, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, - 0x42, 0x75, 0x69, 0x6c, 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x69, 0x6e, - 0x64, 0x65, 0x78, 0x65, 0x64, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, - 0x52, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x64, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x1d, 0x0a, - 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x03, 0x52, 0x09, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x2c, 0x0a, 0x12, - 0x70, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x72, 0x6f, - 0x77, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x70, 0x65, 0x6e, 0x64, 0x69, 0x6e, - 0x67, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x6f, 0x77, 0x73, 0x22, 0x8e, 0x04, 0x0a, 0x0d, 0x53, - 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x18, 0x0a, 0x07, - 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, - 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, - 0x5f, 0x6b, 0x65, 0x79, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x61, 0x63, - 0x63, 0x65, 0x73, 0x73, 0x4b, 0x65, 0x79, 0x49, 0x44, 0x12, 0x2a, 0x0a, 0x11, 0x73, 0x65, 0x63, - 0x72, 0x65, 0x74, 0x5f, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, 0x41, 0x63, 0x63, 0x65, - 0x73, 0x73, 0x4b, 0x65, 0x79, 0x12, 0x16, 0x0a, 0x06, 0x75, 0x73, 0x65, 0x53, 0x53, 0x4c, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x75, 0x73, 0x65, 0x53, 0x53, 0x4c, 0x12, 0x1f, 0x0a, - 0x0b, 0x62, 0x75, 0x63, 0x6b, 0x65, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0a, 0x62, 0x75, 0x63, 0x6b, 0x65, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1b, - 0x0a, 0x09, 0x72, 0x6f, 0x6f, 0x74, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x06, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x08, 0x72, 0x6f, 0x6f, 0x74, 0x50, 0x61, 0x74, 0x68, 0x12, 0x16, 0x0a, 0x06, 0x75, - 0x73, 0x65, 0x49, 0x41, 0x4d, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x75, 0x73, 0x65, - 0x49, 0x41, 0x4d, 0x12, 0x20, 0x0a, 0x0b, 0x49, 0x41, 0x4d, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, - 0x6e, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x49, 0x41, 0x4d, 0x45, 0x6e, 0x64, - 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, - 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x74, 0x6f, - 0x72, 0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x28, 0x0a, 0x10, 0x75, 0x73, 0x65, 0x5f, - 0x76, 0x69, 0x72, 0x74, 0x75, 0x61, 0x6c, 0x5f, 0x68, 0x6f, 0x73, 0x74, 0x18, 0x0a, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x0e, 0x75, 0x73, 0x65, 0x56, 0x69, 0x72, 0x74, 0x75, 0x61, 0x6c, 0x48, 0x6f, - 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x18, 0x0b, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6c, - 0x6f, 0x75, 0x64, 0x5f, 0x70, 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x18, 0x0c, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0d, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x50, 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, - 0x72, 0x12, 0x2c, 0x0a, 0x12, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x74, 0x69, 0x6d, - 0x65, 0x6f, 0x75, 0x74, 0x5f, 0x6d, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x72, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4d, 0x73, 0x12, - 0x1c, 0x0a, 0x09, 0x73, 0x73, 0x6c, 0x43, 0x41, 0x43, 0x65, 0x72, 0x74, 0x18, 0x0e, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x09, 0x73, 0x73, 0x6c, 0x43, 0x41, 0x43, 0x65, 0x72, 0x74, 0x12, 0x2c, 0x0a, - 0x11, 0x47, 0x63, 0x70, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x4a, 0x53, - 0x4f, 0x4e, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x47, 0x63, 0x70, 0x43, 0x72, 0x65, - 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x4a, 0x53, 0x4f, 0x4e, 0x22, 0xa5, 0x01, 0x0a, 0x11, - 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x66, - 0x6f, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x66, - 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x09, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x66, 0x69, - 0x65, 0x6c, 0x64, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, - 0x66, 0x69, 0x65, 0x6c, 0x64, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x64, 0x61, 0x74, - 0x61, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x64, - 0x61, 0x74, 0x61, 0x50, 0x61, 0x74, 0x68, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x64, 0x61, 0x74, 0x61, - 0x5f, 0x69, 0x64, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x03, 0x52, 0x07, 0x64, 0x61, 0x74, 0x61, - 0x49, 0x64, 0x73, 0x22, 0xcc, 0x01, 0x0a, 0x07, 0x4a, 0x6f, 0x62, 0x49, 0x6e, 0x66, 0x6f, 0x12, - 0x19, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x03, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x10, 0x0a, 0x03, 0x64, 0x69, - 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x64, 0x69, 0x6d, 0x12, 0x1d, 0x0a, 0x0a, - 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, - 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x65, - 0x6e, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x65, - 0x6e, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x44, 0x0a, 0x0c, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, - 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, - 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, - 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x14, 0x0a, 0x05, - 0x70, 0x6f, 0x64, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x70, 0x6f, 0x64, - 0x49, 0x44, 0x22, 0x5e, 0x0a, 0x19, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, - 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, - 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, - 0x6d, 0x65, 0x22, 0x91, 0x01, 0x0a, 0x1a, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, - 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x74, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x22, 0x8c, 0x01, 0x0a, 0x15, 0x44, 0x65, 0x73, 0x63, + 0x72, 0x69, 0x62, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, @@ -3131,201 +3060,291 @@ var file_index_coord_proto_rawDesc = []byte{ 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x69, 0x6e, 0x64, 0x65, - 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x22, 0x38, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6e, - 0x64, 0x65, 0x78, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x22, 0x0a, 0x0c, - 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, - 0x22, 0x8a, 0x01, 0x0a, 0x13, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x73, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, - 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x3e, 0x0a, - 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x02, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, - 0x6f, 0x52, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x22, 0xc2, 0x03, - 0x0a, 0x0b, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x22, 0x0a, - 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, - 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x1d, 0x0a, - 0x0a, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x09, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3c, 0x0a, 0x0a, - 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, - 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x52, - 0x09, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x54, 0x79, 0x70, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, - 0x73, 0x6b, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, - 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20, - 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, - 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x03, - 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, 0x16, 0x0a, 0x06, - 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, - 0x64, 0x65, 0x49, 0x44, 0x12, 0x32, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x0a, 0x20, - 0x01, 0x28, 0x0e, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, - 0x5f, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, - 0x61, 0x69, 0x6c, 0x52, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x64, 0x69, 0x6d, - 0x18, 0x0c, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x64, 0x69, 0x6d, 0x12, 0x25, 0x0a, 0x0e, 0x63, - 0x65, 0x6e, 0x74, 0x72, 0x6f, 0x69, 0x64, 0x73, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x18, 0x0d, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x65, 0x6e, 0x74, 0x72, 0x6f, 0x69, 0x64, 0x73, 0x46, 0x69, - 0x6c, 0x65, 0x22, 0x51, 0x0a, 0x0c, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, - 0x74, 0x73, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, - 0x49, 0x44, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x16, 0x0a, - 0x06, 0x6c, 0x6f, 0x67, 0x49, 0x44, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x06, 0x6c, - 0x6f, 0x67, 0x49, 0x44, 0x73, 0x22, 0x47, 0x0a, 0x0c, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x4c, 0x6f, - 0x67, 0x50, 0x61, 0x74, 0x68, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, - 0x1d, 0x0a, 0x0a, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x73, 0x18, 0x02, 0x20, - 0x03, 0x28, 0x09, 0x52, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x50, 0x61, 0x74, 0x68, 0x73, 0x22, 0xc1, - 0x03, 0x0a, 0x09, 0x53, 0x74, 0x61, 0x74, 0x73, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x22, 0x0a, 0x0c, - 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, - 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, - 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x6e, - 0x65, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, - 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, - 0x44, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, - 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, - 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, - 0x65, 0x49, 0x44, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, - 0x44, 0x12, 0x32, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, - 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, - 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72, 0x65, - 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, 0x69, 0x6c, - 0x52, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, - 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x03, - 0x52, 0x0f, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, - 0x44, 0x12, 0x3f, 0x0a, 0x0a, 0x73, 0x75, 0x62, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x18, - 0x0b, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x73, - 0x53, 0x75, 0x62, 0x4a, 0x6f, 0x62, 0x52, 0x0a, 0x73, 0x75, 0x62, 0x4a, 0x6f, 0x62, 0x54, 0x79, - 0x70, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x61, 0x6e, 0x52, 0x65, 0x63, 0x79, 0x63, 0x6c, 0x65, - 0x18, 0x0c, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x63, 0x61, 0x6e, 0x52, 0x65, 0x63, 0x79, 0x63, - 0x6c, 0x65, 0x2a, 0x5b, 0x0a, 0x07, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0f, 0x0a, - 0x0b, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x13, - 0x0a, 0x0f, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, - 0x62, 0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x41, 0x6e, - 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x4a, 0x6f, 0x62, 0x10, 0x02, 0x12, 0x13, 0x0a, 0x0f, 0x4a, 0x6f, - 0x62, 0x54, 0x79, 0x70, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4a, 0x6f, 0x62, 0x10, 0x03, 0x2a, - 0x83, 0x01, 0x0a, 0x08, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x10, 0x0a, 0x0c, - 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x10, - 0x0a, 0x0c, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x69, 0x74, 0x10, 0x01, - 0x12, 0x16, 0x0a, 0x12, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x50, 0x72, - 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x10, 0x02, 0x12, 0x14, 0x0a, 0x10, 0x4a, 0x6f, 0x62, 0x53, - 0x74, 0x61, 0x74, 0x65, 0x46, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x10, 0x03, 0x12, 0x12, - 0x0a, 0x0e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, - 0x10, 0x04, 0x12, 0x11, 0x0a, 0x0d, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, - 0x74, 0x72, 0x79, 0x10, 0x05, 0x2a, 0x40, 0x0a, 0x0b, 0x53, 0x74, 0x61, 0x74, 0x73, 0x53, 0x75, - 0x62, 0x4a, 0x6f, 0x62, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x08, - 0x0a, 0x04, 0x53, 0x6f, 0x72, 0x74, 0x10, 0x01, 0x12, 0x10, 0x0a, 0x0c, 0x54, 0x65, 0x78, 0x74, - 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62, 0x10, 0x02, 0x12, 0x0b, 0x0a, 0x07, 0x42, 0x4d, - 0x32, 0x35, 0x4a, 0x6f, 0x62, 0x10, 0x03, 0x32, 0xd5, 0x0b, 0x0a, 0x0a, 0x49, 0x6e, 0x64, 0x65, - 0x78, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x12, 0x6c, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, - 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x2e, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, - 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x73, 0x22, 0x00, 0x12, 0x71, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, - 0x73, 0x74, 0x69, 0x63, 0x73, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x32, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, - 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, - 0x63, 0x73, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x54, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, - 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, 0x61, - 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, + 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x22, 0x61, 0x0a, 0x1c, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, + 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xc4, 0x01, 0x0a, 0x1d, 0x47, 0x65, + 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, + 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x12, 0x21, 0x0a, 0x0c, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x64, 0x5f, 0x72, 0x6f, 0x77, 0x73, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x64, 0x52, + 0x6f, 0x77, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x72, 0x6f, 0x77, + 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x52, 0x6f, + 0x77, 0x73, 0x12, 0x2c, 0x0a, 0x12, 0x70, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x5f, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, + 0x70, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x6f, 0x77, 0x73, + 0x22, 0x8e, 0x04, 0x0a, 0x0d, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x12, 0x18, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x21, 0x0a, 0x0c, + 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6b, 0x65, 0x79, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0b, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x4b, 0x65, 0x79, 0x49, 0x44, 0x12, + 0x2a, 0x0a, 0x11, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, 0x5f, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, + 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x73, 0x65, 0x63, 0x72, + 0x65, 0x74, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x4b, 0x65, 0x79, 0x12, 0x16, 0x0a, 0x06, 0x75, + 0x73, 0x65, 0x53, 0x53, 0x4c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x75, 0x73, 0x65, + 0x53, 0x53, 0x4c, 0x12, 0x1f, 0x0a, 0x0b, 0x62, 0x75, 0x63, 0x6b, 0x65, 0x74, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x62, 0x75, 0x63, 0x6b, 0x65, 0x74, + 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x72, 0x6f, 0x6f, 0x74, 0x5f, 0x70, 0x61, 0x74, + 0x68, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x72, 0x6f, 0x6f, 0x74, 0x50, 0x61, 0x74, + 0x68, 0x12, 0x16, 0x0a, 0x06, 0x75, 0x73, 0x65, 0x49, 0x41, 0x4d, 0x18, 0x07, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x06, 0x75, 0x73, 0x65, 0x49, 0x41, 0x4d, 0x12, 0x20, 0x0a, 0x0b, 0x49, 0x41, 0x4d, + 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, + 0x49, 0x41, 0x4d, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x73, + 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0b, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x28, + 0x0a, 0x10, 0x75, 0x73, 0x65, 0x5f, 0x76, 0x69, 0x72, 0x74, 0x75, 0x61, 0x6c, 0x5f, 0x68, 0x6f, + 0x73, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x75, 0x73, 0x65, 0x56, 0x69, 0x72, + 0x74, 0x75, 0x61, 0x6c, 0x48, 0x6f, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x67, 0x69, + 0x6f, 0x6e, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, + 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x5f, 0x70, 0x72, 0x6f, 0x76, 0x69, 0x64, + 0x65, 0x72, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x50, + 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x12, 0x2c, 0x0a, 0x12, 0x72, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x5f, 0x6d, 0x73, 0x18, 0x0d, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x10, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x54, 0x69, 0x6d, 0x65, + 0x6f, 0x75, 0x74, 0x4d, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x73, 0x6c, 0x43, 0x41, 0x43, 0x65, + 0x72, 0x74, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x73, 0x6c, 0x43, 0x41, 0x43, + 0x65, 0x72, 0x74, 0x12, 0x2c, 0x0a, 0x11, 0x47, 0x63, 0x70, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, + 0x74, 0x69, 0x61, 0x6c, 0x4a, 0x53, 0x4f, 0x4e, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, + 0x47, 0x63, 0x70, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x4a, 0x53, 0x4f, + 0x4e, 0x22, 0xa5, 0x01, 0x0a, 0x11, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x46, 0x69, + 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, + 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, + 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x4e, 0x61, 0x6d, 0x65, + 0x12, 0x1d, 0x0a, 0x0a, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x54, 0x79, 0x70, 0x65, 0x12, + 0x1d, 0x0a, 0x0a, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x73, 0x18, 0x04, 0x20, + 0x03, 0x28, 0x09, 0x52, 0x09, 0x64, 0x61, 0x74, 0x61, 0x50, 0x61, 0x74, 0x68, 0x73, 0x12, 0x19, + 0x0a, 0x08, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x03, + 0x52, 0x07, 0x64, 0x61, 0x74, 0x61, 0x49, 0x64, 0x73, 0x22, 0xcc, 0x01, 0x0a, 0x07, 0x4a, 0x6f, + 0x62, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, + 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, + 0x12, 0x10, 0x0a, 0x03, 0x64, 0x69, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x64, + 0x69, 0x6d, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, + 0x65, 0x12, 0x19, 0x0a, 0x08, 0x65, 0x6e, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x07, 0x65, 0x6e, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x44, 0x0a, 0x0c, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x05, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, + 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x50, 0x61, 0x72, 0x61, + 0x6d, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x70, 0x6f, 0x64, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x05, 0x70, 0x6f, 0x64, 0x49, 0x44, 0x22, 0x5e, 0x0a, 0x19, 0x47, 0x65, 0x74, 0x49, + 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x91, 0x01, 0x0a, 0x1a, 0x47, 0x65, 0x74, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x3e, 0x0a, 0x0b, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, + 0x52, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x22, 0x38, 0x0a, 0x12, + 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x22, 0x8a, 0x01, 0x0a, 0x13, 0x4c, 0x69, 0x73, 0x74, 0x49, + 0x6e, 0x64, 0x65, 0x78, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, + 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, - 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x52, 0x0a, - 0x0a, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x25, 0x2e, 0x6d, 0x69, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x12, 0x3e, 0x0a, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x69, 0x6e, 0x66, + 0x6f, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, 0x6e, + 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, + 0x66, 0x6f, 0x73, 0x22, 0xc2, 0x03, 0x0a, 0x0b, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x54, + 0x61, 0x73, 0x6b, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x65, + 0x6c, 0x64, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, + 0x64, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x4e, 0x61, + 0x6d, 0x65, 0x12, 0x3c, 0x0a, 0x0a, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x74, 0x79, 0x70, 0x65, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x44, 0x61, 0x74, + 0x61, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x54, 0x79, 0x70, 0x65, + 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, + 0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, + 0x18, 0x08, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, + 0x44, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x09, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x32, 0x0a, 0x05, 0x73, 0x74, + 0x61, 0x74, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, + 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, + 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x0b, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, + 0x10, 0x0a, 0x03, 0x64, 0x69, 0x6d, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x64, 0x69, + 0x6d, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x65, 0x6e, 0x74, 0x72, 0x6f, 0x69, 0x64, 0x73, 0x5f, 0x66, + 0x69, 0x6c, 0x65, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x65, 0x6e, 0x74, 0x72, + 0x6f, 0x69, 0x64, 0x73, 0x46, 0x69, 0x6c, 0x65, 0x22, 0x51, 0x0a, 0x0c, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x49, 0x44, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, + 0x72, 0x6f, 0x77, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, + 0x6f, 0x77, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x6c, 0x6f, 0x67, 0x49, 0x44, 0x73, 0x18, 0x03, 0x20, + 0x03, 0x28, 0x03, 0x52, 0x06, 0x6c, 0x6f, 0x67, 0x49, 0x44, 0x73, 0x22, 0x47, 0x0a, 0x0c, 0x46, + 0x69, 0x65, 0x6c, 0x64, 0x4c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x12, 0x18, 0x0a, 0x07, 0x66, + 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, + 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x70, 0x61, + 0x74, 0x68, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x50, + 0x61, 0x74, 0x68, 0x73, 0x22, 0xc1, 0x03, 0x0a, 0x09, 0x53, 0x74, 0x61, 0x74, 0x73, 0x54, 0x61, + 0x73, 0x6b, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, + 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, + 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x16, 0x0a, + 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, + 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, + 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x32, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, + 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x66, + 0x61, 0x69, 0x6c, 0x5f, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0a, 0x66, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x29, 0x0a, 0x10, + 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, + 0x18, 0x0a, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0f, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x3f, 0x0a, 0x0a, 0x73, 0x75, 0x62, 0x4a, 0x6f, + 0x62, 0x54, 0x79, 0x70, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, - 0x2e, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, - 0x00, 0x12, 0x66, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, - 0x74, 0x65, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, - 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, + 0x2e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x53, 0x75, 0x62, 0x4a, 0x6f, 0x62, 0x52, 0x0a, 0x73, 0x75, + 0x62, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x61, 0x6e, 0x52, + 0x65, 0x63, 0x79, 0x63, 0x6c, 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x63, 0x61, + 0x6e, 0x52, 0x65, 0x63, 0x79, 0x63, 0x6c, 0x65, 0x2a, 0x5b, 0x0a, 0x07, 0x4a, 0x6f, 0x62, 0x54, + 0x79, 0x70, 0x65, 0x12, 0x0f, 0x0a, 0x0b, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x4e, 0x6f, + 0x6e, 0x65, 0x10, 0x00, 0x12, 0x13, 0x0a, 0x0f, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x49, + 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62, 0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x4a, 0x6f, 0x62, + 0x54, 0x79, 0x70, 0x65, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x4a, 0x6f, 0x62, 0x10, 0x02, + 0x12, 0x13, 0x0a, 0x0f, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, + 0x4a, 0x6f, 0x62, 0x10, 0x03, 0x2a, 0x83, 0x01, 0x0a, 0x08, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x12, 0x10, 0x0a, 0x0c, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x4e, 0x6f, + 0x6e, 0x65, 0x10, 0x00, 0x12, 0x10, 0x0a, 0x0c, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x49, 0x6e, 0x69, 0x74, 0x10, 0x01, 0x12, 0x16, 0x0a, 0x12, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x49, 0x6e, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x10, 0x02, 0x12, 0x14, + 0x0a, 0x10, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x46, 0x69, 0x6e, 0x69, 0x73, 0x68, + 0x65, 0x64, 0x10, 0x03, 0x12, 0x12, 0x0a, 0x0e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x10, 0x04, 0x12, 0x11, 0x0a, 0x0d, 0x4a, 0x6f, 0x62, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x74, 0x72, 0x79, 0x10, 0x05, 0x2a, 0x40, 0x0a, 0x0b, 0x53, + 0x74, 0x61, 0x74, 0x73, 0x53, 0x75, 0x62, 0x4a, 0x6f, 0x62, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, + 0x6e, 0x65, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x53, 0x6f, 0x72, 0x74, 0x10, 0x01, 0x12, 0x10, + 0x0a, 0x0c, 0x54, 0x65, 0x78, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62, 0x10, 0x02, + 0x12, 0x0b, 0x0a, 0x07, 0x42, 0x4d, 0x32, 0x35, 0x4a, 0x6f, 0x62, 0x10, 0x03, 0x32, 0xd5, 0x0b, + 0x0a, 0x0a, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x12, 0x6c, 0x0a, 0x12, + 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, + 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, + 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x22, 0x00, 0x12, 0x71, 0x0a, 0x14, 0x47, 0x65, + 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x43, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x12, 0x32, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x74, + 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x74, 0x72, + 0x69, 0x6e, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x54, 0x0a, + 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, - 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7b, 0x0a, 0x14, 0x47, 0x65, 0x74, - 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x78, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x22, 0x00, 0x12, 0x52, 0x0a, 0x0a, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, + 0x78, 0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, + 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x66, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x49, 0x6e, + 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, - 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x64, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, - 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x7b, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, + 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, - 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, - 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x50, 0x0a, 0x09, - 0x44, 0x72, 0x6f, 0x70, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, - 0x72, 0x6f, 0x70, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, - 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x66, - 0x0a, 0x0d, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, - 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, + 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x64, 0x0a, 0x0d, + 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x27, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, + 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x50, 0x0a, 0x09, 0x44, 0x72, 0x6f, 0x70, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, + 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x22, 0x00, 0x12, 0x66, 0x0a, 0x0d, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, + 0x69, 0x62, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x49, 0x6e, 0x64, - 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, - 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x75, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, - 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, 0x2d, 0x2e, 0x6d, + 0x65, 0x78, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x75, 0x0a, 0x12, + 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, + 0x63, 0x73, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, + 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, + 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x7e, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, + 0x75, 0x69, 0x6c, 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, - 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, - 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, - 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, - 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7e, 0x0a, - 0x15, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x50, 0x72, - 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, - 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, - 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, - 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, - 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 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, 0x43, - 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, - 0x65, 0x61, 0x6c, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, - 0x2f, 0x5a, 0x2d, 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, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x70, 0x62, - 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x50, + 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, + 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 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, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, + 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, + 0x74, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x2f, 0x5a, 0x2d, 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, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( diff --git a/pkg/proto/worker.proto b/pkg/proto/worker.proto index 08d4f9828e4da..7d5826d3b33ad 100644 --- a/pkg/proto/worker.proto +++ b/pkg/proto/worker.proto @@ -169,6 +169,7 @@ message IndexTaskInfo { string fail_reason = 5; int32 current_index_version = 6; int64 index_store_version = 7; + uint64 mem_size = 8; } message IndexJobResults { diff --git a/pkg/proto/workerpb/worker.pb.go b/pkg/proto/workerpb/worker.pb.go index bb09ccc728514..47f52b27d4f52 100644 --- a/pkg/proto/workerpb/worker.pb.go +++ b/pkg/proto/workerpb/worker.pb.go @@ -1139,6 +1139,7 @@ type IndexTaskInfo struct { FailReason string `protobuf:"bytes,5,opt,name=fail_reason,json=failReason,proto3" json:"fail_reason,omitempty"` CurrentIndexVersion int32 `protobuf:"varint,6,opt,name=current_index_version,json=currentIndexVersion,proto3" json:"current_index_version,omitempty"` IndexStoreVersion int64 `protobuf:"varint,7,opt,name=index_store_version,json=indexStoreVersion,proto3" json:"index_store_version,omitempty"` + MemSize uint64 `protobuf:"varint,8,opt,name=mem_size,json=memSize,proto3" json:"mem_size,omitempty"` } func (x *IndexTaskInfo) Reset() { @@ -1222,6 +1223,13 @@ func (x *IndexTaskInfo) GetIndexStoreVersion() int64 { return 0 } +func (x *IndexTaskInfo) GetMemSize() uint64 { + if x != nil { + return x.MemSize + } + return 0 +} + type IndexJobResults struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -2002,7 +2010,7 @@ var file_worker_proto_rawDesc = []byte{ 0x44, 0x73, 0x12, 0x36, 0x0a, 0x08, 0x6a, 0x6f, 0x62, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, - 0x65, 0x52, 0x07, 0x6a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x22, 0xb6, 0x02, 0x0a, 0x0d, 0x49, + 0x65, 0x52, 0x07, 0x6a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x22, 0xd1, 0x02, 0x0a, 0x0d, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x54, 0x61, 0x73, 0x6b, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x18, 0x0a, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x35, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, @@ -2022,175 +2030,177 @@ var file_worker_proto_rawDesc = []byte{ 0x69, 0x6f, 0x6e, 0x12, 0x2e, 0x0a, 0x13, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x11, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x6f, 0x72, 0x65, 0x56, 0x65, 0x72, 0x73, - 0x69, 0x6f, 0x6e, 0x22, 0x4e, 0x0a, 0x0f, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62, 0x52, + 0x69, 0x6f, 0x6e, 0x12, 0x19, 0x0a, 0x08, 0x6d, 0x65, 0x6d, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, + 0x08, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x6d, 0x65, 0x6d, 0x53, 0x69, 0x7a, 0x65, 0x22, 0x4e, + 0x0a, 0x0f, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, + 0x73, 0x12, 0x3b, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x54, 0x61, 0x73, + 0x6b, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0xa3, + 0x01, 0x0a, 0x0d, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, + 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x32, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x0b, + 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0a, 0x66, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x25, 0x0a, + 0x0e, 0x63, 0x65, 0x6e, 0x74, 0x72, 0x6f, 0x69, 0x64, 0x73, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x65, 0x6e, 0x74, 0x72, 0x6f, 0x69, 0x64, 0x73, + 0x46, 0x69, 0x6c, 0x65, 0x22, 0x4d, 0x0a, 0x0e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x3b, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, 0x6e, 0x64, - 0x65, 0x78, 0x54, 0x61, 0x73, 0x6b, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, - 0x6c, 0x74, 0x73, 0x22, 0xa3, 0x01, 0x0a, 0x0d, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, - 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x32, 0x0a, - 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, - 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, - 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x61, 0x73, - 0x6f, 0x6e, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x65, 0x6e, 0x74, 0x72, 0x6f, 0x69, 0x64, 0x73, 0x5f, - 0x66, 0x69, 0x6c, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x65, 0x6e, 0x74, - 0x72, 0x6f, 0x69, 0x64, 0x73, 0x46, 0x69, 0x6c, 0x65, 0x22, 0x4d, 0x0a, 0x0e, 0x41, 0x6e, 0x61, - 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x3b, 0x0a, 0x07, 0x72, - 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, - 0x78, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, - 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x91, 0x05, 0x0a, 0x0b, 0x53, 0x74, 0x61, - 0x74, 0x73, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, - 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, - 0x12, 0x32, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, - 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, - 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, - 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72, 0x65, 0x61, - 0x73, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, 0x69, 0x6c, 0x52, - 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, - 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, - 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x73, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, - 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x68, 0x61, - 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x68, 0x61, 0x6e, - 0x6e, 0x65, 0x6c, 0x12, 0x3f, 0x0a, 0x0b, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x6c, 0x6f, - 0x67, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, - 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x0a, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, - 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x3d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x6c, 0x6f, - 0x67, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x41, 0x6e, 0x61, + 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, + 0x6c, 0x74, 0x73, 0x22, 0x91, 0x05, 0x0a, 0x0b, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, + 0x75, 0x6c, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x32, 0x0a, 0x05, 0x73, + 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, + 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, + 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x61, 0x73, 0x6f, 0x6e, + 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x44, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, + 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x3f, + 0x0a, 0x0b, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x08, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, + 0x6c, 0x6f, 0x67, 0x52, 0x0a, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x4c, 0x6f, 0x67, 0x73, 0x12, + 0x3d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x09, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, + 0x6c, 0x6f, 0x67, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x5a, + 0x0a, 0x0f, 0x74, 0x65, 0x78, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x6c, 0x6f, 0x67, + 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, 0x61, + 0x74, 0x73, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x2e, 0x54, 0x65, 0x78, 0x74, 0x53, 0x74, 0x61, + 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0d, 0x74, 0x65, 0x78, + 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x75, + 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6e, 0x75, + 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x3b, 0x0a, 0x09, 0x62, 0x6d, 0x32, 0x35, 0x5f, 0x6c, 0x6f, + 0x67, 0x73, 0x18, 0x0c, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, - 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x73, 0x4c, - 0x6f, 0x67, 0x73, 0x12, 0x5a, 0x0a, 0x0f, 0x74, 0x65, 0x78, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x74, - 0x73, 0x5f, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, - 0x78, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x2e, 0x54, 0x65, - 0x78, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x52, 0x0d, 0x74, 0x65, 0x78, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x12, - 0x19, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, - 0x03, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x3b, 0x0a, 0x09, 0x62, 0x6d, - 0x32, 0x35, 0x5f, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x0c, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x08, 0x62, - 0x6d, 0x32, 0x35, 0x4c, 0x6f, 0x67, 0x73, 0x1a, 0x63, 0x0a, 0x12, 0x54, 0x65, 0x78, 0x74, 0x53, - 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, - 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, - 0x37, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x54, 0x65, 0x78, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, - 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x49, 0x0a, 0x0c, - 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x39, 0x0a, 0x07, - 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, - 0x65, 0x78, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x07, - 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0xeb, 0x02, 0x0a, 0x13, 0x51, 0x75, 0x65, 0x72, - 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, - 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, - 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, - 0x61, 0x74, 0x75, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, - 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, - 0x49, 0x44, 0x12, 0x51, 0x0a, 0x11, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, - 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, - 0x65, 0x78, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x75, 0x6c, - 0x74, 0x73, 0x48, 0x00, 0x52, 0x0f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62, 0x52, 0x65, - 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x54, 0x0a, 0x13, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, - 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, - 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x48, 0x00, 0x52, 0x11, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, - 0x65, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x4e, 0x0a, 0x11, 0x73, - 0x74, 0x61, 0x74, 0x73, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, - 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, 0x61, 0x74, - 0x73, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x48, 0x00, 0x52, 0x0f, 0x73, 0x74, 0x61, 0x74, - 0x73, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x08, 0x0a, 0x06, 0x72, - 0x65, 0x73, 0x75, 0x6c, 0x74, 0x22, 0x83, 0x01, 0x0a, 0x11, 0x44, 0x72, 0x6f, 0x70, 0x4a, 0x6f, - 0x62, 0x73, 0x56, 0x32, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, - 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, - 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x61, 0x73, - 0x6b, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x07, 0x74, 0x61, 0x73, 0x6b, - 0x49, 0x44, 0x73, 0x12, 0x36, 0x0a, 0x08, 0x6a, 0x6f, 0x62, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x54, 0x79, - 0x70, 0x65, 0x52, 0x07, 0x6a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x32, 0xb6, 0x08, 0x0a, 0x09, - 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x6c, 0x0a, 0x12, 0x47, 0x65, 0x74, - 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, - 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, - 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, - 0x74, 0x61, 0x74, 0x65, 0x73, 0x22, 0x00, 0x12, 0x71, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x53, 0x74, - 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, - 0x32, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, - 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, - 0x73, 0x74, 0x69, 0x63, 0x73, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x50, 0x0a, 0x09, 0x43, 0x72, - 0x65, 0x61, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, - 0x61, 0x74, 0x65, 0x4a, 0x6f, 0x62, 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, 0x09, - 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x51, - 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, - 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4e, 0x0a, 0x08, 0x44, 0x72, 0x6f, 0x70, - 0x4a, 0x6f, 0x62, 0x73, 0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x4a, 0x6f, - 0x62, 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, 0x0b, 0x47, 0x65, 0x74, 0x4a, - 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, - 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, - 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x73, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7b, 0x0a, 0x12, 0x53, 0x68, - 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 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, 0x54, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, - 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x56, 0x32, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, - 0x61, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x56, 0x32, 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, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x12, 0x26, 0x2e, + 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x08, 0x62, 0x6d, 0x32, 0x35, 0x4c, 0x6f, + 0x67, 0x73, 0x1a, 0x63, 0x0a, 0x12, 0x54, 0x65, 0x78, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, + 0x6f, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x37, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x65, + 0x78, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x49, 0x0a, 0x0c, 0x53, 0x74, 0x61, 0x74, 0x73, + 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x39, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, + 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, + 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, + 0x74, 0x73, 0x22, 0xeb, 0x02, 0x0a, 0x13, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, + 0x56, 0x32, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, + 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x51, 0x0a, + 0x11, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, + 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, 0x6e, + 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x48, 0x00, 0x52, + 0x0f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, + 0x12, 0x54, 0x0a, 0x13, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, + 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, - 0x65, 0x78, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, - 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x12, 0x52, 0x0a, 0x0a, 0x44, 0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x12, 0x25, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, - 0x64, 0x65, 0x78, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 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, 0x30, 0x5a, 0x2e, 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, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x77, 0x6f, - 0x72, 0x6b, 0x65, 0x72, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x65, 0x78, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, + 0x73, 0x48, 0x00, 0x52, 0x11, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x4a, 0x6f, 0x62, 0x52, + 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x4e, 0x0a, 0x11, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, + 0x6a, 0x6f, 0x62, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x75, + 0x6c, 0x74, 0x73, 0x48, 0x00, 0x52, 0x0f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x4a, 0x6f, 0x62, 0x52, + 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x08, 0x0a, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, + 0x22, 0x83, 0x01, 0x0a, 0x11, 0x44, 0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, + 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, + 0x65, 0x72, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x73, 0x18, + 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x73, 0x12, 0x36, + 0x0a, 0x08, 0x6a, 0x6f, 0x62, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x52, 0x07, 0x6a, + 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x32, 0xb6, 0x08, 0x0a, 0x09, 0x49, 0x6e, 0x64, 0x65, 0x78, + 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x6c, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6f, + 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, + 0x22, 0x00, 0x12, 0x71, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, + 0x69, 0x63, 0x73, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x32, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, + 0x61, 0x6c, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, + 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x50, 0x0a, 0x09, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4a, + 0x6f, 0x62, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4a, 0x6f, + 0x62, 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, 0x09, 0x51, 0x75, 0x65, 0x72, 0x79, + 0x4a, 0x6f, 0x62, 0x73, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, + 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, + 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x4e, 0x0a, 0x08, 0x44, 0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62, 0x73, 0x12, + 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62, 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, 0x0b, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, + 0x74, 0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, + 0x61, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, + 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7b, 0x0a, 0x12, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, + 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 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, 0x54, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4a, 0x6f, 0x62, + 0x56, 0x32, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4a, 0x6f, + 0x62, 0x56, 0x32, 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, 0x51, 0x75, 0x65, + 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x51, 0x75, + 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x56, + 0x32, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x52, 0x0a, 0x0a, 0x44, + 0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, + 0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 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, + 0x30, 0x5a, 0x2e, 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, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x70, + 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( From da1b786ef8e99e4999e15e3a4edc88e7f127c761 Mon Sep 17 00:00:00 2001 From: congqixia Date: Tue, 14 Jan 2025 14:14:58 +0800 Subject: [PATCH 32/34] enhance: Utilize "find0" in segment.find_first (#39229) Related to #39003 Previous PR #39004 has to clone & flip bitset due to bitset does not support find0 operator. #39176 added this feature so clone & flip could be removed now. Signed-off-by: Congqi Xia --- internal/core/src/segcore/ChunkedSegmentSealedImpl.cpp | 8 ++------ internal/core/src/segcore/SegmentSealedImpl.cpp | 9 ++------- 2 files changed, 4 insertions(+), 13 deletions(-) diff --git a/internal/core/src/segcore/ChunkedSegmentSealedImpl.cpp b/internal/core/src/segcore/ChunkedSegmentSealedImpl.cpp index 1a79fc8cff87b..0c7c0c7a910e4 100644 --- a/internal/core/src/segcore/ChunkedSegmentSealedImpl.cpp +++ b/internal/core/src/segcore/ChunkedSegmentSealedImpl.cpp @@ -1255,12 +1255,8 @@ ChunkedSegmentSealedImpl::find_first(int64_t limit, std::vector seg_offsets; seg_offsets.reserve(limit); - // flip bitset since `find_next` is used to find true. - auto flipped = bitset.clone(); - flipped.flip(); - int64_t offset = 0; - std::optional result = flipped.find_first(); + std::optional result = bitset.find_first(false); while (result.has_value() && hit_num < limit) { hit_num++; seg_offsets.push_back(result.value()); @@ -1269,7 +1265,7 @@ ChunkedSegmentSealedImpl::find_first(int64_t limit, // In fact, this case won't happen on sealed segments. continue; } - result = flipped.find_next(offset); + result = bitset.find_next(offset, false); } return {seg_offsets, more_hit_than_limit && result.has_value()}; diff --git a/internal/core/src/segcore/SegmentSealedImpl.cpp b/internal/core/src/segcore/SegmentSealedImpl.cpp index 4ccf68267b0b4..ec6baab667010 100644 --- a/internal/core/src/segcore/SegmentSealedImpl.cpp +++ b/internal/core/src/segcore/SegmentSealedImpl.cpp @@ -1719,13 +1719,8 @@ SegmentSealedImpl::find_first(int64_t limit, const BitsetType& bitset) const { std::vector seg_offsets; seg_offsets.reserve(limit); - // flip bitset since `find_first` & `find_next` is used to find true. - // could be optimized by support find false in bitset. - auto flipped = bitset.clone(); - flipped.flip(); - int64_t offset = 0; - std::optional result = flipped.find_first(); + std::optional result = bitset.find_first(false); while (result.has_value() && hit_num < limit) { hit_num++; seg_offsets.push_back(result.value()); @@ -1734,7 +1729,7 @@ SegmentSealedImpl::find_first(int64_t limit, const BitsetType& bitset) const { // In fact, this case won't happen on sealed segments. continue; } - result = flipped.find_next(offset); + result = bitset.find_next(offset, false); } return {seg_offsets, more_hit_than_limit && result.has_value()}; From fd84ed817c54b4f046b0bdb43cb64fc6675eefc3 Mon Sep 17 00:00:00 2001 From: Zhen Ye Date: Tue, 14 Jan 2025 15:14:59 +0800 Subject: [PATCH 33/34] enhance: add broadcast operation for msgstream (#39040) issue: #38399 - make broadcast service available for msgstream by reusing the architecture streaming service --------- Signed-off-by: chyezh --- cmd/roles/roles.go | 7 +- configs/milvus.yaml | 2 + internal/distributed/streaming/streaming.go | 3 + .../distributed/streaming/streaming_test.go | 5 + internal/distributed/streaming/util.go | 79 +++--------- internal/distributed/streaming/wal.go | 7 +- internal/rootcoord/broadcast_task.go | 105 ++++++++++++++++ internal/rootcoord/dml_channels.go | 3 + internal/rootcoord/root_coord.go | 5 + internal/rootcoord/root_coord_test.go | 3 + internal/streamingcoord/client/client.go | 14 ++- internal/streamingcoord/client/client_impl.go | 4 +- .../server/broadcaster/broadcaster.go | 9 +- .../server/broadcaster/broadcaster_impl.go | 39 ++++-- .../server/broadcaster/broadcaster_test.go | 29 +++-- .../broadcaster/registry/append_operator.go | 42 +++++++ .../broadcaster/registry/test_utility.go | 12 ++ .../streamingcoord/server/broadcaster/task.go | 10 +- internal/streamingcoord/server/server.go | 3 +- .../util/message/adaptor/broadcast_message.go | 25 ++++ pkg/streaming/util/types/responses.go | 113 ++++++++++++++++++ pkg/streaming/util/types/streaming_node.go | 51 -------- pkg/util/paramtable/component_param.go | 12 ++ pkg/util/paramtable/component_param_test.go | 3 + .../search_after_coord_down_test.go | 3 + .../coordrecovery/coord_recovery_test.go | 3 + tests/integration/suite.go | 2 + 27 files changed, 435 insertions(+), 158 deletions(-) create mode 100644 internal/rootcoord/broadcast_task.go create mode 100644 internal/streamingcoord/server/broadcaster/registry/append_operator.go create mode 100644 internal/streamingcoord/server/broadcaster/registry/test_utility.go create mode 100644 pkg/streaming/util/message/adaptor/broadcast_message.go create mode 100644 pkg/streaming/util/types/responses.go diff --git a/cmd/roles/roles.go b/cmd/roles/roles.go index ef59216f0f245..355c30eb1372a 100644 --- a/cmd/roles/roles.go +++ b/cmd/roles/roles.go @@ -43,7 +43,6 @@ import ( kvfactory "github.com/milvus-io/milvus/internal/util/dependency/kv" "github.com/milvus-io/milvus/internal/util/initcore" internalmetrics "github.com/milvus-io/milvus/internal/util/metrics" - "github.com/milvus-io/milvus/internal/util/streamingutil" "github.com/milvus-io/milvus/pkg/config" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" @@ -389,10 +388,8 @@ func (mr *MilvusRoles) Run() { tracer.Init() // Initialize streaming service if enabled. - if streamingutil.IsStreamingServiceEnabled() { - streaming.Init() - defer streaming.Release() - } + streaming.Init() + defer streaming.Release() coordclient.EnableLocalClientRole(&coordclient.LocalClientRoleConfig{ ServerType: mr.ServerType, diff --git a/configs/milvus.yaml b/configs/milvus.yaml index 305db189ef1be..7b591fa847bc6 100644 --- a/configs/milvus.yaml +++ b/configs/milvus.yaml @@ -1121,6 +1121,8 @@ streaming: # It's ok to set it into duration string, such as 30s or 1m30s, see time.ParseDuration backoffInitialInterval: 50ms backoffMultiplier: 2 # The multiplier of balance task trigger backoff, 2 by default + walBroadcaster: + concurrencyRatio: 1 # The concurrency ratio based on number of CPU for wal broadcaster, 1 by default. txn: defaultKeepaliveTimeout: 10s # The default keepalive timeout for wal txn, 10s by default diff --git a/internal/distributed/streaming/streaming.go b/internal/distributed/streaming/streaming.go index efd77d5f2ac34..df54d7c95ecc9 100644 --- a/internal/distributed/streaming/streaming.go +++ b/internal/distributed/streaming/streaming.go @@ -4,6 +4,7 @@ import ( "context" "time" + "github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/registry" kvfactory "github.com/milvus-io/milvus/internal/util/dependency/kv" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/options" @@ -17,6 +18,8 @@ var singleton WALAccesser = nil func Init() { c, _ := kvfactory.GetEtcdAndPath() singleton = newWALAccesser(c) + // Add the wal accesser to the broadcaster registry for making broadcast operation. + registry.Register(registry.AppendOperatorTypeStreaming, singleton) } // Release releases the resources of the wal accesser. diff --git a/internal/distributed/streaming/streaming_test.go b/internal/distributed/streaming/streaming_test.go index e44e18e7c2005..0da14f0923512 100644 --- a/internal/distributed/streaming/streaming_test.go +++ b/internal/distributed/streaming/streaming_test.go @@ -6,6 +6,7 @@ import ( "testing" "time" + "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/distributed/streaming" "github.com/milvus-io/milvus/pkg/streaming/util/message" @@ -34,6 +35,10 @@ func TestStreamingProduce(t *testing.T) { PartitionIds: []int64{1, 2, 3}, }). WithBody(&msgpb.CreateCollectionRequest{ + Base: &commonpb.MsgBase{ + MsgType: commonpb.MsgType_CreateCollection, + Timestamp: 1, + }, CollectionID: 1, }). WithBroadcast(vChannels). diff --git a/internal/distributed/streaming/util.go b/internal/distributed/streaming/util.go index 3a024dc03c5a3..02ba8e4584573 100644 --- a/internal/distributed/streaming/util.go +++ b/internal/distributed/streaming/util.go @@ -8,6 +8,11 @@ import ( "github.com/milvus-io/milvus/pkg/streaming/util/types" ) +type ( + AppendResponses = types.AppendResponses + AppendResponse = types.AppendResponse +) + // AppendMessagesToWAL appends messages to the wal. // It it a helper utility function to append messages to the wal. // If the messages is belong to one vchannel, it will be sent as a transaction. @@ -26,7 +31,7 @@ func (u *walAccesserImpl) AppendMessages(ctx context.Context, msgs ...message.Mu // Otherwise append the messages concurrently. mu := &sync.Mutex{} - resp := newAppendResponseN(len(msgs)) + resp := types.NewAppendResponseN(len(msgs)) wg := &sync.WaitGroup{} wg.Add(len(dispatchedMessages)) @@ -39,7 +44,7 @@ func (u *walAccesserImpl) AppendMessages(ctx context.Context, msgs ...message.Mu singleResp := u.appendToVChannel(ctx, vchannel, msgs...) mu.Lock() for i, idx := range idxes { - resp.fillResponseAtIdx(singleResp.Responses[i], idx) + resp.FillResponseAtIdx(singleResp.Responses[i], idx) } mu.Unlock() return struct{}{}, nil @@ -76,9 +81,9 @@ func (u *walAccesserImpl) dispatchMessages(msgs ...message.MutableMessage) (map[ // appendToVChannel appends the messages to the specified vchannel. func (u *walAccesserImpl) appendToVChannel(ctx context.Context, vchannel string, msgs ...message.MutableMessage) AppendResponses { if len(msgs) == 0 { - return newAppendResponseN(0) + return types.NewAppendResponseN(0) } - resp := newAppendResponseN(len(msgs)) + resp := types.NewAppendResponseN(len(msgs)) // if only one message here, append it directly, no more goroutine needed. // at most time, there's only one message here. @@ -86,7 +91,7 @@ func (u *walAccesserImpl) appendToVChannel(ctx context.Context, vchannel string, // we should optimize the message-format, make it into one; but not the goroutine count. if len(msgs) == 1 { appendResult, err := u.appendToWAL(ctx, msgs[0]) - resp.fillResponseAtIdx(AppendResponse{ + resp.FillResponseAtIdx(AppendResponse{ AppendResult: appendResult, Error: err, }, 0) @@ -99,7 +104,7 @@ func (u *walAccesserImpl) appendToVChannel(ctx context.Context, vchannel string, VChannel: vchannel, }) if err != nil { - resp.fillAllError(err) + resp.FillAllError(err) return resp } @@ -115,7 +120,7 @@ func (u *walAccesserImpl) appendToVChannel(ctx context.Context, vchannel string, defer wg.Done() if err := txn.Append(ctx, msg); err != nil { mu.Lock() - resp.fillResponseAtIdx(AppendResponse{ + resp.FillResponseAtIdx(AppendResponse{ Error: err, }, i) mu.Unlock() @@ -129,75 +134,19 @@ func (u *walAccesserImpl) appendToVChannel(ctx context.Context, vchannel string, // and fill the error with the first error. if err := resp.UnwrapFirstError(); err != nil { _ = txn.Rollback(ctx) // rollback failure can be ignored. - resp.fillAllError(err) + resp.FillAllError(err) return resp } // commit the transaction and fill the response. appendResult, err := txn.Commit(ctx) - resp.fillAllResponse(AppendResponse{ + resp.FillAllResponse(AppendResponse{ AppendResult: appendResult, Error: err, }) return resp } -// newAppendResponseN creates a new append response. -func newAppendResponseN(n int) AppendResponses { - return AppendResponses{ - Responses: make([]AppendResponse, n), - } -} - -// AppendResponse is the response of one append operation. -type AppendResponse struct { - AppendResult *types.AppendResult - Error error -} - -// AppendResponses is the response of append operation. -type AppendResponses struct { - Responses []AppendResponse -} - -func (a AppendResponses) MaxTimeTick() uint64 { - var maxTimeTick uint64 - for _, r := range a.Responses { - if r.AppendResult != nil && r.AppendResult.TimeTick > maxTimeTick { - maxTimeTick = r.AppendResult.TimeTick - } - } - return maxTimeTick -} - -// UnwrapFirstError returns the first error in the responses. -func (a AppendResponses) UnwrapFirstError() error { - for _, r := range a.Responses { - if r.Error != nil { - return r.Error - } - } - return nil -} - -// fillAllError fills all the responses with the same error. -func (a *AppendResponses) fillAllError(err error) { - for i := range a.Responses { - a.Responses[i].Error = err - } -} - -// fillResponseAtIdx fill the response at idx -func (a *AppendResponses) fillResponseAtIdx(resp AppendResponse, idx int) { - a.Responses[idx] = resp -} - -func (a *AppendResponses) fillAllResponse(resp AppendResponse) { - for i := range a.Responses { - a.Responses[i] = resp - } -} - // applyOpt applies the append options to the message. func applyOpt(msg message.MutableMessage, opts ...AppendOption) message.MutableMessage { if len(opts) == 0 { diff --git a/internal/distributed/streaming/wal.go b/internal/distributed/streaming/wal.go index f721f2d63bfdf..052129a1d6887 100644 --- a/internal/distributed/streaming/wal.go +++ b/internal/distributed/streaming/wal.go @@ -11,6 +11,7 @@ import ( "github.com/milvus-io/milvus/internal/distributed/streaming/internal/producer" "github.com/milvus-io/milvus/internal/streamingcoord/client" "github.com/milvus-io/milvus/internal/streamingnode/client/handler" + "github.com/milvus-io/milvus/internal/util/streamingutil" "github.com/milvus-io/milvus/internal/util/streamingutil/status" "github.com/milvus-io/milvus/internal/util/streamingutil/util" "github.com/milvus-io/milvus/pkg/streaming/util/message" @@ -27,7 +28,11 @@ func newWALAccesser(c *clientv3.Client) *walAccesserImpl { // Create a new streaming coord client. streamingCoordClient := client.NewClient(c) // Create a new streamingnode handler client. - handlerClient := handler.NewHandlerClient(streamingCoordClient.Assignment()) + var handlerClient handler.HandlerClient + if streamingutil.IsStreamingServiceEnabled() { + // streaming service is enabled, create the handler client for the streaming service. + handlerClient = handler.NewHandlerClient(streamingCoordClient.Assignment()) + } return &walAccesserImpl{ lifetime: typeutil.NewLifetime(), streamingCoordClient: streamingCoordClient, diff --git a/internal/rootcoord/broadcast_task.go b/internal/rootcoord/broadcast_task.go new file mode 100644 index 0000000000000..d95000318c3ab --- /dev/null +++ b/internal/rootcoord/broadcast_task.go @@ -0,0 +1,105 @@ +package rootcoord + +import ( + "context" + + "github.com/milvus-io/milvus/internal/util/streamingutil/util" + "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/streaming/util/message/adaptor" + "github.com/milvus-io/milvus/pkg/streaming/util/types" + "github.com/milvus-io/milvus/pkg/util/funcutil" + "github.com/milvus-io/milvus/pkg/util/syncutil" +) + +var _ task = (*broadcastTask)(nil) + +// newBroadcastTask creates a new broadcast task. +func newBroadcastTask(ctx context.Context, core *Core, msgs []message.MutableMessage) *broadcastTask { + return &broadcastTask{ + baseTask: newBaseTask(ctx, core), + msgs: msgs, + } +} + +// BroadcastTask is used to implement the broadcast operation based on the msgstream +// by using the streaming service interface. +// msgstream will be deprecated since 2.6.0 with streaming service, so those code will be removed in the future version. +type broadcastTask struct { + baseTask + msgs []message.MutableMessage // The message wait for broadcast + walName string + resultFuture *syncutil.Future[types.AppendResponses] +} + +func (b *broadcastTask) Execute(ctx context.Context) error { + result := types.NewAppendResponseN(len(b.msgs)) + defer func() { + b.resultFuture.Set(result) + }() + + for idx, msg := range b.msgs { + tsMsg, err := adaptor.NewMsgPackFromMutableMessageV1(msg) + if err != nil { + result.FillResponseAtIdx(types.AppendResponse{Error: err}, idx) + return err + } + pchannel := funcutil.ToPhysicalChannel(msg.VChannel()) + msgID, err := b.core.chanTimeTick.broadcastMarkDmlChannels([]string{pchannel}, &msgstream.MsgPack{ + BeginTs: b.ts, + EndTs: b.ts, + Msgs: []msgstream.TsMsg{tsMsg}, + }) + if err != nil { + result.FillResponseAtIdx(types.AppendResponse{Error: err}, idx) + continue + } + result.FillResponseAtIdx(types.AppendResponse{ + AppendResult: &types.AppendResult{ + MessageID: adaptor.MustGetMessageIDFromMQWrapperIDBytes(b.walName, msgID[pchannel]), + TimeTick: b.ts, + }, + }, idx) + } + return result.UnwrapFirstError() +} + +func newMsgStreamAppendOperator(c *Core) *msgstreamAppendOperator { + return &msgstreamAppendOperator{ + core: c, + walName: util.MustSelectWALName(), + } +} + +// msgstreamAppendOperator the code of streamingcoord to make broadcast available on the legacy msgstream. +// Because msgstream is bound to the rootcoord task, so we transfer each broadcast operation into a ddl task. +// to make sure the timetick rule. +// The Msgstream will be deprecated since 2.6.0, so we make a single module to hold it. +type msgstreamAppendOperator struct { + core *Core + walName string +} + +// AppendMessages implements the AppendOperator interface for broadcaster service at streaming service. +func (m *msgstreamAppendOperator) AppendMessages(ctx context.Context, msgs ...message.MutableMessage) types.AppendResponses { + t := &broadcastTask{ + baseTask: newBaseTask(ctx, m.core), + msgs: msgs, + walName: m.walName, + resultFuture: syncutil.NewFuture[types.AppendResponses](), + } + + if err := m.core.scheduler.AddTask(t); err != nil { + resp := types.NewAppendResponseN(len(msgs)) + resp.FillAllError(err) + return resp + } + + result, err := t.resultFuture.GetWithContext(ctx) + if err != nil { + resp := types.NewAppendResponseN(len(msgs)) + resp.FillAllError(err) + return resp + } + return result +} diff --git a/internal/rootcoord/dml_channels.go b/internal/rootcoord/dml_channels.go index 7e210efaedf0a..b626b6e0aade7 100644 --- a/internal/rootcoord/dml_channels.go +++ b/internal/rootcoord/dml_channels.go @@ -327,6 +327,9 @@ func (d *dmlChannels) broadcastMark(chanNames []string, pack *msgstream.MsgPack) result[cn] = id.Serialize() } } + } else { + dms.mutex.RUnlock() + return nil, errors.Newf("channel not in use: %s", chanName) } dms.mutex.RUnlock() } diff --git a/internal/rootcoord/root_coord.go b/internal/rootcoord/root_coord.go index 16e319cb8cf62..ec8fd93ac8628 100644 --- a/internal/rootcoord/root_coord.go +++ b/internal/rootcoord/root_coord.go @@ -46,6 +46,7 @@ import ( kvmetestore "github.com/milvus-io/milvus/internal/metastore/kv/rootcoord" "github.com/milvus-io/milvus/internal/metastore/model" streamingcoord "github.com/milvus-io/milvus/internal/streamingcoord/server" + "github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/registry" tso2 "github.com/milvus-io/milvus/internal/tso" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/dependency" @@ -766,6 +767,10 @@ func (c *Core) startInternal() error { sessionutil.SaveServerInfo(typeutil.RootCoordRole, c.session.ServerID) log.Info("rootcoord startup successfully") + // regster the core as a appendoperator for broadcast service. + // TODO: should be removed at 2.6.0. + // Add the wal accesser to the broadcaster registry for making broadcast operation. + registry.Register(registry.AppendOperatorTypeMsgstream, newMsgStreamAppendOperator(c)) return nil } diff --git a/internal/rootcoord/root_coord_test.go b/internal/rootcoord/root_coord_test.go index 402a6b2242914..951b23d5806e9 100644 --- a/internal/rootcoord/root_coord_test.go +++ b/internal/rootcoord/root_coord_test.go @@ -34,6 +34,7 @@ import ( "github.com/milvus-io/milvus/internal/metastore/model" "github.com/milvus-io/milvus/internal/mocks" mockrootcoord "github.com/milvus-io/milvus/internal/rootcoord/mocks" + "github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/registry" "github.com/milvus-io/milvus/internal/util/dependency" kvfactory "github.com/milvus-io/milvus/internal/util/dependency/kv" "github.com/milvus-io/milvus/internal/util/proxyutil" @@ -1356,6 +1357,7 @@ func TestCore_startTimeTickLoop(t *testing.T) { func TestRootcoord_EnableActiveStandby(t *testing.T) { randVal := rand.Int() paramtable.Init() + registry.ResetRegistration() Params.Save("etcd.rootPath", fmt.Sprintf("/%d", randVal)) // Need to reset global etcd to follow new path kvfactory.CloseEtcdClient() @@ -1416,6 +1418,7 @@ func TestRootcoord_EnableActiveStandby(t *testing.T) { func TestRootcoord_DisableActiveStandby(t *testing.T) { randVal := rand.Int() paramtable.Init() + registry.ResetRegistration() Params.Save("etcd.rootPath", fmt.Sprintf("/%d", randVal)) // Need to reset global etcd to follow new path kvfactory.CloseEtcdClient() diff --git a/internal/streamingcoord/client/client.go b/internal/streamingcoord/client/client.go index 79ef36053d8d9..4d9f30c35a95c 100644 --- a/internal/streamingcoord/client/client.go +++ b/internal/streamingcoord/client/client.go @@ -13,6 +13,7 @@ import ( "github.com/milvus-io/milvus/internal/streamingcoord/client/assignment" "github.com/milvus-io/milvus/internal/streamingcoord/client/broadcast" "github.com/milvus-io/milvus/internal/util/sessionutil" + "github.com/milvus-io/milvus/internal/util/streamingutil" "github.com/milvus-io/milvus/internal/util/streamingutil/service/balancer/picker" streamingserviceinterceptor "github.com/milvus-io/milvus/internal/util/streamingutil/service/interceptor" "github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc" @@ -43,9 +44,14 @@ type BroadcastService interface { // Client is the interface of log service client. type Client interface { + // Broadcast access broadcast service. + // Broadcast service will always be available. + // When streaming service is enabled, the broadcast will use the streaming service. + // When streaming service is disabled, the broadcast will use legacy msgstream. Broadcast() BroadcastService // Assignment access assignment service. + // Assignment service will only be available when streaming service is enabled. Assignment() AssignmentService // Close close the client. @@ -68,12 +74,16 @@ func NewClient(etcdCli *clientv3.Client) Client { dialOptions..., ) }) - assignmentService := lazygrpc.WithServiceCreator(conn, streamingpb.NewStreamingCoordAssignmentServiceClient) + var assignmentServiceImpl *assignment.AssignmentServiceImpl + if streamingutil.IsStreamingServiceEnabled() { + assignmentService := lazygrpc.WithServiceCreator(conn, streamingpb.NewStreamingCoordAssignmentServiceClient) + assignmentServiceImpl = assignment.NewAssignmentService(assignmentService) + } broadcastService := lazygrpc.WithServiceCreator(conn, streamingpb.NewStreamingCoordBroadcastServiceClient) return &clientImpl{ conn: conn, rb: rb, - assignmentService: assignment.NewAssignmentService(assignmentService), + assignmentService: assignmentServiceImpl, broadcastService: broadcast.NewBroadcastService(util.MustSelectWALName(), broadcastService), } } diff --git a/internal/streamingcoord/client/client_impl.go b/internal/streamingcoord/client/client_impl.go index 88c94794e1c4d..e45b6ebd2093e 100644 --- a/internal/streamingcoord/client/client_impl.go +++ b/internal/streamingcoord/client/client_impl.go @@ -26,7 +26,9 @@ func (c *clientImpl) Assignment() AssignmentService { // Close close the client. func (c *clientImpl) Close() { - c.assignmentService.Close() + if c.assignmentService != nil { + c.assignmentService.Close() + } c.conn.Close() c.rb.Close() } diff --git a/internal/streamingcoord/server/broadcaster/broadcaster.go b/internal/streamingcoord/server/broadcaster/broadcaster.go index 79e77bb8829cf..b1b6b3c633fc4 100644 --- a/internal/streamingcoord/server/broadcaster/broadcaster.go +++ b/internal/streamingcoord/server/broadcaster/broadcaster.go @@ -3,7 +3,7 @@ package broadcaster import ( "context" - "github.com/milvus-io/milvus/internal/distributed/streaming" + "github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/registry" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/types" ) @@ -16,9 +16,4 @@ type Broadcaster interface { Close() } -// AppendOperator is used to append messages, there's only two implement of this interface: -// 1. streaming.WAL() -// 2. old msgstream interface -type AppendOperator interface { - AppendMessages(ctx context.Context, msgs ...message.MutableMessage) streaming.AppendResponses -} +type AppendOperator = registry.AppendOperator diff --git a/internal/streamingcoord/server/broadcaster/broadcaster_impl.go b/internal/streamingcoord/server/broadcaster/broadcaster_impl.go index 8019ac9c779d4..0b63ba32880b8 100644 --- a/internal/streamingcoord/server/broadcaster/broadcaster_impl.go +++ b/internal/streamingcoord/server/broadcaster/broadcaster_impl.go @@ -15,13 +15,15 @@ import ( "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/contextutil" + "github.com/milvus-io/milvus/pkg/util/hardware" + "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/syncutil" "github.com/milvus-io/milvus/pkg/util/typeutil" ) func RecoverBroadcaster( ctx context.Context, - appendOperator AppendOperator, + appendOperator *syncutil.Future[AppendOperator], ) (Broadcaster, error) { logger := resource.Resource().Logger().With(log.FieldComponent("broadcaster")) tasks, err := resource.Resource().StreamingCatalog().ListBroadcastTask(ctx) @@ -61,7 +63,7 @@ type broadcasterImpl struct { pendingChan chan *broadcastTask backoffChan chan *broadcastTask workerChan chan *broadcastTask - appendOperator AppendOperator + appendOperator *syncutil.Future[AppendOperator] // TODO: we can remove those lazy future in 2.6.0, by remove the msgstream broadcaster. } // Broadcast broadcasts the message to all channels. @@ -126,21 +128,34 @@ func (b *broadcasterImpl) Close() { // execute the broadcaster func (b *broadcasterImpl) execute() { - b.logger.Info("broadcaster start to execute") + workers := int(float64(hardware.GetCPUNum()) * paramtable.Get().StreamingCfg.WALBroadcasterConcurrencyRatio.GetAsFloat()) + if workers < 1 { + workers = 1 + } + b.logger.Info("broadcaster start to execute", zap.Int("workerNum", workers)) + defer func() { b.backgroundTaskNotifier.Finish(struct{}{}) b.logger.Info("broadcaster execute exit") }() + // Wait for appendOperator ready + appendOperator, err := b.appendOperator.GetWithContext(b.backgroundTaskNotifier.Context()) + if err != nil { + b.logger.Info("broadcaster is closed before appendOperator ready") + return + } + b.logger.Info("broadcaster appendOperator ready, begin to start workers and dispatch") + // Start n workers to handle the broadcast task. wg := sync.WaitGroup{} - for i := 0; i < 4; i++ { + for i := 0; i < workers; i++ { i := i // Start n workers to handle the broadcast task. wg.Add(1) go func() { defer wg.Done() - b.worker(i) + b.worker(i, appendOperator) }() } defer wg.Wait() @@ -174,8 +189,13 @@ func (b *broadcasterImpl) dispatch() { b.backoffs.Push(task) case <-nextBackOff: // backoff is done, move all the backoff done task into pending to retry. + newPops := make([]*broadcastTask, 0) for b.backoffs.Len() > 0 && b.backoffs.Peek().NextInterval() < time.Millisecond { - b.pendings = append(b.pendings, b.backoffs.Pop()) + newPops = append(newPops, b.backoffs.Pop()) + } + if len(newPops) > 0 { + // Push the backoff task into pendings front. + b.pendings = append(newPops, b.pendings...) } case workerChan <- nextTask: // The task is sent to worker, remove it from pending list. @@ -184,9 +204,10 @@ func (b *broadcasterImpl) dispatch() { } } -func (b *broadcasterImpl) worker(no int) { +func (b *broadcasterImpl) worker(no int, appendOperator AppendOperator) { + logger := b.logger.With(zap.Int("workerNo", no)) defer func() { - b.logger.Info("broadcaster worker exit", zap.Int("no", no)) + logger.Info("broadcaster worker exit") }() for { @@ -194,7 +215,7 @@ func (b *broadcasterImpl) worker(no int) { case <-b.backgroundTaskNotifier.Context().Done(): return case task := <-b.workerChan: - if err := task.Poll(b.backgroundTaskNotifier.Context(), b.appendOperator); err != nil { + if err := task.Execute(b.backgroundTaskNotifier.Context(), appendOperator); err != nil { // If the task is not done, repush it into pendings and retry infinitely. select { case <-b.backgroundTaskNotifier.Context().Done(): diff --git a/internal/streamingcoord/server/broadcaster/broadcaster_test.go b/internal/streamingcoord/server/broadcaster/broadcaster_test.go index de0cc86b3b887..738057dfa30cb 100644 --- a/internal/streamingcoord/server/broadcaster/broadcaster_test.go +++ b/internal/streamingcoord/server/broadcaster/broadcaster_test.go @@ -12,7 +12,6 @@ import ( "go.uber.org/atomic" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" - "github.com/milvus-io/milvus/internal/distributed/streaming" "github.com/milvus-io/milvus/internal/mocks/mock_metastore" "github.com/milvus-io/milvus/internal/mocks/streamingcoord/server/mock_broadcaster" "github.com/milvus-io/milvus/internal/streamingcoord/server/resource" @@ -23,10 +22,13 @@ import ( "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/walimplstest" + "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/syncutil" ) func TestBroadcaster(t *testing.T) { + paramtable.Init() + meta := mock_metastore.NewMockStreamingCoordCataLog(t) meta.EXPECT().ListBroadcastTask(mock.Anything). RunAndReturn(func(ctx context.Context) ([]*streamingpb.BroadcastTask, error) { @@ -39,7 +41,7 @@ func TestBroadcaster(t *testing.T) { done := atomic.NewInt64(0) meta.EXPECT().SaveBroadcastTask(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, bt *streamingpb.BroadcastTask) error { // may failure - if rand.Int31n(10) < 5 { + if rand.Int31n(10) < 3 { return errors.New("save task failed") } if bt.State == streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_DONE { @@ -58,7 +60,7 @@ func TestBroadcaster(t *testing.T) { assert.NotNil(t, bc) assert.Eventually(t, func() bool { return appended.Load() == 6 && done.Load() == 3 - }, 10*time.Second, 10*time.Millisecond) + }, 30*time.Second, 10*time.Millisecond) var result *types.BroadcastAppendResult for { @@ -73,7 +75,7 @@ func TestBroadcaster(t *testing.T) { assert.Eventually(t, func() bool { return done.Load() == 4 - }, 10*time.Second, 10*time.Millisecond) + }, 30*time.Second, 10*time.Millisecond) // TODO: error path. bc.Close() @@ -83,23 +85,23 @@ func TestBroadcaster(t *testing.T) { assert.Nil(t, result) } -func createOpeartor(t *testing.T) (AppendOperator, *atomic.Int64) { +func createOpeartor(t *testing.T) (*syncutil.Future[AppendOperator], *atomic.Int64) { id := atomic.NewInt64(1) appended := atomic.NewInt64(0) operator := mock_broadcaster.NewMockAppendOperator(t) - f := func(ctx context.Context, msgs ...message.MutableMessage) streaming.AppendResponses { - resps := streaming.AppendResponses{ - Responses: make([]streaming.AppendResponse, len(msgs)), + f := func(ctx context.Context, msgs ...message.MutableMessage) types.AppendResponses { + resps := types.AppendResponses{ + Responses: make([]types.AppendResponse, len(msgs)), } for idx := range msgs { newID := walimplstest.NewTestMessageID(id.Inc()) - if rand.Int31n(10) < 5 { - resps.Responses[idx] = streaming.AppendResponse{ + if rand.Int31n(10) < 3 { + resps.Responses[idx] = types.AppendResponse{ Error: errors.New("append failed"), } continue } - resps.Responses[idx] = streaming.AppendResponse{ + resps.Responses[idx] = types.AppendResponse{ AppendResult: &types.AppendResult{ MessageID: newID, TimeTick: uint64(time.Now().UnixMilli()), @@ -114,7 +116,10 @@ func createOpeartor(t *testing.T) (AppendOperator, *atomic.Int64) { operator.EXPECT().AppendMessages(mock.Anything, mock.Anything, mock.Anything).RunAndReturn(f) operator.EXPECT().AppendMessages(mock.Anything, mock.Anything, mock.Anything, mock.Anything).RunAndReturn(f) operator.EXPECT().AppendMessages(mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).RunAndReturn(f) - return operator, appended + + fOperator := syncutil.NewFuture[AppendOperator]() + fOperator.Set(operator) + return fOperator, appended } func createNewBroadcastMsg(vchannels []string) message.BroadcastMutableMessage { diff --git a/internal/streamingcoord/server/broadcaster/registry/append_operator.go b/internal/streamingcoord/server/broadcaster/registry/append_operator.go new file mode 100644 index 0000000000000..e5e59d8bce1c5 --- /dev/null +++ b/internal/streamingcoord/server/broadcaster/registry/append_operator.go @@ -0,0 +1,42 @@ +package registry + +import ( + "context" + + "github.com/milvus-io/milvus/internal/util/streamingutil" + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/streaming/util/types" + "github.com/milvus-io/milvus/pkg/util/syncutil" +) + +type AppendOperatorType int + +const ( + AppendOperatorTypeMsgstream AppendOperatorType = iota + 1 + AppendOperatorTypeStreaming +) + +var localRegistry = make(map[AppendOperatorType]*syncutil.Future[AppendOperator]) + +// AppendOperator is used to append messages, there's only two implement of this interface: +// 1. streaming.WAL() +// 2. old msgstream interface +type AppendOperator interface { + AppendMessages(ctx context.Context, msgs ...message.MutableMessage) types.AppendResponses +} + +func init() { + localRegistry[AppendOperatorTypeMsgstream] = syncutil.NewFuture[AppendOperator]() + localRegistry[AppendOperatorTypeStreaming] = syncutil.NewFuture[AppendOperator]() +} + +func Register(typ AppendOperatorType, op AppendOperator) { + localRegistry[typ].Set(op) +} + +func GetAppendOperator() *syncutil.Future[AppendOperator] { + if streamingutil.IsStreamingServiceEnabled() { + return localRegistry[AppendOperatorTypeStreaming] + } + return localRegistry[AppendOperatorTypeMsgstream] +} diff --git a/internal/streamingcoord/server/broadcaster/registry/test_utility.go b/internal/streamingcoord/server/broadcaster/registry/test_utility.go new file mode 100644 index 0000000000000..1e555ff26136c --- /dev/null +++ b/internal/streamingcoord/server/broadcaster/registry/test_utility.go @@ -0,0 +1,12 @@ +//go:build test +// +build test + +package registry + +import "github.com/milvus-io/milvus/pkg/util/syncutil" + +func ResetRegistration() { + localRegistry = make(map[AppendOperatorType]*syncutil.Future[AppendOperator]) + localRegistry[AppendOperatorTypeMsgstream] = syncutil.NewFuture[AppendOperator]() + localRegistry[AppendOperatorTypeStreaming] = syncutil.NewFuture[AppendOperator]() +} diff --git a/internal/streamingcoord/server/broadcaster/task.go b/internal/streamingcoord/server/broadcaster/task.go index fff789e200729..38693bf509515 100644 --- a/internal/streamingcoord/server/broadcaster/task.go +++ b/internal/streamingcoord/server/broadcaster/task.go @@ -49,15 +49,17 @@ type broadcastTask struct { *typeutil.BackoffWithInstant } -// Poll polls the task, return nil if the task is done, otherwise not done. -// Poll can be repeated called until the task is done. -func (b *broadcastTask) Poll(ctx context.Context, operator AppendOperator) error { +// Execute reexecute the task, return nil if the task is done, otherwise not done. +// Execute can be repeated called until the task is done. +// Same semantics as the `Poll` operation in eventloop. +func (b *broadcastTask) Execute(ctx context.Context, operator AppendOperator) error { if len(b.pendingMessages) > 0 { b.logger.Debug("broadcast task is polling to make sent...", zap.Int("pendingMessages", len(b.pendingMessages))) resps := operator.AppendMessages(ctx, b.pendingMessages...) newPendings := make([]message.MutableMessage, 0) for idx, resp := range resps.Responses { if resp.Error != nil { + b.logger.Warn("broadcast task append message failed", zap.Int("idx", idx), zap.Error(resp.Error)) newPendings = append(newPendings, b.pendingMessages[idx]) continue } @@ -67,7 +69,7 @@ func (b *broadcastTask) Poll(ctx context.Context, operator AppendOperator) error if len(newPendings) == 0 { b.future.Set(&types.BroadcastAppendResult{AppendResults: b.appendResult}) } - b.logger.Info("broadcast task make a new broadcast done", zap.Int("pendingMessages", len(b.pendingMessages))) + b.logger.Info("broadcast task make a new broadcast done", zap.Int("backoffRetryMessages", len(b.pendingMessages))) } if len(b.pendingMessages) == 0 { // There's no more pending message, mark the task as done. diff --git a/internal/streamingcoord/server/server.go b/internal/streamingcoord/server/server.go index 67726bcf01e06..cb0a7e65e85ba 100644 --- a/internal/streamingcoord/server/server.go +++ b/internal/streamingcoord/server/server.go @@ -9,6 +9,7 @@ import ( "github.com/milvus-io/milvus/internal/streamingcoord/server/balancer" _ "github.com/milvus-io/milvus/internal/streamingcoord/server/balancer/policy" // register the balancer policy "github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster" + "github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/registry" "github.com/milvus-io/milvus/internal/streamingcoord/server/service" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/streamingutil" @@ -69,7 +70,7 @@ func (s *Server) initBasicComponent(ctx context.Context) (err error) { // So we need to recover it. futures = append(futures, conc.Go(func() (struct{}, error) { s.logger.Info("start recovery broadcaster...") - broadcaster, err := broadcaster.RecoverBroadcaster(ctx, broadcaster.NewAppendOperator()) + broadcaster, err := broadcaster.RecoverBroadcaster(ctx, registry.GetAppendOperator()) if err != nil { s.logger.Warn("recover broadcaster failed", zap.Error(err)) return struct{}{}, err diff --git a/pkg/streaming/util/message/adaptor/broadcast_message.go b/pkg/streaming/util/message/adaptor/broadcast_message.go new file mode 100644 index 0000000000000..69bcde5dad58a --- /dev/null +++ b/pkg/streaming/util/message/adaptor/broadcast_message.go @@ -0,0 +1,25 @@ +package adaptor + +import ( + "github.com/cockroachdb/errors" + + "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/streaming/util/message" +) + +func NewMsgPackFromMutableMessageV1(msg message.MutableMessage) (msgstream.TsMsg, error) { + if msg.Version() != message.VersionV1 { + return nil, errors.New("Invalid message version") + } + + tsMsg, err := unmashalerDispatcher.Unmarshal(msg.Payload(), MustGetCommonpbMsgTypeFromMessageType(msg.MessageType())) + if err != nil { + return nil, errors.Wrap(err, "Failed to unmarshal message") + } + return recoverMutableMessageFromHeader(tsMsg, msg) +} + +func recoverMutableMessageFromHeader(tsMsg msgstream.TsMsg, _ message.MutableMessage) (msgstream.TsMsg, error) { + // TODO: fillback the header information to tsMsg + return tsMsg, nil +} diff --git a/pkg/streaming/util/types/responses.go b/pkg/streaming/util/types/responses.go new file mode 100644 index 0000000000000..7f9dcf6908f89 --- /dev/null +++ b/pkg/streaming/util/types/responses.go @@ -0,0 +1,113 @@ +package types + +import ( + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/types/known/anypb" + + "github.com/milvus-io/milvus/pkg/proto/messagespb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/streaming/util/message" +) + +// BroadcastAppendResult is the result of broadcast append operation. +type BroadcastAppendResult struct { + AppendResults map[string]*AppendResult // make the channel name to the append result. +} + +// GetAppendResult returns the append result of the given channel. +func (r *BroadcastAppendResult) GetAppendResult(channelName string) *AppendResult { + return r.AppendResults[channelName] +} + +// AppendResult is the result of append operation. +type AppendResult struct { + // MessageID is generated by underlying walimpls. + MessageID message.MessageID + + // TimeTick is the time tick of the message. + // Set by timetick interceptor. + TimeTick uint64 + + // TxnCtx is the transaction context of the message. + // If the message is not belong to a transaction, the TxnCtx will be nil. + TxnCtx *message.TxnContext + + // Extra is the extra information of the append result. + Extra *anypb.Any +} + +// GetExtra unmarshal the extra information to the given message. +func (r *AppendResult) GetExtra(m proto.Message) error { + return anypb.UnmarshalTo(r.Extra, m, proto.UnmarshalOptions{ + DiscardUnknown: true, + AllowPartial: true, + }) +} + +// IntoProto converts the append result to proto. +func (r *AppendResult) IntoProto() *streamingpb.ProduceMessageResponseResult { + return &streamingpb.ProduceMessageResponseResult{ + Id: &messagespb.MessageID{ + Id: r.MessageID.Marshal(), + }, + Timetick: r.TimeTick, + TxnContext: r.TxnCtx.IntoProto(), + Extra: r.Extra, + } +} + +// NewAppendResponseN creates a new append response. +func NewAppendResponseN(n int) AppendResponses { + return AppendResponses{ + Responses: make([]AppendResponse, n), + } +} + +// AppendResponse is the response of one append operation. +type AppendResponse struct { + AppendResult *AppendResult + Error error +} + +// AppendResponses is the response of append operation. +type AppendResponses struct { + Responses []AppendResponse +} + +func (a AppendResponses) MaxTimeTick() uint64 { + var maxTimeTick uint64 + for _, r := range a.Responses { + if r.AppendResult != nil && r.AppendResult.TimeTick > maxTimeTick { + maxTimeTick = r.AppendResult.TimeTick + } + } + return maxTimeTick +} + +// UnwrapFirstError returns the first error in the responses. +func (a AppendResponses) UnwrapFirstError() error { + for _, r := range a.Responses { + if r.Error != nil { + return r.Error + } + } + return nil +} + +// FillAllError fills all the responses with the same error. +func (a *AppendResponses) FillAllError(err error) { + for i := range a.Responses { + a.Responses[i].Error = err + } +} + +// FillResponseAtIdx fill the response at idx +func (a *AppendResponses) FillResponseAtIdx(resp AppendResponse, idx int) { + a.Responses[idx] = resp +} + +func (a *AppendResponses) FillAllResponse(resp AppendResponse) { + for i := range a.Responses { + a.Responses[i] = resp + } +} diff --git a/pkg/streaming/util/types/streaming_node.go b/pkg/streaming/util/types/streaming_node.go index bc4118cb30a65..f2a68b99e2515 100644 --- a/pkg/streaming/util/types/streaming_node.go +++ b/pkg/streaming/util/types/streaming_node.go @@ -4,12 +4,8 @@ import ( "context" "github.com/cockroachdb/errors" - "google.golang.org/protobuf/proto" - "google.golang.org/protobuf/types/known/anypb" - "github.com/milvus-io/milvus/pkg/proto/messagespb" "github.com/milvus-io/milvus/pkg/proto/streamingpb" - "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/util/typeutil" ) @@ -88,50 +84,3 @@ func (n *StreamingNodeStatus) ErrorOfNode() error { } return n.Err } - -// BroadcastAppendResult is the result of broadcast append operation. -type BroadcastAppendResult struct { - AppendResults map[string]*AppendResult // make the channel name to the append result. -} - -// GetAppendResult returns the append result of the given channel. -func (r *BroadcastAppendResult) GetAppendResult(channelName string) *AppendResult { - return r.AppendResults[channelName] -} - -// AppendResult is the result of append operation. -type AppendResult struct { - // MessageID is generated by underlying walimpls. - MessageID message.MessageID - - // TimeTick is the time tick of the message. - // Set by timetick interceptor. - TimeTick uint64 - - // TxnCtx is the transaction context of the message. - // If the message is not belong to a transaction, the TxnCtx will be nil. - TxnCtx *message.TxnContext - - // Extra is the extra information of the append result. - Extra *anypb.Any -} - -// GetExtra unmarshal the extra information to the given message. -func (r *AppendResult) GetExtra(m proto.Message) error { - return anypb.UnmarshalTo(r.Extra, m, proto.UnmarshalOptions{ - DiscardUnknown: true, - AllowPartial: true, - }) -} - -// IntoProto converts the append result to proto. -func (r *AppendResult) IntoProto() *streamingpb.ProduceMessageResponseResult { - return &streamingpb.ProduceMessageResponseResult{ - Id: &messagespb.MessageID{ - Id: r.MessageID.Marshal(), - }, - Timetick: r.TimeTick, - TxnContext: r.TxnCtx.IntoProto(), - Extra: r.Extra, - } -} diff --git a/pkg/util/paramtable/component_param.go b/pkg/util/paramtable/component_param.go index 37b6a2fdf7380..b16f836d96180 100644 --- a/pkg/util/paramtable/component_param.go +++ b/pkg/util/paramtable/component_param.go @@ -4828,6 +4828,9 @@ type streamingConfig struct { WALBalancerBackoffInitialInterval ParamItem `refreshable:"true"` WALBalancerBackoffMultiplier ParamItem `refreshable:"true"` + // broadcaster + WALBroadcasterConcurrencyRatio ParamItem `refreshable:"false"` + // txn TxnDefaultKeepaliveTimeout ParamItem `refreshable:"true"` } @@ -4861,6 +4864,15 @@ It's ok to set it into duration string, such as 30s or 1m30s, see time.ParseDura } p.WALBalancerBackoffMultiplier.Init(base.mgr) + p.WALBroadcasterConcurrencyRatio = ParamItem{ + Key: "streaming.walBroadcaster.concurrencyRatio", + Version: "2.5.4", + Doc: `The concurrency ratio based on number of CPU for wal broadcaster, 1 by default.`, + DefaultValue: "1", + Export: true, + } + p.WALBroadcasterConcurrencyRatio.Init(base.mgr) + // txn p.TxnDefaultKeepaliveTimeout = ParamItem{ Key: "streaming.txn.defaultKeepaliveTimeout", diff --git a/pkg/util/paramtable/component_param_test.go b/pkg/util/paramtable/component_param_test.go index 2fbe4079ccfbb..9b95f3384ae4a 100644 --- a/pkg/util/paramtable/component_param_test.go +++ b/pkg/util/paramtable/component_param_test.go @@ -614,14 +614,17 @@ func TestComponentParam(t *testing.T) { assert.Equal(t, 1*time.Minute, params.StreamingCfg.WALBalancerTriggerInterval.GetAsDurationByParse()) assert.Equal(t, 50*time.Millisecond, params.StreamingCfg.WALBalancerBackoffInitialInterval.GetAsDurationByParse()) assert.Equal(t, 2.0, params.StreamingCfg.WALBalancerBackoffMultiplier.GetAsFloat()) + assert.Equal(t, 1.0, params.StreamingCfg.WALBroadcasterConcurrencyRatio.GetAsFloat()) assert.Equal(t, 10*time.Second, params.StreamingCfg.TxnDefaultKeepaliveTimeout.GetAsDurationByParse()) params.Save(params.StreamingCfg.WALBalancerTriggerInterval.Key, "50s") params.Save(params.StreamingCfg.WALBalancerBackoffInitialInterval.Key, "50s") params.Save(params.StreamingCfg.WALBalancerBackoffMultiplier.Key, "3.5") + params.Save(params.StreamingCfg.WALBroadcasterConcurrencyRatio.Key, "1.5") params.Save(params.StreamingCfg.TxnDefaultKeepaliveTimeout.Key, "3500ms") assert.Equal(t, 50*time.Second, params.StreamingCfg.WALBalancerTriggerInterval.GetAsDurationByParse()) assert.Equal(t, 50*time.Second, params.StreamingCfg.WALBalancerBackoffInitialInterval.GetAsDurationByParse()) assert.Equal(t, 3.5, params.StreamingCfg.WALBalancerBackoffMultiplier.GetAsFloat()) + assert.Equal(t, 1.5, params.StreamingCfg.WALBroadcasterConcurrencyRatio.GetAsFloat()) assert.Equal(t, 3500*time.Millisecond, params.StreamingCfg.TxnDefaultKeepaliveTimeout.GetAsDurationByParse()) }) diff --git a/tests/integration/coorddownsearch/search_after_coord_down_test.go b/tests/integration/coorddownsearch/search_after_coord_down_test.go index 1b02bd0a6e894..ead15e6f19aa5 100644 --- a/tests/integration/coorddownsearch/search_after_coord_down_test.go +++ b/tests/integration/coorddownsearch/search_after_coord_down_test.go @@ -30,6 +30,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" + "github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/registry" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/funcutil" @@ -302,6 +303,8 @@ func (s *CoordDownSearch) searchAfterCoordDown() float64 { s.searchFailed(searchCollectionName, Dim, commonpb.ConsistencyLevel_Strong) log.Info(fmt.Sprintf("=========================Failed search cost: %fs=========================", time.Since(failedStart).Seconds())) + registry.ResetRegistration() + log.Info("=========================restart Root Coordinators=========================") c.StartRootCoord() s.search(searchCollectionName, Dim, commonpb.ConsistencyLevel_Eventually) diff --git a/tests/integration/coordrecovery/coord_recovery_test.go b/tests/integration/coordrecovery/coord_recovery_test.go index 46f02c593bbe6..b80c63023af71 100644 --- a/tests/integration/coordrecovery/coord_recovery_test.go +++ b/tests/integration/coordrecovery/coord_recovery_test.go @@ -30,6 +30,7 @@ import ( "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/streamingcoord/server/broadcaster/registry" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/funcutil" @@ -242,6 +243,8 @@ func (s *CoordSwitchSuite) switchCoord() float64 { log.Info("=========================Coordinators stopped=========================", zap.Duration("elapsed", time.Since(start))) start = time.Now() + registry.ResetRegistration() + c.StartRootCoord() log.Info("=========================RootCoord restarted=========================") c.StartDataCoord() diff --git a/tests/integration/suite.go b/tests/integration/suite.go index 68751b40f1b39..884bf945b741a 100644 --- a/tests/integration/suite.go +++ b/tests/integration/suite.go @@ -30,6 +30,7 @@ import ( "go.uber.org/zap/zapcore" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + "github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/registry" "github.com/milvus-io/milvus/internal/util/hookutil" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/etcd" @@ -169,4 +170,5 @@ func (s *MiniClusterSuite) TearDownTest() { if s.Cluster != nil { s.Cluster.Stop() } + registry.ResetRegistration() } From 097d167e96d30cf01c9eb2176c3958c39ac88caa Mon Sep 17 00:00:00 2001 From: junjiejiangjjj Date: Tue, 14 Jan 2025 16:53:00 +0800 Subject: [PATCH 34/34] doc: Update tools info (#39244) Signed-off-by: junjie.jiang --- docs/design_docs/segcore/scripts_and_tools.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/design_docs/segcore/scripts_and_tools.md b/docs/design_docs/segcore/scripts_and_tools.md index e59e7f3b7b384..41800c6670342 100644 --- a/docs/design_docs/segcore/scripts_and_tools.md +++ b/docs/design_docs/segcore/scripts_and_tools.md @@ -14,3 +14,6 @@ The following scripts and commands may be used during segcore development. - if license info is added - if `cpplint.py` standard meets , might need to be fixed by hand - `make verifier` also includes functions in `make cppcheck` + +- under milvus/ directory + - use `make static-check` to check golang code format