diff --git a/internal/core/src/exec/expression/UnaryExpr.cpp b/internal/core/src/exec/expression/UnaryExpr.cpp index 07bba2b9c9..4bfe9f547f 100644 --- a/internal/core/src/exec/expression/UnaryExpr.cpp +++ b/internal/core/src/exec/expression/UnaryExpr.cpp @@ -1776,7 +1776,8 @@ PhyUnaryRangeFilterExpr::ExecTextMatch() { } if (cached_match_res_ == nullptr) { - auto index = segment_->GetTextIndex(op_ctx_, field_id_); + auto pw = segment_->GetTextIndex(op_ctx_, field_id_); + auto index = pw.get(); auto res = std::move(func(index, query)); auto valid_res = index->IsNotNull(); cached_match_res_ = std::make_shared(std::move(res)); diff --git a/internal/core/src/index/TextMatchIndex.h b/internal/core/src/index/TextMatchIndex.h index 7400668b46..ae894d7b7d 100644 --- a/internal/core/src/index/TextMatchIndex.h +++ b/internal/core/src/index/TextMatchIndex.h @@ -14,6 +14,7 @@ #include #include +#include "cachinglayer/Manager.h" #include "index/InvertedIndexTantivy.h" #include "index/IndexStats.h" @@ -96,4 +97,33 @@ class TextMatchIndex : public InvertedIndexTantivy { std::atomic last_commit_time_; int64_t commit_interval_in_ms_; }; + +class TextMatchIndexHolder { + public: + explicit TextMatchIndexHolder( + std::unique_ptr index) + : index_(std::move(index)), size_(index_ ? index_->ByteSize() : 0) { + if (size_ > 0) { + milvus::cachinglayer::Manager::GetInstance().ChargeLoadedResource( + {size_, 0}); + } + } + + ~TextMatchIndexHolder() { + if (size_ > 0) { + milvus::cachinglayer::Manager::GetInstance().RefundLoadedResource( + {size_, 0}); + } + } + + milvus::index::TextMatchIndex* + get() const { + return index_.get(); + } + + private: + std::unique_ptr index_; + int64_t size_; +}; + } // namespace milvus::index diff --git a/internal/core/src/segcore/ChunkedSegmentSealedImpl.cpp b/internal/core/src/segcore/ChunkedSegmentSealedImpl.cpp index 9c78a7a03d..80510298da 100644 --- a/internal/core/src/segcore/ChunkedSegmentSealedImpl.cpp +++ b/internal/core/src/segcore/ChunkedSegmentSealedImpl.cpp @@ -62,9 +62,11 @@ #include "mmap/ChunkedColumnInterface.h" #include "mmap/ChunkedColumnGroup.h" #include "segcore/storagev1translator/InterimSealedIndexTranslator.h" +#include "segcore/storagev1translator/TextMatchIndexTranslator.h" #include "storage/Util.h" #include "storage/ThreadPools.h" #include "storage/MmapManager.h" +#include "storage/RemoteChunkManagerSingleton.h" #include "milvus-storage/filesystem/fs.h" #include "cachinglayer/CacheSlot.h" #include "storage/LocalChunkManagerSingleton.h" @@ -1688,17 +1690,60 @@ ChunkedSegmentSealedImpl::CreateTextIndex(FieldId field_id) { index->RegisterTokenizer("milvus_tokenizer", field_meta.get_analyzer_params().c_str()); - text_indexes_[field_id] = std::move(index); + text_indexes_[field_id] = + std::make_shared(std::move(index)); } void ChunkedSegmentSealedImpl::LoadTextIndex( - FieldId field_id, std::unique_ptr index) { + std::unique_ptr info_proto) { std::unique_lock lck(mutex_); + + milvus::storage::FieldDataMeta field_data_meta{info_proto->collectionid(), + info_proto->partitionid(), + this->get_segment_id(), + info_proto->fieldid(), + info_proto->schema()}; + milvus::storage::IndexMeta index_meta{this->get_segment_id(), + info_proto->fieldid(), + info_proto->buildid(), + info_proto->version()}; + auto remote_chunk_manager = + milvus::storage::RemoteChunkManagerSingleton::GetInstance() + .GetRemoteChunkManager(); + auto fs = milvus_storage::ArrowFileSystemSingleton::GetInstance() + .GetArrowFileSystem(); + AssertInfo(fs != nullptr, "arrow file system is null"); + + milvus::Config config; + std::vector files; + for (const auto& f : info_proto->files()) { + files.push_back(f); + } + config[milvus::index::INDEX_FILES] = files; + config[milvus::LOAD_PRIORITY] = info_proto->load_priority(); + config[milvus::index::ENABLE_MMAP] = info_proto->enable_mmap(); + milvus::storage::FileManagerContext file_ctx( + field_data_meta, index_meta, remote_chunk_manager, fs); + + auto field_id = milvus::FieldId(info_proto->fieldid()); const auto& field_meta = schema_->operator[](field_id); - index->RegisterTokenizer("milvus_tokenizer", - field_meta.get_analyzer_params().c_str()); - text_indexes_[field_id] = std::move(index); + milvus::segcore::storagev1translator::TextMatchIndexLoadInfo load_info{ + info_proto->enable_mmap(), + this->get_segment_id(), + info_proto->fieldid(), + field_meta.get_analyzer_params(), + info_proto->index_size()}; + + std::unique_ptr< + milvus::cachinglayer::Translator> + translator = std::make_unique< + milvus::segcore::storagev1translator::TextMatchIndexTranslator>( + load_info, file_ctx, config); + auto cache_slot = + milvus::cachinglayer::Manager::GetInstance().CreateCacheSlot( + std::move(translator)); + text_indexes_[field_id] = std::move(cache_slot); } std::unique_ptr diff --git a/internal/core/src/segcore/ChunkedSegmentSealedImpl.h b/internal/core/src/segcore/ChunkedSegmentSealedImpl.h index 1e42234bb0..ed73f18431 100644 --- a/internal/core/src/segcore/ChunkedSegmentSealedImpl.h +++ b/internal/core/src/segcore/ChunkedSegmentSealedImpl.h @@ -38,6 +38,7 @@ #include "segcore/SegcoreConfig.h" #include "folly/concurrency/ConcurrentHashMap.h" #include "index/json_stats/JsonKeyStats.h" +#include "pb/index_cgo_msg.pb.h" namespace milvus::segcore { @@ -127,8 +128,8 @@ class ChunkedSegmentSealedImpl : public SegmentSealed { CreateTextIndex(FieldId field_id) override; void - LoadTextIndex(FieldId field_id, - std::unique_ptr index) override; + LoadTextIndex(std::unique_ptr + info_proto) override; void LoadJsonStats(FieldId field_id, diff --git a/internal/core/src/segcore/SegmentGrowingImpl.cpp b/internal/core/src/segcore/SegmentGrowingImpl.cpp index 626883ff7b..972f157adc 100644 --- a/internal/core/src/segcore/SegmentGrowingImpl.cpp +++ b/internal/core/src/segcore/SegmentGrowingImpl.cpp @@ -385,7 +385,8 @@ SegmentGrowingImpl::load_field_data_common( // build text match index if (field_meta.enable_match()) { - auto index = GetTextIndex(nullptr, field_id); + auto pinned = GetTextIndex(nullptr, field_id); + auto index = pinned.get(); index->BuildIndexFromFieldData(field_data, field_meta.is_nullable()); index->Commit(); // Reload reader so that the index can be read immediately @@ -1265,7 +1266,16 @@ SegmentGrowingImpl::AddTexts(milvus::FieldId field_id, ErrorCode::TextIndexNotFound, fmt::format("text index not found for field {}", field_id.get())); } - iter->second->AddTextsGrowing(n, texts, texts_valid_data, offset_begin); + // only unique_ptr is supported for growing segment + if (auto p = std::get_if>( + &iter->second)) { + (*p)->AddTextsGrowing(n, texts, texts_valid_data, offset_begin); + } else { + throw SegcoreError(ErrorCode::UnexpectedError, + fmt::format("text index of growing segment is not a " + "unique_ptr for field {}", + field_id.get())); + } } void diff --git a/internal/core/src/segcore/SegmentInterface.cpp b/internal/core/src/segcore/SegmentInterface.cpp index 2959962333..da7425e2ae 100644 --- a/internal/core/src/segcore/SegmentInterface.cpp +++ b/internal/core/src/segcore/SegmentInterface.cpp @@ -439,17 +439,46 @@ SegmentInternalInterface::GetSkipIndex() const { return skip_index_; } -index::TextMatchIndex* +PinWrapper SegmentInternalInterface::GetTextIndex(milvus::OpContext* op_ctx, FieldId field_id) const { std::shared_lock lock(mutex_); auto iter = text_indexes_.find(field_id); if (iter == text_indexes_.end()) { throw SegcoreError( - ErrorCode::TextIndexNotFound, + milvus::ErrorCode::TextIndexNotFound, fmt::format("text index not found for field {}", field_id.get())); } - return iter->second.get(); + + auto make_pin = [&](auto&& alt) -> PinWrapper { + using Alt = std::decay_t; + + if constexpr (std::is_same_v< + Alt, + std::unique_ptr>) { + return PinWrapper(alt.get()); + } else if constexpr (std::is_same_v< + Alt, + std::shared_ptr< + milvus::index::TextMatchIndexHolder>>) { + return PinWrapper(alt, alt->get()); + } else if constexpr (std::is_same_v< + Alt, + std::shared_ptr< + milvus::cachinglayer::CacheSlot< + milvus::index::TextMatchIndex>>>) { + auto ca = SemiInlineGet(alt->PinCells(op_ctx, {0})); + return PinWrapper(ca, ca->get_cell_of(0)); + } else { + throw SegcoreError( + milvus::ErrorCode::UnexpectedError, + fmt::format( + "text index of segment is not supported for field {}", + field_id.get())); + } + }; + + return std::visit(make_pin, iter->second); } std::unique_ptr diff --git a/internal/core/src/segcore/SegmentInterface.h b/internal/core/src/segcore/SegmentInterface.h index e7aa29ea69..7810aa797b 100644 --- a/internal/core/src/segcore/SegmentInterface.h +++ b/internal/core/src/segcore/SegmentInterface.h @@ -142,7 +142,7 @@ class SegmentInterface { virtual void CreateTextIndex(FieldId field_id) = 0; - virtual index::TextMatchIndex* + virtual PinWrapper GetTextIndex(milvus::OpContext* op_ctx, FieldId field_id) const = 0; virtual std::vector> @@ -361,7 +361,7 @@ class SegmentInternalInterface : public SegmentInterface { virtual DataType GetFieldDataType(FieldId fieldId) const = 0; - index::TextMatchIndex* + PinWrapper GetTextIndex(milvus::OpContext* op_ctx, FieldId field_id) const override; virtual PinWrapper @@ -595,7 +595,12 @@ class SegmentInternalInterface : public SegmentInterface { SkipIndex skip_index_; // text-indexes used to do match. - std::unordered_map> + std::unordered_map< + FieldId, + std::variant, + std::shared_ptr, + std::shared_ptr>>> text_indexes_; // json stats cache (field_id -> CacheSlot of JsonKeyStats) diff --git a/internal/core/src/segcore/SegmentSealed.h b/internal/core/src/segcore/SegmentSealed.h index abdcf84fd3..530b2d7b3f 100644 --- a/internal/core/src/segcore/SegmentSealed.h +++ b/internal/core/src/segcore/SegmentSealed.h @@ -22,6 +22,7 @@ #include "index/Index.h" #include "index/JsonInvertedIndex.h" #include "index/JsonFlatIndex.h" +#include "pb/index_cgo_msg.pb.h" #include "pb/segcore.pb.h" #include "segcore/InsertRecord.h" #include "segcore/SegmentInterface.h" @@ -56,8 +57,8 @@ class SegmentSealed : public SegmentInternalInterface { int64_t count) const = 0; virtual void - LoadTextIndex(FieldId field_id, - std::unique_ptr index) = 0; + LoadTextIndex(std::unique_ptr + info_proto) = 0; virtual InsertRecord& get_insert_record() = 0; diff --git a/internal/core/src/segcore/segment_c.cpp b/internal/core/src/segcore/segment_c.cpp index 13668961c5..d87004c01a 100644 --- a/internal/core/src/segcore/segment_c.cpp +++ b/internal/core/src/segcore/segment_c.cpp @@ -465,39 +465,7 @@ LoadTextIndex(CSegmentInterface c_segment, std::make_unique(); info_proto->ParseFromArray(serialized_load_text_index_info, len); - milvus::storage::FieldDataMeta field_meta{info_proto->collectionid(), - info_proto->partitionid(), - segment->get_segment_id(), - info_proto->fieldid(), - info_proto->schema()}; - milvus::storage::IndexMeta index_meta{segment->get_segment_id(), - info_proto->fieldid(), - info_proto->buildid(), - info_proto->version()}; - auto remote_chunk_manager = - milvus::storage::RemoteChunkManagerSingleton::GetInstance() - .GetRemoteChunkManager(); - auto fs = milvus_storage::ArrowFileSystemSingleton::GetInstance() - .GetArrowFileSystem(); - AssertInfo(fs != nullptr, "arrow file system is null"); - - milvus::Config config; - std::vector files; - for (const auto& f : info_proto->files()) { - files.push_back(f); - } - config[milvus::index::INDEX_FILES] = files; - config[milvus::LOAD_PRIORITY] = info_proto->load_priority(); - config[milvus::index::ENABLE_MMAP] = info_proto->enable_mmap(); - milvus::storage::FileManagerContext ctx( - field_meta, index_meta, remote_chunk_manager, fs); - - auto index = std::make_unique(ctx); - index->Load(config); - - segment->LoadTextIndex(milvus::FieldId(info_proto->fieldid()), - std::move(index)); - + segment->LoadTextIndex(std::move(info_proto)); return milvus::SuccessCStatus(); } catch (std::exception& e) { return milvus::FailureCStatus(&e); diff --git a/internal/core/src/segcore/storagev1translator/TextMatchIndexTranslator.cpp b/internal/core/src/segcore/storagev1translator/TextMatchIndexTranslator.cpp new file mode 100644 index 0000000000..7f2380c9b2 --- /dev/null +++ b/internal/core/src/segcore/storagev1translator/TextMatchIndexTranslator.cpp @@ -0,0 +1,115 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include "segcore/storagev1translator/TextMatchIndexTranslator.h" + +#include + +#include "cachinglayer/CacheSlot.h" +#include "segcore/Utils.h" +#include "monitor/Monitor.h" +#include "common/ScopedTimer.h" + +namespace milvus::segcore::storagev1translator { + +TextMatchIndexTranslator::TextMatchIndexTranslator( + TextMatchIndexLoadInfo load_info, + milvus::storage::FileManagerContext file_manager_context, + milvus::Config config) + : load_info_(std::move(load_info)), + file_manager_context_(std::move(file_manager_context)), + config_(std::move(config)), + key_(fmt::format( + "seg_{}_textindex_{}", load_info_.segment_id, load_info_.field_id)), + meta_(load_info_.enable_mmap ? milvus::cachinglayer::StorageType::DISK + : milvus::cachinglayer::StorageType::MEMORY, + milvus::cachinglayer::CellIdMappingMode::ALWAYS_ZERO, + milvus::segcore::getCellDataType(/* is_vector */ false, + /* is_index */ true), + milvus::segcore::getCacheWarmupPolicy(/* is_vector */ false, + /* is_index */ true), + /* support_eviction */ false) { +} + +size_t +TextMatchIndexTranslator::num_cells() const { + return 1; +} + +milvus::cachinglayer::cid_t +TextMatchIndexTranslator::cell_id_of(milvus::cachinglayer::uid_t) const { + return 0; +} + +std::pair +TextMatchIndexTranslator::estimated_byte_size_of_cell( + milvus::cachinglayer::cid_t) const { + // ignore the cid checking, because there is only one cell + return {{load_info_.index_size, 0}, {2 * load_info_.index_size, 0}}; +} + +int64_t +TextMatchIndexTranslator::cells_storage_bytes( + const std::vector&) const { + // ignore the cids checking, because there is only one cell + constexpr int64_t MIN_STORAGE_BYTES = 1 * 1024 * 1024; + return std::max(load_info_.index_size, MIN_STORAGE_BYTES); +} + +const std::string& +TextMatchIndexTranslator::key() const { + return key_; +} + +std::vector>> +TextMatchIndexTranslator::get_cells( + const std::vector&) { + auto index = + std::make_unique(file_manager_context_); + + { + milvus::ScopedTimer timer( + "text_match_index_load", + [](double /*ms*/) { + // no specific metric defined for text match index load yet + }, + milvus::ScopedTimer::LogLevel::Info); + index->Load(config_); + index->RegisterTokenizer("milvus_tokenizer", + load_info_.analyzer_params.c_str()); + } + + LOG_INFO("load text match index success for field:{} of segment:{}", + load_info_.field_id, + load_info_.segment_id); + + index->SetCellSize({index->ByteSize(), 0}); + + std::vector>> + result; + result.emplace_back(std::make_pair(0, std::move(index))); + return result; +} + +milvus::cachinglayer::Meta* +TextMatchIndexTranslator::meta() { + return &meta_; +} + +} // namespace milvus::segcore::storagev1translator diff --git a/internal/core/src/segcore/storagev1translator/TextMatchIndexTranslator.h b/internal/core/src/segcore/storagev1translator/TextMatchIndexTranslator.h new file mode 100644 index 0000000000..2dd73317cf --- /dev/null +++ b/internal/core/src/segcore/storagev1translator/TextMatchIndexTranslator.h @@ -0,0 +1,79 @@ +// 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 + +#include "cachinglayer/Translator.h" +#include "common/Types.h" +#include "common/LoadInfo.h" +#include "index/TextMatchIndex.h" +#include "storage/FileManager.h" + +namespace milvus::segcore::storagev1translator { + +struct TextMatchIndexLoadInfo { + bool enable_mmap; + int64_t segment_id; + int64_t field_id; + std::string analyzer_params; + int64_t index_size; +}; + +// Translator for TextMatchIndex (non-knowhere index). It loads a single-cell +// TextMatchIndex instance for a sealed segment field and exposes it to the cache +// layer with a stable key and resource usage. +class TextMatchIndexTranslator + : public milvus::cachinglayer::Translator { + public: + TextMatchIndexTranslator( + TextMatchIndexLoadInfo load_info, + milvus::storage::FileManagerContext file_manager_context, + milvus::Config config); + + ~TextMatchIndexTranslator() override = default; + + size_t + num_cells() const override; + + milvus::cachinglayer::cid_t + cell_id_of(milvus::cachinglayer::uid_t uid) const override; + + std::pair + estimated_byte_size_of_cell(milvus::cachinglayer::cid_t cid) const override; + + int64_t + cells_storage_bytes( + const std::vector&) const override; + + const std::string& + key() const override; + + std::vector>> + get_cells(const std::vector& cids) override; + + milvus::cachinglayer::Meta* + meta() override; + + private: + TextMatchIndexLoadInfo load_info_; + milvus::storage::FileManagerContext file_manager_context_; + milvus::Config config_; + std::string key_; + milvus::cachinglayer::Meta meta_; +}; + +} // namespace milvus::segcore::storagev1translator diff --git a/internal/querynodev2/segments/segment.go b/internal/querynodev2/segments/segment.go index a5a359bc8c..bf1426af92 100644 --- a/internal/querynodev2/segments/segment.go +++ b/internal/querynodev2/segments/segment.go @@ -1196,6 +1196,7 @@ func (s *LocalSegment) LoadTextIndex(ctx context.Context, textLogs *datapb.TextI PartitionID: s.Partition(), LoadPriority: s.LoadInfo().GetPriority(), EnableMmap: enableMmap, + IndexSize: textLogs.GetMemorySize(), } marshaled, err := proto.Marshal(cgoProto) diff --git a/pkg/proto/index_cgo_msg.proto b/pkg/proto/index_cgo_msg.proto index a0c31575e2..fd017278e2 100644 --- a/pkg/proto/index_cgo_msg.proto +++ b/pkg/proto/index_cgo_msg.proto @@ -118,6 +118,7 @@ message LoadTextIndexInfo { int64 partitionID = 7; common.LoadPriority load_priority = 8; bool enable_mmap = 9; + int64 index_size = 10; } message LoadJsonKeyIndexInfo { diff --git a/pkg/proto/indexcgopb/index_cgo_msg.pb.go b/pkg/proto/indexcgopb/index_cgo_msg.pb.go index e485cf6253..71d309b6ac 100644 --- a/pkg/proto/indexcgopb/index_cgo_msg.pb.go +++ b/pkg/proto/indexcgopb/index_cgo_msg.pb.go @@ -997,6 +997,7 @@ type LoadTextIndexInfo struct { PartitionID int64 `protobuf:"varint,7,opt,name=partitionID,proto3" json:"partitionID,omitempty"` LoadPriority commonpb.LoadPriority `protobuf:"varint,8,opt,name=load_priority,json=loadPriority,proto3,enum=milvus.proto.common.LoadPriority" json:"load_priority,omitempty"` EnableMmap bool `protobuf:"varint,9,opt,name=enable_mmap,json=enableMmap,proto3" json:"enable_mmap,omitempty"` + IndexSize int64 `protobuf:"varint,10,opt,name=index_size,json=indexSize,proto3" json:"index_size,omitempty"` } func (x *LoadTextIndexInfo) Reset() { @@ -1094,6 +1095,13 @@ func (x *LoadTextIndexInfo) GetEnableMmap() bool { return false } +func (x *LoadTextIndexInfo) GetIndexSize() int64 { + if x != nil { + return x.IndexSize + } + return 0 +} + type LoadJsonKeyIndexInfo struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1417,7 +1425,7 @@ var file_index_cgo_msg_proto_rawDesc = []byte{ 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x65, 0x6e, 0x63, 0x72, 0x79, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x65, 0x6e, 0x63, 0x72, - 0x79, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x22, 0xe0, 0x02, 0x0a, 0x11, 0x4c, 0x6f, + 0x79, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x22, 0xff, 0x02, 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, @@ -1439,38 +1447,40 @@ var file_index_cgo_msg_proto_rawDesc = []byte{ 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x50, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79, 0x52, 0x0c, 0x6c, 0x6f, 0x61, 0x64, 0x50, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79, 0x12, 0x1f, 0x0a, 0x0b, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6d, 0x6d, 0x61, 0x70, 0x18, 0x09, 0x20, 0x01, 0x28, 0x08, - 0x52, 0x0a, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x4d, 0x6d, 0x61, 0x70, 0x22, 0xa6, 0x03, 0x0a, - 0x14, 0x4c, 0x6f, 0x61, 0x64, 0x4a, 0x73, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 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, 0x12, 0x46, 0x0a, 0x0d, 0x6c, 0x6f, 0x61, - 0x64, 0x5f, 0x70, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79, 0x18, 0x08, 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, 0x4c, 0x6f, 0x61, 0x64, 0x50, 0x72, 0x69, 0x6f, 0x72, - 0x69, 0x74, 0x79, 0x52, 0x0c, 0x6c, 0x6f, 0x61, 0x64, 0x50, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, - 0x79, 0x12, 0x1f, 0x0a, 0x0b, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6d, 0x6d, 0x61, 0x70, - 0x18, 0x09, 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, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6d, 0x6d, 0x61, 0x70, 0x44, - 0x69, 0x72, 0x50, 0x61, 0x74, 0x68, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, - 0x73, 0x69, 0x7a, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, - 0x73, 0x53, 0x69, 0x7a, 0x65, 0x42, 0x35, 0x5a, 0x33, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, - 0x63, 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x63, 0x67, 0x6f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x33, + 0x52, 0x0a, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x4d, 0x6d, 0x61, 0x70, 0x12, 0x1d, 0x0a, 0x0a, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x69, 0x7a, 0x65, 0x22, 0xa6, 0x03, 0x0a, 0x14, + 0x4c, 0x6f, 0x61, 0x64, 0x4a, 0x73, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 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, 0x12, 0x46, 0x0a, 0x0d, 0x6c, 0x6f, 0x61, 0x64, + 0x5f, 0x70, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79, 0x18, 0x08, 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, 0x4c, 0x6f, 0x61, 0x64, 0x50, 0x72, 0x69, 0x6f, 0x72, 0x69, + 0x74, 0x79, 0x52, 0x0c, 0x6c, 0x6f, 0x61, 0x64, 0x50, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79, + 0x12, 0x1f, 0x0a, 0x0b, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6d, 0x6d, 0x61, 0x70, 0x18, + 0x09, 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, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6d, 0x6d, 0x61, 0x70, 0x44, 0x69, + 0x72, 0x50, 0x61, 0x74, 0x68, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x73, + 0x69, 0x7a, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x73, + 0x53, 0x69, 0x7a, 0x65, 0x42, 0x35, 0x5a, 0x33, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, + 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x63, 0x67, 0x6f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x33, } var (