enhance: add cachinglayer management for TextMatchIndex (#44741)

issue: #41435, #44502

Signed-off-by: Shawn Wang <shawn.wang@zilliz.com>
This commit is contained in:
sparknack 2025-10-13 14:37:58 +08:00 committed by GitHub
parent 6d5b41644b
commit c8a4d6e2ef
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
14 changed files with 380 additions and 84 deletions

View File

@ -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<TargetBitmap>(std::move(res));

View File

@ -14,6 +14,7 @@
#include <string>
#include <boost/filesystem.hpp>
#include "cachinglayer/Manager.h"
#include "index/InvertedIndexTantivy.h"
#include "index/IndexStats.h"
@ -96,4 +97,33 @@ class TextMatchIndex : public InvertedIndexTantivy<std::string> {
std::atomic<stdclock::time_point> last_commit_time_;
int64_t commit_interval_in_ms_;
};
class TextMatchIndexHolder {
public:
explicit TextMatchIndexHolder(
std::unique_ptr<milvus::index::TextMatchIndex> 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<milvus::index::TextMatchIndex> index_;
int64_t size_;
};
} // namespace milvus::index

View File

@ -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<index::TextMatchIndexHolder>(std::move(index));
}
void
ChunkedSegmentSealedImpl::LoadTextIndex(
FieldId field_id, std::unique_ptr<index::TextMatchIndex> index) {
std::unique_ptr<milvus::proto::indexcgo::LoadTextIndexInfo> 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<std::string> 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<milvus::index::TextMatchIndex>>
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<DataArray>

View File

@ -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::TextMatchIndex> index) override;
LoadTextIndex(std::unique_ptr<milvus::proto::indexcgo::LoadTextIndexInfo>
info_proto) override;
void
LoadJsonStats(FieldId field_id,

View File

@ -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<std::unique_ptr<milvus::index::TextMatchIndex>>(
&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

View File

@ -439,17 +439,46 @@ SegmentInternalInterface::GetSkipIndex() const {
return skip_index_;
}
index::TextMatchIndex*
PinWrapper<index::TextMatchIndex*>
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<index::TextMatchIndex*> {
using Alt = std::decay_t<decltype(alt)>;
if constexpr (std::is_same_v<
Alt,
std::unique_ptr<milvus::index::TextMatchIndex>>) {
return PinWrapper<index::TextMatchIndex*>(alt.get());
} else if constexpr (std::is_same_v<
Alt,
std::shared_ptr<
milvus::index::TextMatchIndexHolder>>) {
return PinWrapper<index::TextMatchIndex*>(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<index::TextMatchIndex*>(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<DataArray>

View File

@ -142,7 +142,7 @@ class SegmentInterface {
virtual void
CreateTextIndex(FieldId field_id) = 0;
virtual index::TextMatchIndex*
virtual PinWrapper<index::TextMatchIndex*>
GetTextIndex(milvus::OpContext* op_ctx, FieldId field_id) const = 0;
virtual std::vector<PinWrapper<const index::IndexBase*>>
@ -361,7 +361,7 @@ class SegmentInternalInterface : public SegmentInterface {
virtual DataType
GetFieldDataType(FieldId fieldId) const = 0;
index::TextMatchIndex*
PinWrapper<index::TextMatchIndex*>
GetTextIndex(milvus::OpContext* op_ctx, FieldId field_id) const override;
virtual PinWrapper<index::NgramInvertedIndex*>
@ -595,7 +595,12 @@ class SegmentInternalInterface : public SegmentInterface {
SkipIndex skip_index_;
// text-indexes used to do match.
std::unordered_map<FieldId, std::unique_ptr<index::TextMatchIndex>>
std::unordered_map<
FieldId,
std::variant<std::unique_ptr<milvus::index::TextMatchIndex>,
std::shared_ptr<milvus::index::TextMatchIndexHolder>,
std::shared_ptr<milvus::cachinglayer::CacheSlot<
milvus::index::TextMatchIndex>>>>
text_indexes_;
// json stats cache (field_id -> CacheSlot of JsonKeyStats)

View File

@ -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::TextMatchIndex> index) = 0;
LoadTextIndex(std::unique_ptr<milvus::proto::indexcgo::LoadTextIndexInfo>
info_proto) = 0;
virtual InsertRecord<true>&
get_insert_record() = 0;

View File

@ -465,39 +465,7 @@ LoadTextIndex(CSegmentInterface c_segment,
std::make_unique<milvus::proto::indexcgo::LoadTextIndexInfo>();
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<std::string> 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<milvus::index::TextMatchIndex>(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);

View File

@ -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 <utility>
#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<milvus::cachinglayer::ResourceUsage,
milvus::cachinglayer::ResourceUsage>
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<milvus::cachinglayer::cid_t>&) 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<std::pair<milvus::cachinglayer::cid_t,
std::unique_ptr<milvus::index::TextMatchIndex>>>
TextMatchIndexTranslator::get_cells(
const std::vector<milvus::cachinglayer::cid_t>&) {
auto index =
std::make_unique<milvus::index::TextMatchIndex>(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<std::pair<milvus::cachinglayer::cid_t,
std::unique_ptr<milvus::index::TextMatchIndex>>>
result;
result.emplace_back(std::make_pair(0, std::move(index)));
return result;
}
milvus::cachinglayer::Meta*
TextMatchIndexTranslator::meta() {
return &meta_;
}
} // namespace milvus::segcore::storagev1translator

View File

@ -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 <memory>
#include <string>
#include <vector>
#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<milvus::index::TextMatchIndex> {
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<milvus::cachinglayer::ResourceUsage,
milvus::cachinglayer::ResourceUsage>
estimated_byte_size_of_cell(milvus::cachinglayer::cid_t cid) const override;
int64_t
cells_storage_bytes(
const std::vector<milvus::cachinglayer::cid_t>&) const override;
const std::string&
key() const override;
std::vector<std::pair<milvus::cachinglayer::cid_t,
std::unique_ptr<milvus::index::TextMatchIndex>>>
get_cells(const std::vector<milvus::cachinglayer::cid_t>& 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

View File

@ -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)

View File

@ -118,6 +118,7 @@ message LoadTextIndexInfo {
int64 partitionID = 7;
common.LoadPriority load_priority = 8;
bool enable_mmap = 9;
int64 index_size = 10;
}
message LoadJsonKeyIndexInfo {

View File

@ -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 (