mirror of
https://gitee.com/milvus-io/milvus.git
synced 2026-01-07 19:31:51 +08:00
enhance: add cachinglayer management for TextMatchIndex (#44741)
issue: #41435, #44502 Signed-off-by: Shawn Wang <shawn.wang@zilliz.com>
This commit is contained in:
parent
6d5b41644b
commit
c8a4d6e2ef
@ -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));
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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>
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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>
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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
|
||||
@ -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
|
||||
@ -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)
|
||||
|
||||
@ -118,6 +118,7 @@ message LoadTextIndexInfo {
|
||||
int64 partitionID = 7;
|
||||
common.LoadPriority load_priority = 8;
|
||||
bool enable_mmap = 9;
|
||||
int64 index_size = 10;
|
||||
}
|
||||
|
||||
message LoadJsonKeyIndexInfo {
|
||||
|
||||
@ -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 (
|
||||
|
||||
Loading…
x
Reference in New Issue
Block a user