feat: supporing load priority for loading (#42413)

related: #40781

Signed-off-by: MrPresent-Han <chun.han@gmail.com>
Co-authored-by: MrPresent-Han <chun.han@gmail.com>
This commit is contained in:
Chun Han 2025-06-17 15:22:38 +08:00 committed by GitHub
parent 9b97131627
commit 001619aef9
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
75 changed files with 2745 additions and 2354 deletions

View File

@ -39,30 +39,4 @@ AddClusteringResultFiles(milvus::storage::ChunkManager* remote_chunk_manager,
map[remote_prefix] = data_size;
}
void
RemoveClusteringResultFiles(
milvus::storage::ChunkManager* remote_chunk_manager,
const std::unordered_map<std::string, int64_t>& map) {
auto& pool = ThreadPools::GetThreadPool(milvus::ThreadPoolPriority::MIDDLE);
std::vector<std::future<void>> futures;
for (auto& [file_path, file_size] : map) {
futures.push_back(pool.Submit(
[&, path = file_path]() { remote_chunk_manager->Remove(path); }));
}
std::exception_ptr first_exception = nullptr;
for (auto& future : futures) {
try {
future.get();
} catch (...) {
if (!first_exception) {
first_exception = std::current_exception();
}
}
}
if (first_exception) {
std::rethrow_exception(first_exception);
}
}
} // namespace milvus::clustering

View File

@ -38,6 +38,8 @@ struct LoadFieldDataInfo {
// mmap file path will be {mmap_dir_path}/{segment_id}/{field_id}
std::string mmap_dir_path = "";
int64_t storage_version = 0;
milvus::proto::common::LoadPriority load_priority =
milvus::proto::common::LoadPriority::HIGH;
};
struct LoadDeletedRecordInfo {

View File

@ -566,12 +566,13 @@ BitmapIndex<T>::LoadWithoutAssemble(const BinarySet& binary_set,
template <typename T>
void
BitmapIndex<T>::Load(milvus::tracer::TraceContext ctx, const Config& config) {
LOG_DEBUG("load bitmap index with config {}", config.dump());
LOG_INFO("load bitmap index with config {}", config.dump());
auto index_files =
GetValueFromConfig<std::vector<std::string>>(config, "index_files");
AssertInfo(index_files.has_value(),
"index file paths is empty when load bitmap index");
auto index_datas = file_manager_->LoadIndexToMemory(index_files.value());
auto index_datas = file_manager_->LoadIndexToMemory(
index_files.value(), config[milvus::LOAD_PRIORITY]);
BinarySet binary_set;
AssembleIndexDatas(index_datas, binary_set);
LoadWithoutAssemble(binary_set, config);

View File

@ -357,7 +357,8 @@ HybridScalarIndex<T>::Load(milvus::tracer::TraceContext ctx,
auto index_type_file = GetRemoteIndexTypeFile(index_files.value());
auto index_datas = mem_file_manager_->LoadIndexToMemory(
std::vector<std::string>{index_type_file});
std::vector<std::string>{index_type_file},
config[milvus::LOAD_PRIORITY]);
BinarySet binary_set;
AssembleIndexDatas(index_datas, binary_set);
DeserializeIndexType(binary_set);

View File

@ -229,8 +229,8 @@ InvertedIndexTantivy<T>::Load(milvus::tracer::TraceContext ctx,
}
}
auto index_datas =
mem_file_manager_->LoadIndexToMemory(null_offset_files);
auto index_datas = mem_file_manager_->LoadIndexToMemory(
null_offset_files, config[milvus::LOAD_PRIORITY]);
auto null_offsets_data = CompactIndexDatas(index_datas);
auto null_offsets_data_codecs =
@ -243,7 +243,8 @@ InvertedIndexTantivy<T>::Load(milvus::tracer::TraceContext ctx,
it != inverted_index_files.end()) {
// null offset file is not sliced
null_offset_files.push_back(*it);
auto index_datas = mem_file_manager_->LoadIndexToMemory({*it});
auto index_datas = mem_file_manager_->LoadIndexToMemory(
{*it}, config[milvus::LOAD_PRIORITY]);
auto null_offset_data =
std::move(index_datas.at(INDEX_NULL_OFFSET_FILE_NAME));
fill_null_offsets(null_offset_data->PayloadData(),
@ -259,7 +260,8 @@ InvertedIndexTantivy<T>::Load(milvus::tracer::TraceContext ctx,
file) != null_offset_files.end();
}),
inverted_index_files.end());
disk_file_manager_->CacheIndexToDisk(inverted_index_files);
disk_file_manager_->CacheIndexToDisk(inverted_index_files,
config[milvus::LOAD_PRIORITY]);
path_ = prefix;
wrapper_ = std::make_shared<TantivyIndexWrapper>(prefix.c_str(),
milvus::index::SetBitset);

View File

@ -399,7 +399,8 @@ JsonKeyStatsInvertedIndex::Load(milvus::tracer::TraceContext ctx,
index_file = remote_prefix + "/" + index_file;
}
}
disk_file_manager_->CacheJsonKeyIndexToDisk(index_files.value());
disk_file_manager_->CacheJsonKeyIndexToDisk(index_files.value(),
config[milvus::LOAD_PRIORITY]);
AssertInfo(
tantivy_index_exist(path_.c_str()), "index not exist: {}", path_);
wrapper_ = std::make_shared<TantivyIndexWrapper>(path_.c_str(),

View File

@ -206,7 +206,8 @@ ScalarIndexSort<T>::Load(milvus::tracer::TraceContext ctx,
GetValueFromConfig<std::vector<std::string>>(config, "index_files");
AssertInfo(index_files.has_value(),
"index file paths is empty when load disk ann index");
auto index_datas = file_manager_->LoadIndexToMemory(index_files.value());
auto index_datas = file_manager_->LoadIndexToMemory(
index_files.value(), config[milvus::LOAD_PRIORITY]);
BinarySet binary_set;
AssembleIndexDatas(index_datas, binary_set);
LoadWithoutAssemble(binary_set, config);

View File

@ -228,7 +228,8 @@ StringIndexMarisa::Load(milvus::tracer::TraceContext ctx,
GetValueFromConfig<std::vector<std::string>>(config, "index_files");
AssertInfo(index_files.has_value(),
"index file paths is empty when load index");
auto index_datas = file_manager_->LoadIndexToMemory(index_files.value());
auto index_datas = file_manager_->LoadIndexToMemory(
index_files.value(), config[milvus::LOAD_PRIORITY]);
BinarySet binary_set;
AssembleIndexDatas(index_datas, binary_set);
LoadWithoutAssemble(binary_set, config);

View File

@ -16,6 +16,7 @@
#include "index/TextMatchIndex.h"
#include "index/InvertedIndexUtil.h"
#include "index/Utils.h"
#include "storage/ThreadPools.h"
namespace milvus::index {
constexpr const char* TMP_TEXT_LOG_PREFIX = "/tmp/milvus/text-log/";
@ -149,7 +150,8 @@ TextMatchIndex::Load(const Config& config) {
std::vector<std::string> file;
file.push_back(*it);
files_value.erase(it);
auto index_datas = mem_file_manager_->LoadIndexToMemory(file);
auto index_datas = mem_file_manager_->LoadIndexToMemory(
file, config[milvus::LOAD_PRIORITY]);
BinarySet binary_set;
AssembleIndexDatas(index_datas, binary_set);
auto index_valid_data = binary_set.GetByName("index_null_offset");
@ -159,7 +161,8 @@ TextMatchIndex::Load(const Config& config) {
index_valid_data->data.get(),
(size_t)index_valid_data->size);
}
disk_file_manager_->CacheTextLogToDisk(files_value);
disk_file_manager_->CacheTextLogToDisk(files_value,
config[milvus::LOAD_PRIORITY]);
AssertInfo(
tantivy_index_exist(prefix.c_str()), "index not exist: {}", prefix);
wrapper_ = std::make_shared<TantivyIndexWrapper>(prefix.c_str(),

View File

@ -96,7 +96,8 @@ VectorDiskAnnIndex<T>::Load(milvus::tracer::TraceContext ctx,
GetValueFromConfig<std::vector<std::string>>(config, "index_files");
AssertInfo(index_files.has_value(),
"index file paths is empty when load disk ann index data");
file_manager_->CacheIndexToDisk(index_files.value());
file_manager_->CacheIndexToDisk(index_files.value(),
config[milvus::LOAD_PRIORITY]);
read_file_span->End();
}

View File

@ -206,8 +206,8 @@ VectorMemIndex<T>::Load(milvus::tracer::TraceContext ctx,
std::string index_file_prefix = slice_meta_filepath.substr(
0, slice_meta_filepath.find_last_of('/') + 1);
auto result =
file_manager_->LoadIndexToMemory({slice_meta_filepath});
auto result = file_manager_->LoadIndexToMemory(
{slice_meta_filepath}, config[milvus::LOAD_PRIORITY]);
auto raw_slice_meta = std::move(result[INDEX_FILE_SLICE_META]);
Config meta_data = Config::parse(std::string(
reinterpret_cast<const char*>(raw_slice_meta->PayloadData()),
@ -224,7 +224,8 @@ VectorMemIndex<T>::Load(milvus::tracer::TraceContext ctx,
batch.push_back(index_file_prefix + file_name);
}
auto batch_data = file_manager_->LoadIndexToMemory(batch);
auto batch_data = file_manager_->LoadIndexToMemory(
batch, config[milvus::LOAD_PRIORITY]);
int64_t payload_size = 0;
index_data_codecs.insert({prefix, IndexDataCodec{}});
auto& index_data_codec = index_data_codecs.at(prefix);
@ -249,9 +250,10 @@ VectorMemIndex<T>::Load(milvus::tracer::TraceContext ctx,
}
if (!pending_index_files.empty()) {
auto result =
file_manager_->LoadIndexToMemory(std::vector<std::string>(
pending_index_files.begin(), pending_index_files.end()));
auto result = file_manager_->LoadIndexToMemory(
std::vector<std::string>(pending_index_files.begin(),
pending_index_files.end()),
config[milvus::LOAD_PRIORITY]);
for (auto&& index_data : result) {
auto prefix = index_data.first;
index_data_codecs.insert({prefix, IndexDataCodec{}});
@ -555,7 +557,8 @@ void VectorMemIndex<T>::LoadFromFile(const Config& config) {
std::vector<std::string> batch{};
batch.reserve(parallel_degree);
auto result = file_manager_->LoadIndexToMemory({slice_meta_filepath});
auto result = file_manager_->LoadIndexToMemory(
{slice_meta_filepath}, config[milvus::LOAD_PRIORITY]);
auto raw_slice_meta = std::move(result[INDEX_FILE_SLICE_META]);
Config meta_data = Config::parse(std::string(
reinterpret_cast<const char*>(raw_slice_meta->PayloadData()),
@ -567,7 +570,8 @@ void VectorMemIndex<T>::LoadFromFile(const Config& config) {
auto total_len = static_cast<size_t>(item[TOTAL_LEN]);
auto HandleBatch = [&](int index) {
auto start_load2_mem = std::chrono::system_clock::now();
auto batch_data = file_manager_->LoadIndexToMemory(batch);
auto batch_data = file_manager_->LoadIndexToMemory(
batch, config[milvus::LOAD_PRIORITY]);
load_duration_sum +=
(std::chrono::system_clock::now() - start_load2_mem);
for (int j = index - batch.size() + 1; j <= index; j++) {
@ -606,8 +610,10 @@ void VectorMemIndex<T>::LoadFromFile(const Config& config) {
} else {
//1. load files into memory
auto start_load_files2_mem = std::chrono::system_clock::now();
auto result = file_manager_->LoadIndexToMemory(std::vector<std::string>(
pending_index_files.begin(), pending_index_files.end()));
auto result = file_manager_->LoadIndexToMemory(
std::vector<std::string>(pending_index_files.begin(),
pending_index_files.end()),
config[milvus::LOAD_PRIORITY]);
load_duration_sum +=
(std::chrono::system_clock::now() - start_load_files2_mem);
//2. write data into files

View File

@ -306,7 +306,8 @@ ChunkedSegmentSealedImpl::load_column_group_data_internal(
insert_files,
info.enable_mmap,
row_group_meta_list,
field_id_list);
field_id_list,
load_info.load_priority);
auto chunked_column_group =
std::make_shared<ChunkedColumnGroup>(std::move(translator));
@ -358,7 +359,8 @@ ChunkedSegmentSealedImpl::load_field_data_internal(
ThreadPools::GetThreadPool(milvus::ThreadPoolPriority::MIDDLE);
pool.Submit(LoadArrowReaderFromRemote,
insert_files,
field_data_info.arrow_reader_channel);
field_data_info.arrow_reader_channel,
load_info.load_priority);
LOG_INFO("segment {} submits load field {} task to thread pool",
this->get_segment_id(),
@ -383,7 +385,8 @@ ChunkedSegmentSealedImpl::load_field_data_internal(
field_meta,
field_data_info,
std::move(insert_files_with_entries_nums),
info.enable_mmap);
info.enable_mmap,
load_info.load_priority);
auto data_type = field_meta.get_data_type();
auto column = MakeChunkedColumnBase(

View File

@ -314,8 +314,10 @@ SegmentGrowingImpl::load_field_data_internal(const LoadFieldDataInfo& infos) {
this->get_segment_id(),
field_id.get(),
num_rows);
auto load_future =
pool.Submit(LoadFieldDatasFromRemote, insert_files, channel);
auto load_future = pool.Submit(LoadFieldDatasFromRemote,
insert_files,
channel,
infos.load_priority);
LOG_INFO("segment {} submits load field {} task to thread pool",
this->get_segment_id(),
@ -468,7 +470,9 @@ SegmentGrowingImpl::load_column_group_data_internal(
column_group_info.arrow_reader_channel,
DEFAULT_FIELD_MAX_MEMORY_LIMIT,
std::move(strategy),
row_group_lists);
row_group_lists,
nullptr,
infos.load_priority);
});
LOG_INFO("segment {} submits load fields {} task to thread pool",

View File

@ -944,11 +944,13 @@ ReverseDataFromIndex(const index::IndexBase* index,
// segcore use default remote chunk manager to load data from minio/s3
void
LoadArrowReaderFromRemote(const std::vector<std::string>& remote_files,
std::shared_ptr<ArrowReaderChannel> channel) {
std::shared_ptr<ArrowReaderChannel> channel,
milvus::proto::common::LoadPriority priority) {
try {
auto rcm = storage::RemoteChunkManagerSingleton::GetInstance()
.GetRemoteChunkManager();
auto& pool = ThreadPools::GetThreadPool(ThreadPoolPriority::HIGH);
auto& pool =
ThreadPools::GetThreadPool(milvus::PriorityForLoad(priority));
std::vector<std::future<std::shared_ptr<milvus::ArrowDataWrapper>>>
futures;
@ -980,11 +982,13 @@ LoadArrowReaderFromRemote(const std::vector<std::string>& remote_files,
void
LoadFieldDatasFromRemote(const std::vector<std::string>& remote_files,
FieldDataChannelPtr channel) {
FieldDataChannelPtr channel,
milvus::proto::common::LoadPriority priority) {
try {
auto rcm = storage::RemoteChunkManagerSingleton::GetInstance()
.GetRemoteChunkManager();
auto& pool = ThreadPools::GetThreadPool(ThreadPoolPriority::HIGH);
auto& pool =
ThreadPools::GetThreadPool(milvus::PriorityForLoad(priority));
std::vector<std::future<FieldDataPtr>> futures;
futures.reserve(remote_files.size());

View File

@ -109,11 +109,13 @@ ReverseDataFromIndex(const index::IndexBase* index,
void
LoadArrowReaderFromRemote(const std::vector<std::string>& remote_files,
std::shared_ptr<ArrowReaderChannel> channel);
std::shared_ptr<ArrowReaderChannel> channel,
milvus::proto::common::LoadPriority priority);
void
LoadFieldDatasFromRemote(const std::vector<std::string>& remote_files,
FieldDataChannelPtr channel);
FieldDataChannelPtr channel,
milvus::proto::common::LoadPriority priority);
/**
* Returns an index pointing to the first element in the range [first, last) such that `value < element` is true
* (i.e. that is strictly greater than value), or last if no such element is found.

View File

@ -105,3 +105,9 @@ EnableMmap(CLoadFieldDataInfo c_load_field_data_info,
auto info = static_cast<LoadFieldDataInfo*>(c_load_field_data_info);
info->field_infos[field_id].enable_mmap = enabled;
}
void
SetLoadPriority(CLoadFieldDataInfo c_load_field_data_info, int32_t priority) {
auto info = static_cast<LoadFieldDataInfo*>(c_load_field_data_info);
info->load_priority = milvus::proto::common::LoadPriority(priority);
}

View File

@ -56,6 +56,9 @@ EnableMmap(CLoadFieldDataInfo c_load_field_data_info,
int64_t field_id,
bool enabled);
void
SetLoadPriority(CLoadFieldDataInfo c_load_field_data_info, int32_t priority);
#ifdef __cplusplus
}
#endif

View File

@ -303,6 +303,10 @@ AppendIndexV2(CTraceContext c_trace, CLoadIndexInfo c_load_index_info) {
auto config = milvus::index::ParseConfigFromIndexParams(
load_index_info->index_params);
auto load_priority_str =
config[milvus::LOAD_PRIORITY].get<std::string>();
auto priority_for_load = milvus::PriorityForLoad(load_priority_str);
config[milvus::LOAD_PRIORITY] = priority_for_load;
// Config should have value for milvus::index::SCALAR_INDEX_ENGINE_VERSION for production calling chain.
// Use value_or(1) for unit test without setting this value
@ -322,12 +326,14 @@ AppendIndexV2(CTraceContext c_trace, CLoadIndexInfo c_load_index_info) {
milvus::tracer::SetRootSpan(span);
LOG_INFO(
"[collection={}][segment={}][field={}][enable_mmap={}] load index "
"[collection={}][segment={}][field={}][enable_mmap={}][load_"
"priority={}] load index "
"{}",
load_index_info->collection_id,
load_index_info->segment_id,
load_index_info->field_id,
load_index_info->enable_mmap,
load_priority_str,
load_index_info->index_id);
// get index type

View File

@ -160,14 +160,16 @@ LoadWithStrategy(const std::vector<std::string>& remote_files,
int64_t memory_limit,
std::unique_ptr<RowGroupSplitStrategy> strategy,
const std::vector<std::vector<int64_t>>& row_group_lists,
const std::shared_ptr<arrow::Schema> schema) {
const std::shared_ptr<arrow::Schema> schema,
milvus::proto::common::LoadPriority priority) {
try {
AssertInfo(
remote_files.size() == row_group_lists.size(),
"Number of remote files must match number of row group lists");
auto fs = milvus_storage::ArrowFileSystemSingleton::GetInstance()
.GetArrowFileSystem();
auto& pool = ThreadPools::GetThreadPool(ThreadPoolPriority::HIGH);
auto& pool =
ThreadPools::GetThreadPool(milvus::PriorityForLoad(priority));
for (size_t file_idx = 0; file_idx < remote_files.size(); ++file_idx) {
const auto& file = remote_files[file_idx];

View File

@ -84,6 +84,8 @@ LoadWithStrategy(const std::vector<std::string>& remote_files,
int64_t memory_limit,
std::unique_ptr<RowGroupSplitStrategy> strategy,
const std::vector<std::vector<int64_t>>& row_group_lists,
const std::shared_ptr<arrow::Schema> schema = nullptr);
const std::shared_ptr<arrow::Schema> schema = nullptr,
milvus::proto::common::LoadPriority priority =
milvus::proto::common::LoadPriority::HIGH);
} // namespace milvus::segcore

View File

@ -435,9 +435,11 @@ LoadTextIndex(CSegmentInterface c_segment,
files.push_back(f);
}
config["index_files"] = files;
config[milvus::LOAD_PRIORITY] = info_proto->load_priority();
milvus::storage::FileManagerContext ctx(
field_meta, index_meta, remote_chunk_manager);
field_meta,
index_meta,
remote_chunk_manager);
auto index = std::make_unique<milvus::index::TextMatchIndex>(ctx);
index->Load(config);
@ -488,7 +490,7 @@ LoadJsonKeyIndex(CTraceContext c_trace,
files.push_back(f);
}
config["index_files"] = files;
config[milvus::LOAD_PRIORITY] = info_proto->load_priority();
milvus::storage::FileManagerContext file_ctx(
field_meta, index_meta, remote_chunk_manager);

View File

@ -32,7 +32,8 @@ ChunkTranslator::ChunkTranslator(
FieldMeta field_meta,
FieldDataInfo field_data_info,
std::vector<std::pair<std::string, int64_t>>&& files_and_rows,
bool use_mmap)
bool use_mmap,
milvus::proto::common::LoadPriority load_priority)
: segment_id_(segment_id),
field_id_(field_data_info.field_id),
field_meta_(field_meta),
@ -46,7 +47,8 @@ ChunkTranslator::ChunkTranslator(
IsVectorDataType(field_meta.get_data_type()),
/* is_index */ false,
/* in_load_list*/ field_data_info.in_load_list),
/* support_eviction */ false) {
/* support_eviction */ false),
load_priority_(load_priority) {
AssertInfo(!SystemProperty::Instance().IsSystem(FieldId(field_id_)),
"ChunkTranslator not supported for system field");
meta_.num_rows_until_chunk_.push_back(0);
@ -104,7 +106,8 @@ ChunkTranslator::get_cells(
segment_id_,
field_id_,
fmt::format("{}", fmt::join(cids, " ")));
pool.Submit(LoadArrowReaderFromRemote, remote_files, channel);
pool.Submit(
LoadArrowReaderFromRemote, remote_files, channel, load_priority_);
auto data_type = field_meta_.get_data_type();

View File

@ -40,7 +40,8 @@ class ChunkTranslator : public milvus::cachinglayer::Translator<milvus::Chunk> {
FieldMeta field_meta,
FieldDataInfo field_data_info,
std::vector<std::pair<std::string, int64_t>>&& files_and_rows,
bool use_mmap);
bool use_mmap,
milvus::proto::common::LoadPriority load_priority);
size_t
num_cells() const override;
@ -68,6 +69,8 @@ class ChunkTranslator : public milvus::cachinglayer::Translator<milvus::Chunk> {
CTMeta meta_;
FieldMeta field_meta_;
std::string mmap_dir_path_;
milvus::proto::common::LoadPriority load_priority_{
milvus::proto::common::LoadPriority::HIGH};
};
} // namespace milvus::segcore::storagev1translator

View File

@ -86,7 +86,7 @@ SealedIndexTranslator::get_cells(const std::vector<cid_t>& cids) {
config_[milvus::index::MMAP_FILE_PATH] = filepath.string();
}
LOG_DEBUG("load index with configs: {}", config_.dump());
LOG_INFO("load index with configs: {}", config_.dump());
index->Load(ctx_, config_);
std::vector<std::pair<cid_t, std::unique_ptr<milvus::index::IndexBase>>>

View File

@ -42,7 +42,8 @@ GroupChunkTranslator::GroupChunkTranslator(
std::vector<std::string> insert_files,
bool use_mmap,
std::vector<milvus_storage::RowGroupMetadataVector>& row_group_meta_list,
milvus_storage::FieldIDList field_id_list)
milvus_storage::FieldIDList field_id_list,
milvus::proto::common::LoadPriority load_priority)
: segment_id_(segment_id),
key_(fmt::format("seg_{}_cg_{}", segment_id, column_group_info.field_id)),
field_metas_(field_metas),
@ -51,6 +52,7 @@ GroupChunkTranslator::GroupChunkTranslator(
use_mmap_(use_mmap),
row_group_meta_list_(row_group_meta_list),
field_id_list_(field_id_list),
load_priority_(load_priority),
meta_(
field_id_list.size(),
use_mmap ? milvus::cachinglayer::StorageType::DISK
@ -156,7 +158,9 @@ GroupChunkTranslator::get_cells(const std::vector<cachinglayer::cid_t>& cids) {
column_group_info_.arrow_reader_channel,
DEFAULT_FIELD_MAX_MEMORY_LIMIT,
std::move(strategy),
row_group_lists);
row_group_lists,
nullptr,
load_priority_);
});
LOG_INFO("segment {} submits load fields {} task to thread pool",
segment_id_,

View File

@ -43,7 +43,8 @@ class GroupChunkTranslator
bool use_mmap,
std::vector<milvus_storage::RowGroupMetadataVector>&
row_group_meta_list,
milvus_storage::FieldIDList field_id_list);
milvus_storage::FieldIDList field_id_list,
milvus::proto::common::LoadPriority load_priority);
~GroupChunkTranslator() override;
@ -90,6 +91,8 @@ class GroupChunkTranslator
GroupCTMeta meta_;
int64_t timestamp_offet_;
bool use_mmap_;
milvus::proto::common::LoadPriority load_priority_{
milvus::proto::common::LoadPriority::HIGH};
};
} // namespace milvus::segcore::storagev2translator

View File

@ -226,7 +226,8 @@ DiskFileManagerImpl::AddBatchIndexFiles(
void
DiskFileManagerImpl::CacheIndexToDiskInternal(
const std::vector<std::string>& remote_files,
const std::function<std::string()>& get_local_index_prefix) {
const std::function<std::string()>& get_local_index_prefix,
milvus::proto::common::LoadPriority priority) {
auto local_chunk_manager =
LocalChunkManagerSingleton::GetInstance().GetChunkManager();
@ -267,7 +268,9 @@ DiskFileManagerImpl::CacheIndexToDiskInternal(
auto appendIndexFiles = [&]() {
auto index_chunks_futures =
GetObjectData(rcm_.get(), batch_remote_files);
GetObjectData(rcm_.get(),
batch_remote_files,
milvus::PriorityForLoad(priority));
for (auto& chunk_future : index_chunks_futures) {
auto chunk_codec = chunk_future.get();
file.Write(chunk_codec->PayloadData(),
@ -293,23 +296,30 @@ DiskFileManagerImpl::CacheIndexToDiskInternal(
void
DiskFileManagerImpl::CacheIndexToDisk(
const std::vector<std::string>& remote_files) {
const std::vector<std::string>& remote_files,
milvus::proto::common::LoadPriority priority) {
return CacheIndexToDiskInternal(
remote_files, [this]() { return GetLocalIndexObjectPrefix(); });
remote_files,
[this]() { return GetLocalIndexObjectPrefix(); },
priority);
}
void
DiskFileManagerImpl::CacheTextLogToDisk(
const std::vector<std::string>& remote_files) {
const std::vector<std::string>& remote_files,
milvus::proto::common::LoadPriority priority) {
return CacheIndexToDiskInternal(
remote_files, [this]() { return GetLocalTextIndexPrefix(); });
remote_files, [this]() { return GetLocalTextIndexPrefix(); }, priority);
}
void
DiskFileManagerImpl::CacheJsonKeyIndexToDisk(
const std::vector<std::string>& remote_files) {
const std::vector<std::string>& remote_files,
milvus::proto::common::LoadPriority priority) {
return CacheIndexToDiskInternal(
remote_files, [this]() { return GetLocalJsonKeyIndexPrefix(); });
remote_files,
[this]() { return GetLocalJsonKeyIndexPrefix(); },
priority);
}
template <typename DataType>

View File

@ -103,13 +103,16 @@ class DiskFileManagerImpl : public FileManagerImpl {
}
void
CacheIndexToDisk(const std::vector<std::string>& remote_files);
CacheIndexToDisk(const std::vector<std::string>& remote_files,
milvus::proto::common::LoadPriority priority);
void
CacheTextLogToDisk(const std::vector<std::string>& remote_files);
CacheTextLogToDisk(const std::vector<std::string>& remote_files,
milvus::proto::common::LoadPriority priority);
void
CacheJsonKeyIndexToDisk(const std::vector<std::string>& remote_files);
CacheJsonKeyIndexToDisk(const std::vector<std::string>& remote_files,
milvus::proto::common::LoadPriority priority);
void
AddBatchIndexFiles(const std::string& local_file_name,
@ -160,7 +163,9 @@ class DiskFileManagerImpl : public FileManagerImpl {
void
CacheIndexToDiskInternal(
const std::vector<std::string>& remote_files,
const std::function<std::string()>& get_local_index_prefix);
const std::function<std::string()>& get_local_index_prefix,
milvus::proto::common::LoadPriority priority =
milvus::proto::common::LoadPriority::HIGH);
template <typename DataType>
std::string

View File

@ -100,14 +100,16 @@ MemFileManagerImpl::LoadFile(const std::string& filename) noexcept {
std::map<std::string, std::unique_ptr<DataCodec>>
MemFileManagerImpl::LoadIndexToMemory(
const std::vector<std::string>& remote_files) {
const std::vector<std::string>& remote_files,
milvus::proto::common::LoadPriority priority) {
std::map<std::string, std::unique_ptr<DataCodec>> file_to_index_data;
auto parallel_degree =
static_cast<uint64_t>(DEFAULT_FIELD_MAX_MEMORY_LIMIT / FILE_SLICE_SIZE);
std::vector<std::string> batch_files;
auto LoadBatchIndexFiles = [&]() {
auto index_datas = GetObjectData(rcm_.get(), batch_files);
auto index_datas = GetObjectData(
rcm_.get(), batch_files, milvus::PriorityForLoad(priority));
for (size_t idx = 0; idx < batch_files.size(); ++idx) {
auto file_name =
batch_files[idx].substr(batch_files[idx].find_last_of('/') + 1);

View File

@ -52,7 +52,8 @@ class MemFileManagerImpl : public FileManagerImpl {
}
std::map<std::string, std::unique_ptr<DataCodec>>
LoadIndexToMemory(const std::vector<std::string>& remote_files);
LoadIndexToMemory(const std::vector<std::string>& remote_files,
milvus::proto::common::LoadPriority priority);
std::vector<FieldDataPtr>
CacheRawDataToMemory(const Config& config);

View File

@ -22,12 +22,29 @@
namespace milvus {
constexpr const char* LOAD_PRIORITY = "load_priority";
enum ThreadPoolPriority {
HIGH = 0,
MIDDLE = 1,
LOW = 2,
};
inline ThreadPoolPriority
PriorityForLoad(milvus::proto::common::LoadPriority priority) {
return priority == milvus::proto::common::LoadPriority::HIGH
? ThreadPoolPriority::HIGH
: ThreadPoolPriority::LOW;
}
inline milvus::proto::common::LoadPriority
PriorityForLoad(const std::string& priority_str) {
if (priority_str == "LOW") {
return milvus::proto::common::LoadPriority::LOW;
}
return milvus::proto::common::LoadPriority::HIGH;
}
class ThreadPools {
public:
static ThreadPool&

View File

@ -715,8 +715,9 @@ EncodeAndUploadIndexSlice(ChunkManager* chunk_manager,
std::vector<std::future<std::unique_ptr<DataCodec>>>
GetObjectData(ChunkManager* remote_chunk_manager,
const std::vector<std::string>& remote_files) {
auto& pool = ThreadPools::GetThreadPool(milvus::ThreadPoolPriority::HIGH);
const std::vector<std::string>& remote_files,
milvus::ThreadPoolPriority priority) {
auto& pool = ThreadPools::GetThreadPool(priority);
std::vector<std::future<std::unique_ptr<DataCodec>>> futures;
futures.reserve(remote_files.size());
for (auto& file : remote_files) {

View File

@ -34,6 +34,7 @@
#include "storage/DataCodec.h"
#include "storage/Types.h"
#include "milvus-storage/filesystem/fs.h"
#include "storage/ThreadPools.h"
namespace milvus::storage {
@ -153,8 +154,10 @@ EncodeAndUploadIndexSlice(ChunkManager* chunk_manager,
std::string object_key);
std::vector<std::future<std::unique_ptr<DataCodec>>>
GetObjectData(ChunkManager* remote_chunk_manager,
const std::vector<std::string>& remote_files);
GetObjectData(
ChunkManager* remote_chunk_manager,
const std::vector<std::string>& remote_files,
milvus::ThreadPoolPriority priority = milvus::ThreadPoolPriority::HIGH);
std::vector<FieldDataPtr>
GetFieldDatasFromStorageV2(std::vector<std::vector<std::string>>& remote_files,

View File

@ -255,7 +255,8 @@ class ArrayBitmapIndexTest : public testing::Test {
index_info.field_type = DataType::ARRAY;
config["index_files"] = index_files;
config[milvus::LOAD_PRIORITY] =
milvus::proto::common::LoadPriority::HIGH;
ctx.set_for_loading_index(true);
index_ =
index::IndexFactory::GetInstance().CreateIndex(index_info, ctx);

View File

@ -193,6 +193,8 @@ class BitmapIndexTest : public testing::Test {
field_id);
;
}
config[milvus::LOAD_PRIORITY] =
milvus::proto::common::LoadPriority::HIGH;
index_ =
index::IndexFactory::GetInstance().CreateIndex(index_info, ctx);
index_->Load(milvus::tracer::TraceContext{}, config);

View File

@ -94,7 +94,8 @@ TEST_F(DiskAnnFileManagerTest, AddFilePositiveParallel) {
std::cout << file2size.first << std::endl;
remote_files.emplace_back(file2size.first);
}
diskAnnFileManager->CacheIndexToDisk(remote_files);
diskAnnFileManager->CacheIndexToDisk(
remote_files, milvus::proto::common::LoadPriority::HIGH);
auto local_files = diskAnnFileManager->GetLocalFilePaths();
for (auto& file : local_files) {
auto file_size = lcm->Size(file);

View File

@ -115,7 +115,8 @@ TEST_P(GroupChunkTranslatorTest, TestWithMmap) {
paths_,
use_mmap,
row_group_meta_list,
field_id_list);
field_id_list,
milvus::proto::common::LoadPriority::LOW);
// num cells
EXPECT_EQ(translator.num_cells(), row_group_meta_list[0].size());

View File

@ -184,7 +184,8 @@ class HybridIndexTestV1 : public testing::Test {
index_info.field_type = type_;
config["index_files"] = index_files;
config[milvus::LOAD_PRIORITY] =
milvus::proto::common::LoadPriority::HIGH;
ctx.set_for_loading_index(true);
index_ =
index::IndexFactory::GetInstance().CreateIndex(index_info, ctx);

View File

@ -499,6 +499,8 @@ TEST_P(IndexTest, BuildAndQuery) {
ASSERT_GT(serializedSize, 0);
load_conf = generate_load_conf(index_type, metric_type, 0);
load_conf["index_files"] = index_files;
load_conf[milvus::LOAD_PRIORITY] =
milvus::proto::common::LoadPriority::HIGH;
ASSERT_NO_THROW(vec_index->Load(milvus::tracer::TraceContext{}, load_conf));
EXPECT_EQ(vec_index->Count(), NB);
if (!is_sparse) {
@ -569,6 +571,8 @@ TEST_P(IndexTest, Mmap) {
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;
load_conf[milvus::LOAD_PRIORITY] =
milvus::proto::common::LoadPriority::HIGH;
vec_index->Load(milvus::tracer::TraceContext{}, load_conf);
EXPECT_EQ(vec_index->Count(), NB);
EXPECT_EQ(vec_index->GetDim(), is_sparse ? kTestSparseDim : DIM);
@ -624,6 +628,8 @@ TEST_P(IndexTest, GetVector) {
load_conf["index_files"] = index_files;
vec_index = dynamic_cast<milvus::index::VectorIndex*>(new_index.get());
load_conf[milvus::LOAD_PRIORITY] =
milvus::proto::common::LoadPriority::HIGH;
vec_index->Load(milvus::tracer::TraceContext{}, load_conf);
if (!is_sparse) {
EXPECT_EQ(vec_index->GetDim(), DIM);
@ -724,7 +730,8 @@ TEST_P(IndexTest, GetVector_EmptySparseVector) {
create_index_info, file_manager_context);
load_conf = generate_load_conf(index_type, metric_type, 0);
load_conf["index_files"] = index_files;
load_conf[milvus::LOAD_PRIORITY] =
milvus::proto::common::LoadPriority::HIGH;
vec_index = dynamic_cast<milvus::index::VectorIndex*>(new_index.get());
vec_index->Load(milvus::tracer::TraceContext{}, load_conf);
EXPECT_EQ(vec_index->Count(), NB);
@ -808,6 +815,8 @@ TEST(Indexing, SearchDiskAnnWithInvalidParam) {
auto vec_index = dynamic_cast<milvus::index::VectorIndex*>(new_index.get());
auto load_conf = generate_load_conf(index_type, metric_type, NB);
load_conf["index_files"] = index_files;
load_conf[milvus::LOAD_PRIORITY] =
milvus::proto::common::LoadPriority::HIGH;
vec_index->Load(milvus::tracer::TraceContext{}, load_conf);
EXPECT_EQ(vec_index->Count(), NB);
@ -892,6 +901,8 @@ TEST(Indexing, SearchDiskAnnWithFloat16) {
auto vec_index = dynamic_cast<milvus::index::VectorIndex*>(new_index.get());
auto load_conf = generate_load_conf<float16>(index_type, metric_type, NB);
load_conf["index_files"] = index_files;
load_conf[milvus::LOAD_PRIORITY] =
milvus::proto::common::LoadPriority::HIGH;
vec_index->Load(milvus::tracer::TraceContext{}, load_conf);
EXPECT_EQ(vec_index->Count(), NB);
@ -975,6 +986,8 @@ TEST(Indexing, SearchDiskAnnWithBFloat16) {
auto vec_index = dynamic_cast<milvus::index::VectorIndex*>(new_index.get());
auto load_conf = generate_load_conf<bfloat16>(index_type, metric_type, NB);
load_conf["index_files"] = index_files;
load_conf[milvus::LOAD_PRIORITY] =
milvus::proto::common::LoadPriority::HIGH;
vec_index->Load(milvus::tracer::TraceContext{}, load_conf);
EXPECT_EQ(vec_index->Count(), NB);

View File

@ -191,7 +191,8 @@ test_run() {
Config config;
config["index_files"] = index_files;
config[milvus::LOAD_PRIORITY] =
milvus::proto::common::LoadPriority::HIGH;
ctx.set_for_loading_index(true);
auto index =
index::IndexFactory::GetInstance().CreateIndex(index_info, ctx);
@ -583,7 +584,8 @@ test_string() {
Config config;
config["index_files"] = index_files;
config[milvus::LOAD_PRIORITY] =
milvus::proto::common::LoadPriority::HIGH;
ctx.set_for_loading_index(true);
auto index =
index::IndexFactory::GetInstance().CreateIndex(index_info, ctx);

View File

@ -188,6 +188,8 @@ class JsonFlatIndexTest : public ::testing::Test {
Config load_config;
load_config["index_files"] = index_files_;
load_config[milvus::LOAD_PRIORITY] =
milvus::proto::common::LoadPriority::HIGH;
ctx_->set_for_loading_index(true);
json_index_ = std::make_shared<index::JsonFlatIndex>(*ctx_, "");

View File

@ -129,7 +129,8 @@ class JsonKeyStatsIndexTest : public ::testing::TestWithParam<bool> {
index::CreateIndexInfo index_info{};
config["index_files"] = index_files;
config[milvus::LOAD_PRIORITY] =
milvus::proto::common::LoadPriority::HIGH;
index_ = std::make_shared<JsonKeyStatsInvertedIndex>(ctx, true);
index_->Load(milvus::tracer::TraceContext{}, config);
}

View File

@ -1372,6 +1372,7 @@ GenVecIndexing(int64_t N,
auto create_index_result = indexing->Upload();
auto index_files = create_index_result->GetIndexFiles();
conf["index_files"] = index_files;
conf[milvus::LOAD_PRIORITY] = milvus::proto::common::LoadPriority::HIGH;
// we need a load stage to use index as the producation does
// knowhere would do some data preparation in this stage
indexing->Load(milvus::tracer::TraceContext{}, conf);

View File

@ -378,6 +378,7 @@ func GenTestIndexInfoList(collectionID int64, schema *schemapb.CollectionSchema)
{Key: common.MetricTypeKey, Value: metric.L2},
{Key: common.IndexTypeKey, Value: IndexFaissIVFFlat},
{Key: "nlist", Value: "128"},
{Key: common.LoadPriorityKey, Value: "high"},
}
}
case schemapb.DataType_BinaryVector:
@ -386,6 +387,7 @@ func GenTestIndexInfoList(collectionID int64, schema *schemapb.CollectionSchema)
{Key: common.MetricTypeKey, Value: metric.JACCARD},
{Key: common.IndexTypeKey, Value: IndexFaissBinIVFFlat},
{Key: "nlist", Value: "128"},
{Key: common.LoadPriorityKey, Value: "high"},
}
}
case schemapb.DataType_SparseFloatVector:
@ -394,6 +396,7 @@ func GenTestIndexInfoList(collectionID int64, schema *schemapb.CollectionSchema)
{Key: common.MetricTypeKey, Value: metric.IP},
{Key: common.IndexTypeKey, Value: IndexSparseWand},
{Key: "M", Value: "16"},
{Key: common.LoadPriorityKey, Value: "high"},
}
}
}
@ -824,7 +827,7 @@ func GenAndSaveIndexV2(collectionID, partitionID, segmentID, buildID int64,
func GenAndSaveIndex(collectionID, partitionID, segmentID, fieldID int64, msgLength int, indexType, metricType string, cm storage.ChunkManager) (*querypb.FieldIndexInfo, error) {
typeParams, indexParams := genIndexParams(indexType, metricType)
indexParams[common.LoadPriorityKey] = "HIGH"
index, err := indexcgowrapper.NewCgoIndex(schemapb.DataType_FloatVector, typeParams, indexParams)
if err != nil {
return nil, err

View File

@ -1885,6 +1885,8 @@ func (t *showPartitionsTask) PostExecute(ctx context.Context) error {
return nil
}
const LoadPriorityName = "load_priority"
type loadCollectionTask struct {
baseTask
Condition
@ -1951,6 +1953,15 @@ func (t *loadCollectionTask) PreExecute(ctx context.Context) error {
return nil
}
func (t *loadCollectionTask) GetLoadPriority() commonpb.LoadPriority {
loadPriority := commonpb.LoadPriority_HIGH
loadPriorityStr, ok := t.LoadCollectionRequest.LoadParams[LoadPriorityName]
if ok && loadPriorityStr == "low" {
loadPriority = commonpb.LoadPriority_LOW
}
return loadPriority
}
func (t *loadCollectionTask) Execute(ctx context.Context) (err error) {
collID, err := globalMetaCache.GetCollectionID(ctx, t.GetDbName(), t.CollectionName)
@ -2023,9 +2034,11 @@ func (t *loadCollectionTask) Execute(ctx context.Context) (err error) {
Refresh: t.Refresh,
ResourceGroups: t.ResourceGroups,
LoadFields: loadFields,
Priority: t.GetLoadPriority(),
}
log.Debug("send LoadCollectionRequest to query coordinator",
zap.Any("schema", request.Schema))
log.Info("send LoadCollectionRequest to query coordinator",
zap.Any("schema", request.Schema),
zap.Int32("priority", int32(request.GetPriority())))
t.result, err = t.mixCoord.LoadCollection(ctx, request)
if err = merr.CheckRPCCall(t.result, err); err != nil {
return fmt.Errorf("call query coordinator LoadCollection: %s", err)
@ -2207,6 +2220,15 @@ func (t *loadPartitionsTask) PreExecute(ctx context.Context) error {
return nil
}
func (t *loadPartitionsTask) GetLoadPriority() commonpb.LoadPriority {
loadPriority := commonpb.LoadPriority_HIGH
loadPriorityStr, ok := t.LoadPartitionsRequest.LoadParams[LoadPriorityName]
if ok && loadPriorityStr == "low" {
loadPriority = commonpb.LoadPriority_LOW
}
return loadPriority
}
func (t *loadPartitionsTask) Execute(ctx context.Context) error {
var partitionIDs []int64
collID, err := globalMetaCache.GetCollectionID(ctx, t.GetDbName(), t.CollectionName)
@ -2284,7 +2306,11 @@ func (t *loadPartitionsTask) Execute(ctx context.Context) error {
Refresh: t.Refresh,
ResourceGroups: t.ResourceGroups,
LoadFields: loadFields,
Priority: t.GetLoadPriority(),
}
log.Info("send LoadPartitionRequest to query coordinator",
zap.Any("schema", request.Schema),
zap.Int32("priority", int32(request.GetPriority())))
t.result, err = t.mixCoord.LoadPartitions(ctx, request)
if err = merr.CheckRPCCall(t.result, err); err != nil {
return err

View File

@ -24,6 +24,7 @@ import (
"github.com/blang/semver/v4"
"github.com/samber/lo"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/internal/querycoordv2/meta"
"github.com/milvus-io/milvus/internal/querycoordv2/session"
"github.com/milvus-io/milvus/internal/querycoordv2/task"
@ -39,6 +40,7 @@ type SegmentAssignPlan struct {
FromScore int64
ToScore int64
SegmentScore int64
LoadPriority commonpb.LoadPriority
}
func (segPlan *SegmentAssignPlan) String() string {

View File

@ -23,6 +23,7 @@ import (
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/suite"
"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/metastore/kv/querycoord"
"github.com/milvus-io/milvus/internal/querycoordv2/meta"
@ -883,7 +884,8 @@ func (suite *ChannelLevelScoreBalancerTestSuite) TestExclusiveChannelBalance_Cha
collection.Status = querypb.LoadStatus_Loaded
balancer.meta.CollectionManager.PutCollection(ctx, collection)
balancer.meta.CollectionManager.PutPartition(ctx, utils.CreateTestPartition(collectionID, partitionID))
balancer.meta.ReplicaManager.Spawn(ctx, 1, map[string]int{meta.DefaultResourceGroupName: 1}, []string{"channel1", "channel2"})
balancer.meta.ReplicaManager.Spawn(ctx, 1, map[string]int{meta.DefaultResourceGroupName: 1}, []string{"channel1", "channel2"},
commonpb.LoadPriority_LOW)
balancer.targetMgr.UpdateCollectionNextTarget(ctx, collectionID)
balancer.targetMgr.UpdateCollectionCurrentTarget(ctx, collectionID)
@ -959,7 +961,7 @@ func (suite *ChannelLevelScoreBalancerTestSuite) TestExclusiveChannelBalance_Seg
collection.Status = querypb.LoadStatus_Loaded
balancer.meta.CollectionManager.PutCollection(ctx, collection)
balancer.meta.CollectionManager.PutPartition(ctx, utils.CreateTestPartition(collectionID, partitionID))
balancer.meta.ReplicaManager.Spawn(ctx, 1, map[string]int{meta.DefaultResourceGroupName: 1}, []string{"channel1", "channel2"})
balancer.meta.ReplicaManager.Spawn(ctx, 1, map[string]int{meta.DefaultResourceGroupName: 1}, []string{"channel1", "channel2"}, commonpb.LoadPriority_LOW)
balancer.targetMgr.UpdateCollectionNextTarget(ctx, collectionID)
balancer.targetMgr.UpdateCollectionCurrentTarget(ctx, collectionID)
@ -1059,7 +1061,7 @@ func (suite *ChannelLevelScoreBalancerTestSuite) TestExclusiveChannelBalance_Nod
collection.Status = querypb.LoadStatus_Loaded
balancer.meta.CollectionManager.PutCollection(ctx, collection)
balancer.meta.CollectionManager.PutPartition(ctx, utils.CreateTestPartition(collectionID, partitionID))
balancer.meta.ReplicaManager.Spawn(ctx, 1, map[string]int{meta.DefaultResourceGroupName: 1}, []string{"channel1", "channel2"})
balancer.meta.ReplicaManager.Spawn(ctx, 1, map[string]int{meta.DefaultResourceGroupName: 1}, []string{"channel1", "channel2"}, commonpb.LoadPriority_LOW)
balancer.targetMgr.UpdateCollectionNextTarget(ctx, collectionID)
balancer.targetMgr.UpdateCollectionCurrentTarget(ctx, collectionID)
@ -1186,7 +1188,7 @@ func (suite *ChannelLevelScoreBalancerTestSuite) TestExclusiveChannelBalance_Seg
collection.Status = querypb.LoadStatus_Loaded
balancer.meta.CollectionManager.PutCollection(ctx, collection)
balancer.meta.CollectionManager.PutPartition(ctx, utils.CreateTestPartition(collectionID, partitionID))
balancer.meta.ReplicaManager.Spawn(ctx, 1, map[string]int{meta.DefaultResourceGroupName: 1}, []string{"channel1", "channel2"})
balancer.meta.ReplicaManager.Spawn(ctx, 1, map[string]int{meta.DefaultResourceGroupName: 1}, []string{"channel1", "channel2"}, commonpb.LoadPriority_LOW)
balancer.targetMgr.UpdateCollectionNextTarget(ctx, collectionID)
balancer.targetMgr.UpdateCollectionCurrentTarget(ctx, collectionID)

View File

@ -25,6 +25,7 @@ import (
"github.com/stretchr/testify/suite"
"go.uber.org/atomic"
"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/metastore/kv/querycoord"
"github.com/milvus-io/milvus/internal/querycoordv2/meta"
@ -1252,7 +1253,7 @@ func (suite *ScoreBasedBalancerTestSuite) TestBalanceChannelOnMultiCollections()
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)
balancer.meta.ReplicaManager.Spawn(ctx, collectionID, map[string]int{meta.DefaultResourceGroupName: 1}, nil, commonpb.LoadPriority_LOW)
channels := make([]*datapb.VchannelInfo, channelNum)
for i := 0; i < channelNum; i++ {
@ -1339,7 +1340,7 @@ func (suite *ScoreBasedBalancerTestSuite) TestBalanceChannelOnDifferentQN() {
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)
balancer.meta.ReplicaManager.Spawn(ctx, collectionID, map[string]int{meta.DefaultResourceGroupName: 1}, nil, commonpb.LoadPriority_LOW)
channels := make([]*datapb.VchannelInfo, channelNum)
for i := 0; i < channelNum; i++ {
@ -1417,7 +1418,7 @@ func (suite *ScoreBasedBalancerTestSuite) TestBalanceChannelOnChannelExclusive()
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)
balancer.meta.ReplicaManager.Spawn(ctx, collectionID, map[string]int{meta.DefaultResourceGroupName: 1}, nil, commonpb.LoadPriority_LOW)
channels := make([]*datapb.VchannelInfo, channelNum)
for i := 0; i < channelNum; i++ {
@ -1510,7 +1511,7 @@ func (suite *ScoreBasedBalancerTestSuite) TestBalanceChannelOnStoppingNode() {
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)
balancer.meta.ReplicaManager.Spawn(ctx, collectionID, map[string]int{meta.DefaultResourceGroupName: 1}, nil, commonpb.LoadPriority_LOW)
channels := make([]*datapb.VchannelInfo, channelNum)
for i := 0; i < channelNum; i++ {

View File

@ -52,6 +52,7 @@ func CreateSegmentTasksFromPlans(ctx context.Context, source task.Source, timeou
source,
p.Segment.GetCollectionID(),
p.Replica,
p.LoadPriority,
actions...,
)
if err != nil {
@ -73,6 +74,7 @@ func CreateSegmentTasksFromPlans(ctx context.Context, source task.Source, timeou
zap.Int64("replica", p.Replica.GetID()),
zap.String("channel", p.Segment.GetInsertChannel()),
zap.String("level", p.Segment.GetLevel().String()),
zap.Int32("loadPriority", int32(p.LoadPriority)),
zap.Int64("from", p.From),
zap.Int64("to", p.To))
if task.GetTaskType(t) == task.TaskTypeMove {

View File

@ -217,6 +217,7 @@ func (c *IndexChecker) createSegmentUpdateTask(ctx context.Context, segment *met
c.ID(),
segment.GetCollectionID(),
replica,
replica.LoadPriority(),
action,
)
if err != nil {
@ -273,6 +274,7 @@ func (c *IndexChecker) createSegmentStatsUpdateTask(ctx context.Context, segment
c.ID(),
segment.GetCollectionID(),
replica,
replica.LoadPriority(),
action,
)
if err != nil {

View File

@ -25,6 +25,7 @@ import (
"go.opentelemetry.io/otel/trace"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/internal/querycoordv2/balance"
"github.com/milvus-io/milvus/internal/querycoordv2/meta"
. "github.com/milvus-io/milvus/internal/querycoordv2/params"
@ -127,8 +128,8 @@ func (c *SegmentChecker) checkReplica(ctx context.Context, replica *meta.Replica
ret := make([]task.Task, 0)
// compare with targets to find the lack and redundancy of segments
lacks, redundancies := c.getSealedSegmentDiff(ctx, replica.GetCollectionID(), replica.GetID())
tasks := c.createSegmentLoadTasks(c.getTraceCtx(ctx, replica.GetCollectionID()), lacks, replica)
lacks, loadPriorities, redundancies := c.getSealedSegmentDiff(ctx, replica.GetCollectionID(), replica.GetID())
tasks := c.createSegmentLoadTasks(c.getTraceCtx(ctx, replica.GetCollectionID()), lacks, loadPriorities, replica)
task.SetReason("lacks of segment", tasks...)
task.SetPriority(task.TaskPriorityNormal, tasks...)
ret = append(ret, tasks...)
@ -224,7 +225,7 @@ func (c *SegmentChecker) getSealedSegmentDiff(
ctx context.Context,
collectionID int64,
replicaID int64,
) (toLoad []*datapb.SegmentInfo, toRelease []*meta.Segment) {
) (toLoad []*datapb.SegmentInfo, loadPriorities []commonpb.LoadPriority, toRelease []*meta.Segment) {
replica := c.meta.Get(ctx, replicaID)
if replica == nil {
log.Info("replica does not exist, skip it")
@ -247,10 +248,15 @@ func (c *SegmentChecker) getSealedSegmentDiff(
nextTargetExist := c.targetMgr.IsNextTargetExist(ctx, collectionID)
nextTargetMap := c.targetMgr.GetSealedSegmentsByCollection(ctx, collectionID, meta.NextTarget)
currentTargetMap := c.targetMgr.GetSealedSegmentsByCollection(ctx, collectionID, meta.CurrentTarget)
// Segment which exist on next target, but not on dist
for _, segment := range nextTargetMap {
if isSegmentLack(segment) {
_, existOnCurrent := currentTargetMap[segment.GetID()]
if existOnCurrent {
loadPriorities = append(loadPriorities, commonpb.LoadPriority_HIGH)
} else {
loadPriorities = append(loadPriorities, replica.LoadPriority())
}
toLoad = append(toLoad, segment)
}
}
@ -359,10 +365,14 @@ func (c *SegmentChecker) filterOutSegmentInUse(ctx context.Context, replica *met
return notUsed
}
func (c *SegmentChecker) createSegmentLoadTasks(ctx context.Context, segments []*datapb.SegmentInfo, replica *meta.Replica) []task.Task {
func (c *SegmentChecker) createSegmentLoadTasks(ctx context.Context, segments []*datapb.SegmentInfo, loadPriorities []commonpb.LoadPriority, replica *meta.Replica) []task.Task {
if len(segments) == 0 {
return nil
}
priorityMap := make(map[int64]commonpb.LoadPriority)
for i, s := range segments {
priorityMap[s.GetID()] = loadPriorities[i]
}
shardSegments := lo.GroupBy(segments, func(s *datapb.SegmentInfo) string {
return s.GetInsertChannel()
@ -389,6 +399,7 @@ func (c *SegmentChecker) createSegmentLoadTasks(ctx context.Context, segments []
shardPlans := c.getBalancerFunc().AssignSegment(ctx, replica.GetCollectionID(), segmentInfos, rwNodes, true)
for i := range shardPlans {
shardPlans[i].Replica = replica
shardPlans[i].LoadPriority = priorityMap[shardPlans[i].Segment.GetID()]
}
plans = append(plans, shardPlans...)
}
@ -406,6 +417,7 @@ func (c *SegmentChecker) createSegmentReduceTasks(ctx context.Context, segments
c.ID(),
s.GetCollectionID(),
replica,
replica.LoadPriority(),
action,
)
if err != nil {

View File

@ -24,6 +24,7 @@ import (
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/suite"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"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"
@ -629,6 +630,96 @@ func (suite *SegmentCheckerTestSuite) TestReleaseDroppedSegments() {
suite.Equal(tasks[0].Priority(), task.TaskPriorityNormal)
}
func (suite *SegmentCheckerTestSuite) TestLoadPriority() {
ctx := context.Background()
collectionID := int64(1)
replicaID := int64(1)
// prepare replica
replica := meta.NewReplicaWithPriority(&querypb.Replica{
ID: replicaID,
CollectionID: collectionID,
Nodes: []int64{1, 2},
}, commonpb.LoadPriority_LOW)
suite.meta.ReplicaManager.Put(ctx, replica)
// prepare segments
segment1 := &datapb.SegmentInfo{
ID: 1,
CollectionID: collectionID,
PartitionID: -1,
InsertChannel: "channel1",
State: commonpb.SegmentState_Sealed,
NumOfRows: 100,
StartPosition: &msgpb.MsgPosition{Timestamp: 100},
DmlPosition: &msgpb.MsgPosition{Timestamp: 200},
}
segment2 := &datapb.SegmentInfo{
ID: 2,
CollectionID: collectionID,
PartitionID: -1,
InsertChannel: "channel1",
State: commonpb.SegmentState_Sealed,
NumOfRows: 100,
StartPosition: &msgpb.MsgPosition{Timestamp: 100},
DmlPosition: &msgpb.MsgPosition{Timestamp: 200},
}
// set up current target
suite.broker.EXPECT().GetRecoveryInfoV2(mock.Anything, collectionID).Return(
[]*datapb.VchannelInfo{
{
CollectionID: collectionID,
ChannelName: "channel1",
},
},
[]*datapb.SegmentInfo{segment1},
nil,
).Once()
suite.checker.targetMgr.UpdateCollectionNextTarget(ctx, collectionID)
suite.checker.targetMgr.UpdateCollectionCurrentTarget(ctx, collectionID)
// set up next target with segment1 and segment2
suite.broker.EXPECT().GetRecoveryInfoV2(mock.Anything, collectionID).Return(
[]*datapb.VchannelInfo{
{
CollectionID: collectionID,
ChannelName: "channel1",
},
},
[]*datapb.SegmentInfo{segment1, segment2},
nil,
).Once()
suite.checker.targetMgr.UpdateCollectionNextTarget(ctx, collectionID)
// test getSealedSegmentDiff
toLoad, loadPriorities, toRelease := suite.checker.getSealedSegmentDiff(ctx, collectionID, replicaID)
// verify results
suite.Equal(2, len(toLoad))
suite.Equal(2, len(loadPriorities))
suite.Equal(0, len(toRelease))
// segment2 not in current target, should use replica's priority
suite.True(segment2.GetID() == toLoad[0].GetID() || segment2.GetID() == toLoad[1].GetID())
suite.True(segment1.GetID() == toLoad[0].GetID() || segment1.GetID() == toLoad[1].GetID())
if segment2.GetID() == toLoad[0].GetID() {
suite.Equal(commonpb.LoadPriority_LOW, loadPriorities[0])
suite.Equal(commonpb.LoadPriority_HIGH, loadPriorities[1])
} else {
suite.Equal(commonpb.LoadPriority_HIGH, loadPriorities[0])
suite.Equal(commonpb.LoadPriority_LOW, loadPriorities[1])
}
// update current target to include segment2
suite.checker.targetMgr.UpdateCollectionCurrentTarget(ctx, collectionID)
// test again
toLoad, loadPriorities, toRelease = suite.checker.getSealedSegmentDiff(ctx, collectionID, replicaID)
// verify results
suite.Equal(0, len(toLoad))
suite.Equal(0, len(loadPriorities))
suite.Equal(0, len(toRelease))
}
func (suite *SegmentCheckerTestSuite) TestFilterOutExistedOnLeader() {
checker := suite.checker

View File

@ -132,6 +132,7 @@ func (s *Server) balanceSegments(ctx context.Context,
utils.ManualBalance,
collectionID,
plan.Replica,
replica.LoadPriority(),
actions...,
)
if err != nil {

View File

@ -168,7 +168,8 @@ func (job *LoadCollectionJob) Execute() error {
if len(replicas) == 0 {
// 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(), job.collInfo.GetVirtualChannelNames())
_, err = utils.SpawnReplicasWithRG(job.ctx, job.meta, req.GetCollectionID(), req.GetResourceGroups(),
req.GetReplicaNumber(), job.collInfo.GetVirtualChannelNames(), req.GetPriority())
if err != nil {
msg := "failed to spawn replica for collection"
log.Warn(msg, zap.Error(err))
@ -357,7 +358,8 @@ func (job *LoadPartitionJob) Execute() error {
// 2. create replica if not exist
replicas := job.meta.ReplicaManager.GetByCollection(context.TODO(), req.GetCollectionID())
if len(replicas) == 0 {
_, err = utils.SpawnReplicasWithRG(job.ctx, job.meta, req.GetCollectionID(), req.GetResourceGroups(), req.GetReplicaNumber(), job.collInfo.GetVirtualChannelNames())
_, err = utils.SpawnReplicasWithRG(job.ctx, job.meta, req.GetCollectionID(), req.GetResourceGroups(), req.GetReplicaNumber(),
job.collInfo.GetVirtualChannelNames(), req.GetPriority())
if err != nil {
msg := "failed to spawn replica for collection"
log.Warn(msg, zap.Error(err))

View File

@ -22,6 +22,7 @@ import (
"github.com/samber/lo"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/internal/querycoordv2/meta"
"github.com/milvus-io/milvus/internal/querycoordv2/observers"
"github.com/milvus-io/milvus/internal/querycoordv2/utils"
@ -99,7 +100,7 @@ func (job *UpdateLoadConfigJob) Execute() error {
// 3. try to spawn new replica
channels := job.targetMgr.GetDmChannelsByCollection(job.ctx, job.collectionID, meta.CurrentTargetFirst)
newReplicas, spawnErr := job.meta.ReplicaManager.Spawn(job.ctx, job.collectionID, toSpawn, lo.Keys(channels))
newReplicas, spawnErr := job.meta.ReplicaManager.Spawn(job.ctx, job.collectionID, toSpawn, lo.Keys(channels), commonpb.LoadPriority_LOW)
if spawnErr != nil {
log.Warn("failed to spawn replica", zap.Error(spawnErr))
err := spawnErr

View File

@ -3,6 +3,7 @@ package meta
import (
"google.golang.org/protobuf/proto"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/pkg/v2/proto/querypb"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
@ -71,6 +72,7 @@ type Replica struct {
// always keep consistent with replicaPB.RoSqNodes.
// node used by replica but cannot add more channel on it.
// include the rebalance node.
loadPriority commonpb.LoadPriority
}
// Deprecated: may break the consistency of ReplicaManager, use `Spawn` of `ReplicaManager` or `newReplica` instead.
@ -92,9 +94,23 @@ func newReplica(replica *querypb.Replica) *Replica {
roNodes: typeutil.NewUniqueSet(replica.RoNodes...),
rwSQNodes: typeutil.NewUniqueSet(replica.RwSqNodes...),
roSQNodes: typeutil.NewUniqueSet(replica.RoSqNodes...),
loadPriority: commonpb.LoadPriority_HIGH,
}
}
func NewReplicaWithPriority(replica *querypb.Replica, priority commonpb.LoadPriority) *Replica {
return &Replica{
replicaPB: proto.Clone(replica).(*querypb.Replica),
rwNodes: typeutil.NewUniqueSet(replica.Nodes...),
roNodes: typeutil.NewUniqueSet(replica.RoNodes...),
loadPriority: priority,
}
}
func (replica *Replica) LoadPriority() commonpb.LoadPriority {
return replica.loadPriority
}
// GetID returns the id of the replica.
func (replica *Replica) GetID() typeutil.UniqueID {
return replica.replicaPB.GetID()
@ -253,6 +269,7 @@ func (replica *Replica) CopyForWrite() *mutableReplica {
roNodes: typeutil.NewUniqueSet(replica.replicaPB.RoNodes...),
rwSQNodes: typeutil.NewUniqueSet(replica.replicaPB.RwSqNodes...),
roSQNodes: typeutil.NewUniqueSet(replica.replicaPB.RoSqNodes...),
loadPriority: replica.LoadPriority(),
},
exclusiveRWNodeToChannel: exclusiveRWNodeToChannel,
}

View File

@ -25,6 +25,7 @@ import (
"github.com/samber/lo"
"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/metastore"
"github.com/milvus-io/milvus/pkg/v2/log"
@ -42,7 +43,8 @@ type ReplicaManagerInterface interface {
// Basic operations
Recover(ctx context.Context, collections []int64) error
Get(ctx context.Context, id typeutil.UniqueID) *Replica
Spawn(ctx context.Context, collection int64, replicaNumInRG map[string]int, channels []string) ([]*Replica, error)
Spawn(ctx context.Context, collection int64,
replicaNumInRG map[string]int, channels []string, loadPriority commonpb.LoadPriority) ([]*Replica, error)
// Replica manipulation
TransferReplica(ctx context.Context, collectionID typeutil.UniqueID, srcRGName string, dstRGName string, replicaNum int) error
@ -126,7 +128,8 @@ func (m *ReplicaManager) Recover(ctx context.Context, collections []int64) error
}
if collectionSet.Contain(replica.GetCollectionID()) {
m.putReplicaInMemory(newReplica(replica))
rep := NewReplicaWithPriority(replica, commonpb.LoadPriority_HIGH)
m.putReplicaInMemory(rep)
log.Info("recover replica",
zap.Int64("collectionID", replica.GetCollectionID()),
zap.Int64("replicaID", replica.GetID()),
@ -157,7 +160,9 @@ func (m *ReplicaManager) Get(ctx context.Context, id typeutil.UniqueID) *Replica
}
// Spawn spawns N replicas at resource group for given collection in ReplicaManager.
func (m *ReplicaManager) Spawn(ctx context.Context, collection int64, replicaNumInRG map[string]int, channels []string) ([]*Replica, error) {
func (m *ReplicaManager) Spawn(ctx context.Context, collection int64, replicaNumInRG map[string]int,
channels []string, loadPriority commonpb.LoadPriority,
) ([]*Replica, error) {
m.rwmutex.Lock()
defer m.rwmutex.Unlock()
@ -178,12 +183,12 @@ func (m *ReplicaManager) Spawn(ctx context.Context, collection int64, replicaNum
channelExclusiveNodeInfo[channel] = &querypb.ChannelNodeInfo{}
}
}
replicas = append(replicas, newReplica(&querypb.Replica{
replicas = append(replicas, NewReplicaWithPriority(&querypb.Replica{
ID: id,
CollectionID: collection,
ResourceGroup: rgName,
ChannelNodeInfos: channelExclusiveNodeInfo,
}))
}, loadPriority))
}
}
if err := m.put(ctx, replicas...); err != nil {

View File

@ -26,6 +26,7 @@ import (
"github.com/stretchr/testify/suite"
"google.golang.org/protobuf/proto"
"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"
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
@ -120,14 +121,14 @@ func (suite *ReplicaManagerSuite) TestSpawn() {
ctx := suite.ctx
mgr.idAllocator = ErrorIDAllocator()
_, err := mgr.Spawn(ctx, 1, map[string]int{DefaultResourceGroupName: 1}, nil)
_, err := mgr.Spawn(ctx, 1, map[string]int{DefaultResourceGroupName: 1}, nil, commonpb.LoadPriority_LOW)
suite.Error(err)
replicas := mgr.GetByCollection(ctx, 1)
suite.Len(replicas, 0)
mgr.idAllocator = suite.idAllocator
replicas, err = mgr.Spawn(ctx, 1, map[string]int{DefaultResourceGroupName: 1}, []string{"channel1", "channel2"})
replicas, err = mgr.Spawn(ctx, 1, map[string]int{DefaultResourceGroupName: 1}, []string{"channel1", "channel2"}, commonpb.LoadPriority_LOW)
suite.NoError(err)
for _, replica := range replicas {
suite.Len(replica.replicaPB.GetChannelNodeInfos(), 0)
@ -135,7 +136,7 @@ func (suite *ReplicaManagerSuite) TestSpawn() {
paramtable.Get().Save(paramtable.Get().QueryCoordCfg.Balancer.Key, ChannelLevelScoreBalancerName)
defer paramtable.Get().Reset(paramtable.Get().QueryCoordCfg.Balancer.Key)
replicas, err = mgr.Spawn(ctx, 2, map[string]int{DefaultResourceGroupName: 1}, []string{"channel1", "channel2"})
replicas, err = mgr.Spawn(ctx, 2, map[string]int{DefaultResourceGroupName: 1}, []string{"channel1", "channel2"}, commonpb.LoadPriority_LOW)
suite.NoError(err)
for _, replica := range replicas {
suite.Len(replica.replicaPB.GetChannelNodeInfos(), 2)
@ -292,7 +293,7 @@ func (suite *ReplicaManagerSuite) spawnAll() {
ctx := suite.ctx
for id, cfg := range suite.collections {
replicas, err := mgr.Spawn(ctx, id, cfg.spawnConfig, nil)
replicas, err := mgr.Spawn(ctx, id, cfg.spawnConfig, nil, commonpb.LoadPriority_LOW)
suite.NoError(err)
totalSpawn := 0
rgsOfCollection := make(map[string]typeutil.UniqueSet)
@ -308,12 +309,12 @@ func (suite *ReplicaManagerSuite) spawnAll() {
func (suite *ReplicaManagerSuite) TestResourceGroup() {
mgr := NewReplicaManager(suite.idAllocator, suite.catalog)
ctx := suite.ctx
replicas1, err := mgr.Spawn(ctx, int64(1000), map[string]int{DefaultResourceGroupName: 1}, nil)
replicas1, err := mgr.Spawn(ctx, int64(1000), map[string]int{DefaultResourceGroupName: 1}, nil, commonpb.LoadPriority_LOW)
suite.NoError(err)
suite.NotNil(replicas1)
suite.Len(replicas1, 1)
replica2, err := mgr.Spawn(ctx, int64(2000), map[string]int{DefaultResourceGroupName: 1}, nil)
replica2, err := mgr.Spawn(ctx, int64(2000), map[string]int{DefaultResourceGroupName: 1}, nil, commonpb.LoadPriority_LOW)
suite.NoError(err)
suite.NotNil(replica2)
suite.Len(replica2, 1)
@ -403,7 +404,7 @@ func (suite *ReplicaManagerV2Suite) TestSpawn() {
ctx := suite.ctx
for id, cfg := range suite.collections {
replicas, err := mgr.Spawn(ctx, id, cfg.spawnConfig, nil)
replicas, err := mgr.Spawn(ctx, id, cfg.spawnConfig, nil, commonpb.LoadPriority_LOW)
suite.NoError(err)
rgsOfCollection := make(map[string]typeutil.UniqueSet)
for rg := range cfg.spawnConfig {

View File

@ -26,6 +26,7 @@ import (
"github.com/stretchr/testify/suite"
clientv3 "go.etcd.io/etcd/client/v3"
"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/metastore"
"github.com/milvus-io/milvus/internal/metastore/kv/querycoord"
@ -494,7 +495,8 @@ func (suite *CollectionObserverSuite) loadAll() {
func (suite *CollectionObserverSuite) load(collection int64) {
ctx := suite.ctx
// Mock meta data
replicas, err := suite.meta.ReplicaManager.Spawn(ctx, collection, map[string]int{meta.DefaultResourceGroupName: int(suite.replicaNumber[collection])}, nil)
replicas, err := suite.meta.ReplicaManager.Spawn(ctx, collection, map[string]int{meta.DefaultResourceGroupName: int(suite.replicaNumber[collection])},
nil, commonpb.LoadPriority_LOW)
suite.NoError(err)
for _, replica := range replicas {
replica.AddRWNode(suite.nodes...)

View File

@ -23,6 +23,7 @@ import (
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/suite"
"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/coordinator/snmanager"
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
@ -132,7 +133,7 @@ func (suite *ReplicaObserverSuite) TestCheckNodesInReplica() {
replicas, err := suite.meta.Spawn(ctx, suite.collectionID, map[string]int{
"rg1": 1,
"rg2": 1,
}, nil)
}, nil, commonpb.LoadPriority_LOW)
suite.NoError(err)
suite.Equal(2, len(replicas))
@ -263,7 +264,7 @@ func (suite *ReplicaObserverSuite) TestCheckSQnodesInReplica() {
replicas, err := suite.meta.Spawn(ctx, suite.collectionID, map[string]int{
"rg1": 1,
"rg2": 1,
}, nil)
}, nil, commonpb.LoadPriority_LOW)
suite.NoError(err)
suite.Equal(2, len(replicas))

View File

@ -25,6 +25,7 @@ import (
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/suite"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"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"
@ -107,7 +108,7 @@ func (suite *TargetObserverSuite) SetupTest() {
suite.NoError(err)
err = suite.meta.CollectionManager.PutPartition(suite.ctx, utils.CreateTestPartition(suite.collectionID, suite.partitionID))
suite.NoError(err)
replicas, err := suite.meta.ReplicaManager.Spawn(suite.ctx, suite.collectionID, map[string]int{meta.DefaultResourceGroupName: 1}, nil)
replicas, err := suite.meta.ReplicaManager.Spawn(suite.ctx, suite.collectionID, map[string]int{meta.DefaultResourceGroupName: 1}, nil, commonpb.LoadPriority_LOW)
suite.NoError(err)
replicas[0].AddRWNode(2)
err = suite.meta.ReplicaManager.Put(suite.ctx, replicas...)
@ -349,7 +350,7 @@ func (suite *TargetObserverCheckSuite) SetupTest() {
suite.NoError(err)
err = suite.meta.CollectionManager.PutPartition(suite.ctx, utils.CreateTestPartition(suite.collectionID, suite.partitionID))
suite.NoError(err)
replicas, err := suite.meta.ReplicaManager.Spawn(suite.ctx, suite.collectionID, map[string]int{meta.DefaultResourceGroupName: 1}, nil)
replicas, err := suite.meta.ReplicaManager.Spawn(suite.ctx, suite.collectionID, map[string]int{meta.DefaultResourceGroupName: 1}, nil, commonpb.LoadPriority_LOW)
suite.NoError(err)
replicas[0].AddRWNode(2)
err = suite.meta.ReplicaManager.Put(suite.ctx, replicas...)

View File

@ -35,6 +35,7 @@ import (
. "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/v2/common"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
@ -207,11 +208,10 @@ func (ex *Executor) loadSegment(task *SegmentTask, step int) error {
return err
}
loadInfo, indexInfos, err := ex.getLoadInfo(ctx, task.CollectionID(), action.SegmentID, channel)
loadInfo, indexInfos, err := ex.getLoadInfo(ctx, task.CollectionID(), action.SegmentID, channel, task.LoadPriority())
if err != nil {
return err
}
req := packLoadSegmentRequest(
task,
action,
@ -586,7 +586,7 @@ func (ex *Executor) setDistribution(task *LeaderTask, step int) error {
return err
}
loadInfo, indexInfo, err := ex.getLoadInfo(ctx, task.CollectionID(), action.SegmentID(), channel)
loadInfo, indexInfo, err := ex.getLoadInfo(ctx, task.CollectionID(), action.SegmentID(), channel, commonpb.LoadPriority_LOW)
if err != nil {
return err
}
@ -717,7 +717,7 @@ func (ex *Executor) getMetaInfo(ctx context.Context, task Task) (*milvuspb.Descr
return collectionInfo, loadMeta, channel, nil
}
func (ex *Executor) getLoadInfo(ctx context.Context, collectionID, segmentID int64, channel *meta.DmChannel) (*querypb.SegmentLoadInfo, []*indexpb.IndexInfo, error) {
func (ex *Executor) getLoadInfo(ctx context.Context, collectionID, segmentID int64, channel *meta.DmChannel, priority commonpb.LoadPriority) (*querypb.SegmentLoadInfo, []*indexpb.IndexInfo, error) {
log := log.Ctx(ctx)
segmentInfos, err := ex.broker.GetSegmentInfo(ctx, segmentID)
if err != nil || len(segmentInfos) == 0 {
@ -758,8 +758,11 @@ func (ex *Executor) getLoadInfo(ctx context.Context, collectionID, segmentID int
}
}
segmentIndex.IndexParams = funcutil.Map2KeyValuePair(params)
segmentIndex.IndexParams = append(segmentIndex.IndexParams,
&commonpb.KeyValuePair{Key: common.LoadPriorityKey, Value: priority.String()})
}
loadInfo := utils.PackSegmentLoadInfo(segment, channel.GetSeekPosition(), indexes[segment.GetID()])
loadInfo.Priority = priority
return loadInfo, indexInfos, nil
}

View File

@ -27,6 +27,7 @@ import (
"go.opentelemetry.io/otel/trace"
"go.uber.org/atomic"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/internal/json"
"github.com/milvus-io/milvus/internal/querycoordv2/meta"
"github.com/milvus-io/milvus/pkg/v2/proto/querypb"
@ -325,6 +326,7 @@ type SegmentTask struct {
*baseTask
segmentID typeutil.UniqueID
loadPriority commonpb.LoadPriority
}
// NewSegmentTask creates a SegmentTask with actions,
@ -335,6 +337,7 @@ func NewSegmentTask(ctx context.Context,
source Source,
collectionID typeutil.UniqueID,
replica *meta.Replica,
loadPriority commonpb.LoadPriority,
actions ...Action,
) (*SegmentTask, error) {
if len(actions) == 0 {
@ -361,9 +364,14 @@ func NewSegmentTask(ctx context.Context,
return &SegmentTask{
baseTask: base,
segmentID: segmentID,
loadPriority: loadPriority,
}, nil
}
func (task *SegmentTask) LoadPriority() commonpb.LoadPriority {
return task.loadPriority
}
func (task *SegmentTask) SegmentID() typeutil.UniqueID {
return task.segmentID
}
@ -377,7 +385,7 @@ func (task *SegmentTask) Name() string {
}
func (task *SegmentTask) String() string {
return fmt.Sprintf("%s [segmentID=%d]", task.baseTask.String(), task.segmentID)
return fmt.Sprintf("%s [segmentID=%d][loadPriority=%d]", task.baseTask.String(), task.segmentID, task.loadPriority)
}
func (task *SegmentTask) MarshalJSON() ([]byte, error) {

View File

@ -495,6 +495,7 @@ func (suite *TaskSuite) TestLoadSegmentTask() {
WrapIDSource(0),
suite.collection,
suite.replica,
commonpb.LoadPriority_LOW,
NewSegmentAction(targetNode, ActionTypeGrow, channel.GetChannelName(), segment),
)
suite.NoError(err)
@ -608,6 +609,7 @@ func (suite *TaskSuite) TestLoadSegmentTaskNotIndex() {
WrapIDSource(0),
suite.collection,
suite.replica,
commonpb.LoadPriority_LOW,
NewSegmentAction(targetNode, ActionTypeGrow, channel.GetChannelName(), segment),
)
suite.NoError(err)
@ -706,6 +708,7 @@ func (suite *TaskSuite) TestLoadSegmentTaskFailed() {
WrapIDSource(0),
suite.collection,
suite.replica,
commonpb.LoadPriority_LOW,
NewSegmentAction(targetNode, ActionTypeGrow, channel.GetChannelName(), segment),
)
suite.NoError(err)
@ -771,6 +774,7 @@ func (suite *TaskSuite) TestReleaseSegmentTask() {
WrapIDSource(0),
suite.collection,
suite.replica,
commonpb.LoadPriority_LOW,
NewSegmentAction(targetNode, ActionTypeReduce, channel.GetChannelName(), segment),
)
suite.NoError(err)
@ -813,6 +817,7 @@ func (suite *TaskSuite) TestReleaseGrowingSegmentTask() {
WrapIDSource(0),
suite.collection,
suite.replica,
commonpb.LoadPriority_LOW,
NewSegmentActionWithScope(targetNode, ActionTypeReduce, "", segment, querypb.DataScope_Streaming, 0),
)
suite.NoError(err)
@ -920,6 +925,7 @@ func (suite *TaskSuite) TestMoveSegmentTask() {
WrapIDSource(0),
suite.collection,
suite.replica,
commonpb.LoadPriority_LOW,
NewSegmentAction(targetNode, ActionTypeGrow, channel.GetChannelName(), segment),
NewSegmentAction(sourceNode, ActionTypeReduce, channel.GetChannelName(), segment),
)
@ -1020,6 +1026,7 @@ func (suite *TaskSuite) TestMoveSegmentTaskStale() {
WrapIDSource(0),
suite.collection,
suite.replica,
commonpb.LoadPriority_LOW,
NewSegmentAction(targetNode, ActionTypeGrow, channel.GetChannelName(), segment),
NewSegmentAction(sourceNode, ActionTypeReduce, channel.GetChannelName(), segment),
)
@ -1102,6 +1109,7 @@ func (suite *TaskSuite) TestTaskCanceled() {
WrapIDSource(0),
suite.collection,
suite.replica,
commonpb.LoadPriority_LOW,
NewSegmentAction(targetNode, ActionTypeGrow, channel.GetChannelName(), segment),
)
suite.NoError(err)
@ -1190,6 +1198,7 @@ func (suite *TaskSuite) TestSegmentTaskStale() {
WrapIDSource(0),
suite.collection,
suite.replica,
commonpb.LoadPriority_LOW,
NewSegmentAction(targetNode, ActionTypeGrow, channel.GetChannelName(), segment),
)
suite.NoError(err)
@ -1427,19 +1436,19 @@ func (suite *TaskSuite) TestCreateTaskBehavior() {
suite.ErrorIs(err, merr.ErrParameterInvalid)
suite.Nil(chanelTask)
segmentTask, err := NewSegmentTask(context.TODO(), 5*time.Second, WrapIDSource(0), 0, meta.NilReplica)
segmentTask, err := NewSegmentTask(context.TODO(), 5*time.Second, WrapIDSource(0), 0, meta.NilReplica, commonpb.LoadPriority_LOW)
suite.ErrorIs(err, merr.ErrParameterInvalid)
suite.Nil(segmentTask)
channelAction := NewChannelAction(0, 0, "fake-channel1")
segmentTask, err = NewSegmentTask(context.TODO(), 5*time.Second, WrapIDSource(0), 0, meta.NilReplica, channelAction)
segmentTask, err = NewSegmentTask(context.TODO(), 5*time.Second, WrapIDSource(0), 0, meta.NilReplica, commonpb.LoadPriority_LOW, channelAction)
suite.ErrorIs(err, merr.ErrParameterInvalid)
suite.Nil(segmentTask)
segmentAction1 := NewSegmentAction(0, 0, "", 0)
segmentAction2 := NewSegmentAction(0, 0, "", 1)
segmentTask, err = NewSegmentTask(context.TODO(), 5*time.Second, WrapIDSource(0), 0, meta.NilReplica, segmentAction1, segmentAction2)
segmentTask, err = NewSegmentTask(context.TODO(), 5*time.Second, WrapIDSource(0), 0, meta.NilReplica, commonpb.LoadPriority_LOW, segmentAction1, segmentAction2)
suite.ErrorIs(err, merr.ErrParameterInvalid)
suite.Nil(segmentTask)
@ -1460,6 +1469,7 @@ func (suite *TaskSuite) TestSegmentTaskReplace() {
WrapIDSource(0),
suite.collection,
suite.replica,
commonpb.LoadPriority_LOW,
NewSegmentAction(targetNode, ActionTypeGrow, "", segment),
)
suite.NoError(err)
@ -1477,6 +1487,7 @@ func (suite *TaskSuite) TestSegmentTaskReplace() {
WrapIDSource(0),
suite.collection,
suite.replica,
commonpb.LoadPriority_LOW,
NewSegmentAction(targetNode, ActionTypeGrow, "", segment),
)
suite.NoError(err)
@ -1496,6 +1507,7 @@ func (suite *TaskSuite) TestSegmentTaskReplace() {
WrapIDSource(0),
suite.collection,
suite.replica,
commonpb.LoadPriority_LOW,
NewSegmentAction(targetNode, ActionTypeGrow, "", segment),
)
suite.NoError(err)
@ -1538,6 +1550,7 @@ func (suite *TaskSuite) TestNoExecutor() {
WrapIDSource(0),
suite.collection,
suite.replica,
commonpb.LoadPriority_LOW,
NewSegmentAction(targetNode, ActionTypeGrow, channel.GetChannelName(), segment),
)
suite.NoError(err)
@ -1801,6 +1814,7 @@ func (suite *TaskSuite) TestGetTasksJSON() {
WrapIDSource(0),
suite.collection,
suite.replica,
commonpb.LoadPriority_LOW,
NewSegmentAction(1, ActionTypeGrow, "", 1),
)
suite.NoError(err)
@ -1842,6 +1856,7 @@ func (suite *TaskSuite) TestCalculateTaskDelta() {
WrapIDSource(0),
coll,
suite.replica,
commonpb.LoadPriority_LOW,
NewSegmentActionWithScope(nodeID, ActionTypeGrow, "", segmentID, querypb.DataScope_Historical, 100),
)
task1.SetID(1)
@ -1871,6 +1886,7 @@ func (suite *TaskSuite) TestCalculateTaskDelta() {
WrapIDSource(0),
coll2,
suite.replica,
commonpb.LoadPriority_LOW,
NewSegmentActionWithScope(nodeID2, ActionTypeGrow, "", segmentID2, querypb.DataScope_Historical, 100),
)
suite.NoError(err)
@ -1990,6 +2006,7 @@ func (suite *TaskSuite) TestRemoveTaskWithError() {
WrapIDSource(0),
coll,
suite.replica,
commonpb.LoadPriority_LOW,
NewSegmentActionWithScope(nodeID, ActionTypeGrow, "", 1, querypb.DataScope_Historical, 100),
)
suite.NoError(err)

View File

@ -44,6 +44,7 @@ func (s *UtilsSuite) TestPackLoadSegmentRequest() {
nil,
1,
newReplicaDefaultRG(10),
commonpb.LoadPriority_LOW,
action,
)
s.NoError(err)
@ -99,6 +100,7 @@ func (s *UtilsSuite) TestPackLoadSegmentRequestMmap() {
nil,
1,
newReplicaDefaultRG(10),
commonpb.LoadPriority_LOW,
action,
)
s.NoError(err)

View File

@ -25,6 +25,7 @@ import (
"github.com/samber/lo"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/internal/coordinator/snmanager"
"github.com/milvus-io/milvus/internal/querycoordv2/meta"
"github.com/milvus-io/milvus/internal/util/streamingutil"
@ -158,13 +159,15 @@ func AssignReplica(ctx context.Context, m *meta.Meta, resourceGroups []string, r
}
// SpawnReplicasWithRG spawns replicas in rgs one by one for given collection.
func SpawnReplicasWithRG(ctx context.Context, m *meta.Meta, collection int64, resourceGroups []string, replicaNumber int32, channels []string) ([]*meta.Replica, error) {
func SpawnReplicasWithRG(ctx context.Context, m *meta.Meta, collection int64, resourceGroups []string,
replicaNumber int32, channels []string, loadPriority commonpb.LoadPriority,
) ([]*meta.Replica, error) {
replicaNumInRG, err := AssignReplica(ctx, m, resourceGroups, replicaNumber, true)
if err != nil {
return nil, err
}
// Spawn it in replica manager.
replicas, err := m.ReplicaManager.Spawn(ctx, collection, replicaNumInRG, channels)
replicas, err := m.ReplicaManager.Spawn(ctx, collection, replicaNumInRG, channels, loadPriority)
if err != nil {
return nil, err
}

View File

@ -25,6 +25,7 @@ import (
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/rgpb"
etcdKV "github.com/milvus-io/milvus/internal/kv/etcd"
"github.com/milvus-io/milvus/internal/metastore/kv/querycoord"
@ -122,7 +123,7 @@ func TestSpawnReplicasWithRG(t *testing.T) {
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
got, err := SpawnReplicasWithRG(ctx, tt.args.m, tt.args.collection, tt.args.resourceGroups, tt.args.replicaNumber, nil)
got, err := SpawnReplicasWithRG(ctx, tt.args.m, tt.args.collection, tt.args.resourceGroups, tt.args.replicaNumber, nil, commonpb.LoadPriority_LOW)
if (err != nil) != tt.wantErr {
t.Errorf("SpawnReplicasWithRG() error = %v, wantErr %v", err, tt.wantErr)
return

View File

@ -845,6 +845,7 @@ func (s *LocalSegment) AddFieldDataInfo(ctx context.Context, rowCount int64, fie
req := &segcore.AddFieldDataInfoRequest{
Fields: make([]segcore.LoadFieldDataInfo, 0, len(fields)),
RowCount: rowCount,
LoadPriority: s.loadInfo.Load().GetPriority(),
}
for _, field := range fields {
req.Fields = append(req.Fields, segcore.LoadFieldDataInfo{
@ -933,7 +934,7 @@ func (s *LocalSegment) LoadDeltaData(ctx context.Context, deltaData *storage.Del
func GetCLoadInfoWithFunc(ctx context.Context,
fieldSchema *schemapb.FieldSchema,
s *querypb.SegmentLoadInfo,
loadInfo *querypb.SegmentLoadInfo,
indexInfo *querypb.FieldIndexInfo,
f func(c *LoadIndexInfo) error,
) error {
@ -967,9 +968,9 @@ func GetCLoadInfoWithFunc(ctx context.Context,
enableMmap := isIndexMmapEnable(fieldSchema, indexInfo)
indexInfoProto := &cgopb.LoadIndexInfo{
CollectionID: s.GetCollectionID(),
PartitionID: s.GetPartitionID(),
SegmentID: s.GetSegmentID(),
CollectionID: loadInfo.GetCollectionID(),
PartitionID: loadInfo.GetPartitionID(),
SegmentID: loadInfo.GetSegmentID(),
Field: fieldSchema,
EnableMmap: enableMmap,
MmapDirPath: paramtable.Get().QueryNodeCfg.MmapDirPath.GetValue(),
@ -1098,6 +1099,7 @@ func (s *LocalSegment) LoadTextIndex(ctx context.Context, textLogs *datapb.TextI
Schema: f,
CollectionID: s.Collection(),
PartitionID: s.Partition(),
LoadPriority: s.LoadInfo().GetPriority(),
}
marshaled, err := proto.Marshal(cgoProto)
@ -1144,6 +1146,7 @@ func (s *LocalSegment) LoadJSONKeyIndex(ctx context.Context, jsonKeyStats *datap
Schema: f,
CollectionID: s.Collection(),
PartitionID: s.Partition(),
LoadPriority: s.loadInfo.Load().GetPriority(),
}
marshaled, err := proto.Marshal(cgoProto)

View File

@ -889,7 +889,8 @@ func (loader *segmentLoader) LoadSegment(ctx context.Context,
log.Info("start loading segment files",
zap.Int64("rowNum", loadInfo.GetNumOfRows()),
zap.String("segmentType", segment.Type().String()))
zap.String("segmentType", segment.Type().String()),
zap.Int32("priority", int32(loadInfo.GetPriority())))
collection := loader.manager.Collection.Get(segment.Collection())
if collection == nil {

View File

@ -11,6 +11,7 @@ import (
"github.com/cockroachdb/errors"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/proto/segcorepb"
@ -38,6 +39,7 @@ type LoadFieldDataRequest struct {
MMapDir string
RowCount int64
StorageVersion int64
LoadPriority commonpb.LoadPriority
}
type LoadFieldDataInfo struct {
@ -83,6 +85,7 @@ func (req *LoadFieldDataRequest) getCLoadFieldDataRequest() (result *cLoadFieldD
defer C.free(unsafe.Pointer(mmapDir))
C.AppendMMapDirPath(cLoadFieldDataInfo, mmapDir)
}
C.SetLoadPriority(cLoadFieldDataInfo, C.int32_t(req.LoadPriority))
return &cLoadFieldDataRequest{
cLoadFieldDataInfo: cLoadFieldDataInfo,
}, nil

View File

@ -210,6 +210,7 @@ const (
const (
MmapEnabledKey = "mmap.enabled"
LazyLoadEnableKey = "lazyload.enabled"
LoadPriorityKey = "load_priority"
PartitionKeyIsolationKey = "partitionkey.isolation"
FieldSkipLoadKey = "field.skipLoad"
IndexOffsetCacheEnabledKey = "indexoffsetcache.enabled"

View File

@ -105,6 +105,7 @@ message LoadTextIndexInfo {
schema.FieldSchema schema = 5;
int64 collectionID = 6;
int64 partitionID = 7;
common.LoadPriority load_priority = 8;
}
message LoadJsonKeyIndexInfo {
@ -115,4 +116,5 @@ message LoadJsonKeyIndexInfo {
schema.FieldSchema schema = 5;
int64 collectionID = 6;
int64 partitionID = 7;
common.LoadPriority load_priority = 8;
}

View File

@ -892,6 +892,7 @@ type LoadTextIndexInfo struct {
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"`
LoadPriority commonpb.LoadPriority `protobuf:"varint,8,opt,name=load_priority,json=loadPriority,proto3,enum=milvus.proto.common.LoadPriority" json:"load_priority,omitempty"`
}
func (x *LoadTextIndexInfo) Reset() {
@ -975,6 +976,13 @@ func (x *LoadTextIndexInfo) GetPartitionID() int64 {
return 0
}
func (x *LoadTextIndexInfo) GetLoadPriority() commonpb.LoadPriority {
if x != nil {
return x.LoadPriority
}
return commonpb.LoadPriority(0)
}
type LoadJsonKeyIndexInfo struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@ -987,6 +995,7 @@ type LoadJsonKeyIndexInfo struct {
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"`
LoadPriority commonpb.LoadPriority `protobuf:"varint,8,opt,name=load_priority,json=loadPriority,proto3,enum=milvus.proto.common.LoadPriority" json:"load_priority,omitempty"`
}
func (x *LoadJsonKeyIndexInfo) Reset() {
@ -1070,6 +1079,13 @@ func (x *LoadJsonKeyIndexInfo) GetPartitionID() int64 {
return 0
}
func (x *LoadJsonKeyIndexInfo) GetLoadPriority() commonpb.LoadPriority {
if x != nil {
return x.LoadPriority
}
return commonpb.LoadPriority(0)
}
var File_index_cgo_msg_proto protoreflect.FileDescriptor
var file_index_cgo_msg_proto_rawDesc = []byte{
@ -1235,7 +1251,7 @@ var file_index_cgo_msg_proto_rawDesc = []byte{
0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78,
0x63, 0x67, 0x6f, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x73, 0x65, 0x72,
0x74, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x52, 0x12, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49,
0x6e, 0x73, 0x65, 0x72, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x22, 0xf7, 0x01, 0x0a, 0x11, 0x4c,
0x6e, 0x73, 0x65, 0x72, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x22, 0xbf, 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,
@ -1251,23 +1267,32 @@ 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, 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, 0x22, 0xfa, 0x01, 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, 0x42, 0x35, 0x5a, 0x33, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f,
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, 0x22, 0xc2, 0x02, 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, 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,
@ -1303,6 +1328,7 @@ var file_index_cgo_msg_proto_goTypes = []interface{}{
nil, // 13: milvus.proto.indexcgo.MapParamsV2.ParamsEntry
(*commonpb.KeyValuePair)(nil), // 14: milvus.proto.common.KeyValuePair
(*schemapb.FieldSchema)(nil), // 15: milvus.proto.schema.FieldSchema
(commonpb.LoadPriority)(0), // 16: milvus.proto.common.LoadPriority
}
var file_index_cgo_msg_proto_depIdxs = []int32{
14, // 0: milvus.proto.indexcgo.TypeParams.params:type_name -> milvus.proto.common.KeyValuePair
@ -1318,12 +1344,14 @@ var file_index_cgo_msg_proto_depIdxs = []int32{
9, // 10: milvus.proto.indexcgo.BuildIndexInfo.opt_fields:type_name -> milvus.proto.indexcgo.OptionalFieldInfo
7, // 11: milvus.proto.indexcgo.BuildIndexInfo.segment_insert_files:type_name -> milvus.proto.indexcgo.SegmentInsertFiles
15, // 12: milvus.proto.indexcgo.LoadTextIndexInfo.schema:type_name -> milvus.proto.schema.FieldSchema
15, // 13: milvus.proto.indexcgo.LoadJsonKeyIndexInfo.schema:type_name -> milvus.proto.schema.FieldSchema
14, // [14:14] is the sub-list for method output_type
14, // [14:14] is the sub-list for method input_type
14, // [14:14] is the sub-list for extension type_name
14, // [14:14] is the sub-list for extension extendee
0, // [0:14] is the sub-list for field type_name
16, // 13: milvus.proto.indexcgo.LoadTextIndexInfo.load_priority:type_name -> milvus.proto.common.LoadPriority
15, // 14: milvus.proto.indexcgo.LoadJsonKeyIndexInfo.schema:type_name -> milvus.proto.schema.FieldSchema
16, // 15: milvus.proto.indexcgo.LoadJsonKeyIndexInfo.load_priority:type_name -> milvus.proto.common.LoadPriority
16, // [16:16] is the sub-list for method output_type
16, // [16:16] is the sub-list for method input_type
16, // [16:16] is the sub-list for extension type_name
16, // [16:16] is the sub-list for extension extendee
0, // [0:16] is the sub-list for field type_name
}
func init() { file_index_cgo_msg_proto_init() }

View File

@ -217,6 +217,7 @@ message LoadCollectionRequest {
// resource group names
repeated string resource_groups = 8;
repeated int64 load_fields = 9;
common.LoadPriority priority = 10;
}
message ReleaseCollectionRequest {
@ -248,6 +249,7 @@ message LoadPartitionsRequest {
repeated string resource_groups = 9;
repeated index.IndexInfo index_info_list = 10;
repeated int64 load_fields = 11;
common.LoadPriority priority = 12;
}
message ReleasePartitionsRequest {
@ -373,6 +375,7 @@ message SegmentLoadInfo {
map<int64, data.TextIndexStats> textStatsLogs = 20;
repeated data.FieldBinlog bm25logs = 21;
map<int64, data.JsonKeyStats> jsonKeyStatsLogs = 22;
common.LoadPriority priority = 23;
}
message FieldIndexInfo {

File diff suppressed because it is too large Load Diff