mirror of
https://gitee.com/milvus-io/milvus.git
synced 2024-11-30 02:48:45 +08:00
enhance: Allows proactive warming up of chunk cache (#30182)
Allows proactive warming up of chunk cache. Original vector data will be asynchronously loaded into the chunk cache during the load process. It has the potential to significantly reduce query/search latency for a certain duration after the load, albeit with a concurrent increase in disk usage. issue: https://github.com/milvus-io/milvus/issues/30181 --------- Signed-off-by: bigsheeper <yihao.dai@zilliz.com>
This commit is contained in:
parent
f2c0ead51a
commit
c02fb64ad6
@ -303,6 +303,16 @@ queryNode:
|
||||
enabled: true # deprecated, TODO: remove it
|
||||
memoryLimit: 2147483648 # 2 GB, 2 * 1024 *1024 *1024 # deprecated, TODO: remove it
|
||||
readAheadPolicy: willneed # The read ahead policy of chunk cache, options: `normal, random, sequential, willneed, dontneed`
|
||||
# Specifies the necessity for warming up the chunk cache.
|
||||
# 1. If set to "sync" or "async," the original vector data
|
||||
# will be synchronously/asynchronously loaded into the
|
||||
# chunk cache during the load process. This approach has
|
||||
# the potential to substantially reduce query/search latency
|
||||
# for a specific duration post-load, albeit accompanied
|
||||
# by a concurrent increase in disk usage;
|
||||
# 2. If set to "off," original vector data will only
|
||||
# be loaded into the chunk cache during search/query.
|
||||
warmup: async # options: `sync, async, off`
|
||||
grouping:
|
||||
enabled: true
|
||||
maxNQ: 1000
|
||||
|
@ -40,6 +40,8 @@ class SegmentSealed : public SegmentInternalInterface {
|
||||
MapFieldData(const FieldId field_id, FieldDataInfo& data) = 0;
|
||||
virtual void
|
||||
AddFieldDataInfoForSealed(const LoadFieldDataInfo& field_data_info) = 0;
|
||||
virtual void
|
||||
WarmupChunkCache(const FieldId field_id) = 0;
|
||||
|
||||
SegmentType
|
||||
type() const override {
|
||||
|
@ -119,6 +119,36 @@ SegmentSealedImpl::LoadVecIndex(const LoadIndexInfo& info) {
|
||||
set_bit(index_ready_bitset_, field_id, true);
|
||||
}
|
||||
|
||||
void
|
||||
SegmentSealedImpl::WarmupChunkCache(const FieldId field_id) {
|
||||
auto& field_meta = schema_->operator[](field_id);
|
||||
AssertInfo(field_meta.is_vector(), "vector field is not vector type");
|
||||
|
||||
if (!get_bit(index_ready_bitset_, field_id) &&
|
||||
!get_bit(binlog_index_bitset_, field_id)) {
|
||||
return;
|
||||
}
|
||||
|
||||
AssertInfo(vector_indexings_.is_ready(field_id),
|
||||
"vector index is not ready");
|
||||
auto field_indexing = vector_indexings_.get_field_indexing(field_id);
|
||||
auto vec_index =
|
||||
dynamic_cast<index::VectorIndex*>(field_indexing->indexing_.get());
|
||||
AssertInfo(vec_index, "invalid vector indexing");
|
||||
|
||||
auto it = field_data_info_.field_infos.find(field_id.get());
|
||||
AssertInfo(it != field_data_info_.field_infos.end(),
|
||||
"cannot find binlog file for field: {}, seg: {}",
|
||||
field_id.get(),
|
||||
id_);
|
||||
auto field_info = it->second;
|
||||
|
||||
auto cc = storage::ChunkCacheSingleton::GetInstance().GetChunkCache();
|
||||
for (const auto& data_path : field_info.insert_files) {
|
||||
auto column = cc->Read(data_path);
|
||||
}
|
||||
}
|
||||
|
||||
void
|
||||
SegmentSealedImpl::LoadScalarIndex(const LoadIndexInfo& info) {
|
||||
// NOTE: lock only when data is ready to avoid starvation
|
||||
@ -869,10 +899,12 @@ SegmentSealedImpl::get_vector(FieldId field_id,
|
||||
AssertInfo(path_to_column.count(data_path) != 0,
|
||||
"column not found");
|
||||
const auto& column = path_to_column.at(data_path);
|
||||
AssertInfo(offset_in_binlog * row_bytes < column->ByteSize(),
|
||||
fmt::format("column idx out of range, idx: {}, size: {}",
|
||||
offset_in_binlog * row_bytes,
|
||||
column->ByteSize()));
|
||||
AssertInfo(
|
||||
offset_in_binlog * row_bytes < column->ByteSize(),
|
||||
"column idx out of range, idx: {}, size: {}, data_path: {}",
|
||||
offset_in_binlog * row_bytes,
|
||||
column->ByteSize(),
|
||||
data_path);
|
||||
auto vector = &column->Data()[offset_in_binlog * row_bytes];
|
||||
std::memcpy(buf.data() + i * row_bytes, vector, row_bytes);
|
||||
}
|
||||
|
@ -261,6 +261,9 @@ class SegmentSealedImpl : public SegmentSealed {
|
||||
void
|
||||
LoadScalarIndex(const LoadIndexInfo& info);
|
||||
|
||||
void
|
||||
WarmupChunkCache(const FieldId field_id) override;
|
||||
|
||||
bool
|
||||
generate_binlog_index(const FieldId field_id);
|
||||
|
||||
|
@ -410,3 +410,18 @@ AddFieldDataInfoForSealed(CSegmentInterface c_segment,
|
||||
return milvus::FailureCStatus(milvus::UnexpectedError, e.what());
|
||||
}
|
||||
}
|
||||
|
||||
CStatus
|
||||
WarmupChunkCache(CSegmentInterface c_segment, int64_t field_id) {
|
||||
try {
|
||||
auto segment_interface =
|
||||
reinterpret_cast<milvus::segcore::SegmentInterface*>(c_segment);
|
||||
auto segment =
|
||||
dynamic_cast<milvus::segcore::SegmentSealed*>(segment_interface);
|
||||
AssertInfo(segment != nullptr, "segment conversion failed");
|
||||
segment->WarmupChunkCache(milvus::FieldId(field_id));
|
||||
return milvus::SuccessCStatus();
|
||||
} catch (std::exception& e) {
|
||||
return milvus::FailureCStatus(milvus::UnexpectedError, e.what());
|
||||
}
|
||||
}
|
||||
|
@ -126,6 +126,9 @@ CStatus
|
||||
AddFieldDataInfoForSealed(CSegmentInterface c_segment,
|
||||
CLoadFieldDataInfo c_load_field_data_info);
|
||||
|
||||
CStatus
|
||||
WarmupChunkCache(CSegmentInterface c_segment, int64_t field_id);
|
||||
|
||||
////////////////////////////// interfaces for SegmentInterface //////////////////////////////
|
||||
CStatus
|
||||
ExistPk(CSegmentInterface c_segment,
|
||||
|
@ -22,23 +22,24 @@ std::shared_ptr<ColumnBase>
|
||||
ChunkCache::Read(const std::string& filepath) {
|
||||
auto path = std::filesystem::path(path_prefix_) / filepath;
|
||||
|
||||
ColumnTable::const_accessor ca;
|
||||
if (columns_.find(ca, path)) {
|
||||
return ca->second;
|
||||
{
|
||||
std::shared_lock lck(mutex_);
|
||||
auto it = columns_.find(path);
|
||||
if (it != columns_.end()) {
|
||||
AssertInfo(it->second, "unexpected null column, file={}", filepath);
|
||||
return it->second;
|
||||
}
|
||||
}
|
||||
ca.release();
|
||||
|
||||
auto field_data = DownloadAndDecodeRemoteFile(cm_.get(), filepath);
|
||||
auto column = Mmap(path, field_data->GetFieldData());
|
||||
auto ok =
|
||||
madvise(reinterpret_cast<void*>(const_cast<char*>(column->Data())),
|
||||
column->ByteSize(),
|
||||
read_ahead_policy_);
|
||||
AssertInfo(ok == 0,
|
||||
fmt::format("failed to madvise to the data file {}, err: {}",
|
||||
path.c_str(),
|
||||
strerror(errno)));
|
||||
|
||||
std::unique_lock lck(mutex_);
|
||||
auto it = columns_.find(path);
|
||||
if (it != columns_.end()) {
|
||||
return it->second;
|
||||
}
|
||||
auto column = Mmap(path, field_data->GetFieldData());
|
||||
AssertInfo(column, "unexpected null column, file={}", filepath);
|
||||
columns_.emplace(path, column);
|
||||
return column;
|
||||
}
|
||||
@ -46,32 +47,34 @@ ChunkCache::Read(const std::string& filepath) {
|
||||
void
|
||||
ChunkCache::Remove(const std::string& filepath) {
|
||||
auto path = std::filesystem::path(path_prefix_) / filepath;
|
||||
std::unique_lock lck(mutex_);
|
||||
columns_.erase(path);
|
||||
}
|
||||
|
||||
void
|
||||
ChunkCache::Prefetch(const std::string& filepath) {
|
||||
auto path = std::filesystem::path(path_prefix_) / filepath;
|
||||
ColumnTable::const_accessor ca;
|
||||
if (!columns_.find(ca, path)) {
|
||||
|
||||
std::shared_lock lck(mutex_);
|
||||
auto it = columns_.find(path);
|
||||
if (it == columns_.end()) {
|
||||
return;
|
||||
}
|
||||
auto column = ca->second;
|
||||
|
||||
auto column = it->second;
|
||||
auto ok =
|
||||
madvise(reinterpret_cast<void*>(const_cast<char*>(column->Data())),
|
||||
column->ByteSize(),
|
||||
read_ahead_policy_);
|
||||
AssertInfo(ok == 0,
|
||||
fmt::format("failed to madvise to the data file {}, err: {}",
|
||||
path.c_str(),
|
||||
strerror(errno)));
|
||||
"failed to madvise to the data file {}, err: {}",
|
||||
path.c_str(),
|
||||
strerror(errno));
|
||||
}
|
||||
|
||||
std::shared_ptr<ColumnBase>
|
||||
ChunkCache::Mmap(const std::filesystem::path& path,
|
||||
const FieldDataPtr& field_data) {
|
||||
std::unique_lock lck(mutex_);
|
||||
|
||||
auto dir = path.parent_path();
|
||||
std::filesystem::create_directories(dir);
|
||||
|
||||
@ -86,17 +89,18 @@ ChunkCache::Mmap(const std::filesystem::path& path,
|
||||
std::vector<std::vector<uint64_t>> element_indices{};
|
||||
auto written = WriteFieldData(file, data_type, field_data, element_indices);
|
||||
AssertInfo(written == data_size,
|
||||
fmt::format("failed to write data file {}, written "
|
||||
"{} but total {}, err: {}",
|
||||
path.c_str(),
|
||||
written,
|
||||
data_size,
|
||||
strerror(errno)));
|
||||
"failed to write data file {}, written "
|
||||
"{} but total {}, err: {}",
|
||||
path.c_str(),
|
||||
written,
|
||||
data_size,
|
||||
strerror(errno));
|
||||
|
||||
std::shared_ptr<ColumnBase> column{};
|
||||
|
||||
if (datatype_is_variable(data_type)) {
|
||||
AssertInfo(false, "TODO: unimplemented for variable data type");
|
||||
AssertInfo(
|
||||
false, "TODO: unimplemented for variable data type: {}", data_type);
|
||||
} else {
|
||||
column = std::make_shared<Column>(file, data_size, dim, data_type);
|
||||
}
|
||||
@ -104,9 +108,9 @@ ChunkCache::Mmap(const std::filesystem::path& path,
|
||||
// unlink
|
||||
auto ok = unlink(path.c_str());
|
||||
AssertInfo(ok == 0,
|
||||
fmt::format("failed to unlink mmap data file {}, err: {}",
|
||||
path.c_str(),
|
||||
strerror(errno)));
|
||||
"failed to unlink mmap data file {}, err: {}",
|
||||
path.c_str(),
|
||||
strerror(errno));
|
||||
|
||||
return column;
|
||||
}
|
||||
|
@ -16,7 +16,6 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include <oneapi/tbb/concurrent_hash_map.h>
|
||||
#include "mmap/Column.h"
|
||||
|
||||
namespace milvus::storage {
|
||||
@ -59,13 +58,12 @@ class ChunkCache {
|
||||
|
||||
private:
|
||||
using ColumnTable =
|
||||
oneapi::tbb::concurrent_hash_map<std::string,
|
||||
std::shared_ptr<ColumnBase>>;
|
||||
std::unordered_map<std::string, std::shared_ptr<ColumnBase>>;
|
||||
|
||||
private:
|
||||
mutable std::mutex mutex_;
|
||||
mutable std::shared_mutex mutex_;
|
||||
int read_ahead_policy_;
|
||||
std::string path_prefix_;
|
||||
const std::string path_prefix_;
|
||||
ChunkManagerPtr cm_;
|
||||
ColumnTable columns_;
|
||||
};
|
||||
|
@ -1285,6 +1285,119 @@ TEST(Sealed, GetVectorFromChunkCache) {
|
||||
Assert(!exist);
|
||||
}
|
||||
|
||||
TEST(Sealed, WarmupChunkCache) {
|
||||
// skip test due to mem leak from AWS::InitSDK
|
||||
return;
|
||||
|
||||
auto dim = 16;
|
||||
auto topK = 5;
|
||||
auto N = ROW_COUNT;
|
||||
auto metric_type = knowhere::metric::L2;
|
||||
auto index_type = knowhere::IndexEnum::INDEX_FAISS_IVFPQ;
|
||||
|
||||
auto mmap_dir = "/tmp/mmap";
|
||||
auto file_name = std::string(
|
||||
"sealed_test_get_vector_from_chunk_cache/insert_log/1/101/1000000");
|
||||
|
||||
auto sc = milvus::storage::StorageConfig{};
|
||||
milvus::storage::RemoteChunkManagerSingleton::GetInstance().Init(sc);
|
||||
auto mcm = std::make_unique<milvus::storage::MinioChunkManager>(sc);
|
||||
// mcm->CreateBucket(sc.bucket_name);
|
||||
milvus::storage::ChunkCacheSingleton::GetInstance().Init(mmap_dir,
|
||||
"willneed");
|
||||
|
||||
auto schema = std::make_shared<Schema>();
|
||||
auto fakevec_id = schema->AddDebugField(
|
||||
"fakevec", DataType::VECTOR_FLOAT, dim, metric_type);
|
||||
auto counter_id = schema->AddDebugField("counter", DataType::INT64);
|
||||
auto double_id = schema->AddDebugField("double", DataType::DOUBLE);
|
||||
auto nothing_id = schema->AddDebugField("nothing", DataType::INT32);
|
||||
auto str_id = schema->AddDebugField("str", DataType::VARCHAR);
|
||||
schema->AddDebugField("int8", DataType::INT8);
|
||||
schema->AddDebugField("int16", DataType::INT16);
|
||||
schema->AddDebugField("float", DataType::FLOAT);
|
||||
schema->set_primary_field_id(counter_id);
|
||||
|
||||
auto dataset = DataGen(schema, N);
|
||||
auto field_data_meta =
|
||||
milvus::storage::FieldDataMeta{1, 2, 3, fakevec_id.get()};
|
||||
auto field_meta = milvus::FieldMeta(milvus::FieldName("facevec"),
|
||||
fakevec_id,
|
||||
milvus::DataType::VECTOR_FLOAT,
|
||||
dim,
|
||||
metric_type);
|
||||
|
||||
auto rcm = milvus::storage::RemoteChunkManagerSingleton::GetInstance()
|
||||
.GetRemoteChunkManager();
|
||||
auto data = dataset.get_col<float>(fakevec_id);
|
||||
auto data_slices = std::vector<const uint8_t*>{(uint8_t*)data.data()};
|
||||
auto slice_sizes = std::vector<int64_t>{static_cast<int64_t>(N)};
|
||||
auto slice_names = std::vector<std::string>{file_name};
|
||||
PutFieldData(rcm.get(),
|
||||
data_slices,
|
||||
slice_sizes,
|
||||
slice_names,
|
||||
field_data_meta,
|
||||
field_meta);
|
||||
|
||||
auto fakevec = dataset.get_col<float>(fakevec_id);
|
||||
auto conf = generate_build_conf(index_type, metric_type);
|
||||
auto ds = knowhere::GenDataSet(N, dim, fakevec.data());
|
||||
auto indexing = std::make_unique<index::VectorMemIndex<float>>(
|
||||
index_type,
|
||||
metric_type,
|
||||
knowhere::Version::GetCurrentVersion().VersionNumber());
|
||||
indexing->BuildWithDataset(ds, conf);
|
||||
auto segment_sealed = CreateSealedSegment(schema);
|
||||
|
||||
LoadIndexInfo vec_info;
|
||||
vec_info.field_id = fakevec_id.get();
|
||||
vec_info.index = std::move(indexing);
|
||||
vec_info.index_params["metric_type"] = knowhere::metric::L2;
|
||||
segment_sealed->LoadIndex(vec_info);
|
||||
|
||||
auto field_binlog_info =
|
||||
FieldBinlogInfo{fakevec_id.get(),
|
||||
N,
|
||||
std::vector<int64_t>{N},
|
||||
false,
|
||||
std::vector<std::string>{file_name}};
|
||||
segment_sealed->AddFieldDataInfoForSealed(LoadFieldDataInfo{
|
||||
std::map<int64_t, FieldBinlogInfo>{
|
||||
{fakevec_id.get(), field_binlog_info}},
|
||||
mmap_dir,
|
||||
});
|
||||
|
||||
auto segment = dynamic_cast<SegmentSealedImpl*>(segment_sealed.get());
|
||||
auto has = segment->HasRawData(vec_info.field_id);
|
||||
EXPECT_FALSE(has);
|
||||
|
||||
segment_sealed->WarmupChunkCache(FieldId(vec_info.field_id));
|
||||
|
||||
auto ids_ds = GenRandomIds(N);
|
||||
auto result =
|
||||
segment->get_vector(fakevec_id, ids_ds->GetIds(), ids_ds->GetRows());
|
||||
|
||||
auto vector = result.get()->mutable_vectors()->float_vector().data();
|
||||
EXPECT_TRUE(vector.size() == fakevec.size());
|
||||
for (size_t i = 0; i < N; ++i) {
|
||||
auto id = ids_ds->GetIds()[i];
|
||||
for (size_t j = 0; j < dim; ++j) {
|
||||
auto expect = fakevec[id * dim + j];
|
||||
auto actual = vector[i * dim + j];
|
||||
AssertInfo(expect == actual,
|
||||
fmt::format("expect {}, actual {}", expect, actual));
|
||||
}
|
||||
}
|
||||
|
||||
rcm->Remove(file_name);
|
||||
std::filesystem::remove_all(mmap_dir);
|
||||
auto exist = rcm->Exist(file_name);
|
||||
Assert(!exist);
|
||||
exist = std::filesystem::exists(mmap_dir);
|
||||
Assert(!exist);
|
||||
}
|
||||
|
||||
TEST(Sealed, LoadArrayFieldData) {
|
||||
auto dim = 16;
|
||||
auto topK = 5;
|
||||
|
@ -30,6 +30,7 @@ import (
|
||||
"fmt"
|
||||
"io"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
"unsafe"
|
||||
|
||||
@ -1115,7 +1116,16 @@ func (s *LocalSegment) LoadIndex(ctx context.Context, indexInfo *querypb.FieldIn
|
||||
return errors.New(errMsg)
|
||||
}
|
||||
|
||||
return s.LoadIndexInfo(ctx, indexInfo, loadIndexInfo)
|
||||
err = s.LoadIndexInfo(ctx, indexInfo, loadIndexInfo)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if !typeutil.IsVectorType(fieldType) || s.HasRawData(indexInfo.GetFieldID()) {
|
||||
return nil
|
||||
}
|
||||
s.WarmupChunkCache(ctx, indexInfo.GetFieldID())
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *LocalSegment) LoadIndexInfo(ctx context.Context, indexInfo *querypb.FieldIndexInfo, info *LoadIndexInfo) error {
|
||||
@ -1153,10 +1163,59 @@ func (s *LocalSegment) LoadIndexInfo(ctx context.Context, indexInfo *querypb.Fie
|
||||
IndexInfo: indexInfo,
|
||||
})
|
||||
log.Info("updateSegmentIndex done")
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *LocalSegment) WarmupChunkCache(ctx context.Context, fieldID int64) {
|
||||
log := log.Ctx(ctx).With(
|
||||
zap.Int64("collectionID", s.Collection()),
|
||||
zap.Int64("partitionID", s.Partition()),
|
||||
zap.Int64("segmentID", s.ID()),
|
||||
zap.Int64("fieldID", fieldID),
|
||||
)
|
||||
s.ptrLock.RLock()
|
||||
defer s.ptrLock.RUnlock()
|
||||
|
||||
if s.ptr == nil {
|
||||
return
|
||||
}
|
||||
|
||||
var status C.CStatus
|
||||
|
||||
warmingUp := strings.ToLower(paramtable.Get().QueryNodeCfg.ChunkCacheWarmingUp.GetValue())
|
||||
switch warmingUp {
|
||||
case "sync":
|
||||
GetLoadPool().Submit(func() (any, error) {
|
||||
cFieldID := C.int64_t(fieldID)
|
||||
status = C.WarmupChunkCache(s.ptr, cFieldID)
|
||||
if err := HandleCStatus(ctx, &status, "warming up chunk cache failed"); err != nil {
|
||||
log.Warn("warming up chunk cache synchronously failed", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
log.Info("warming up chunk cache synchronously done")
|
||||
return nil, nil
|
||||
}).Await()
|
||||
case "async":
|
||||
GetLoadPool().Submit(func() (any, error) {
|
||||
s.ptrLock.RLock()
|
||||
defer s.ptrLock.RUnlock()
|
||||
if s.ptr == nil {
|
||||
return nil, nil
|
||||
}
|
||||
cFieldID := C.int64_t(fieldID)
|
||||
status = C.WarmupChunkCache(s.ptr, cFieldID)
|
||||
if err := HandleCStatus(ctx, &status, ""); err != nil {
|
||||
log.Warn("warming up chunk cache asynchronously failed", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
log.Info("warming up chunk cache asynchronously done")
|
||||
return nil, nil
|
||||
})
|
||||
default:
|
||||
// no warming up
|
||||
}
|
||||
}
|
||||
|
||||
func (s *LocalSegment) UpdateFieldRawDataSize(ctx context.Context, numRows int64, fieldBinlog *datapb.FieldBinlog) error {
|
||||
var status C.CStatus
|
||||
fieldID := fieldBinlog.FieldID
|
||||
|
@ -395,6 +395,10 @@ func (loader *segmentLoaderV2) loadSegment(ctx context.Context,
|
||||
}
|
||||
}
|
||||
|
||||
if err := segment.AddFieldDataInfo(ctx, loadInfo.GetNumOfRows(), loadInfo.GetBinlogPaths()); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
log.Info("load fields...",
|
||||
zap.Int64s("indexedFields", lo.Keys(indexedFieldInfos)),
|
||||
)
|
||||
@ -417,9 +421,6 @@ func (loader *segmentLoaderV2) loadSegment(ctx context.Context,
|
||||
if err := loader.loadSealedSegmentFields(ctx, segment, fieldsMap, loadInfo.GetNumOfRows()); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := segment.AddFieldDataInfo(ctx, loadInfo.GetNumOfRows(), loadInfo.GetBinlogPaths()); err != nil {
|
||||
return err
|
||||
}
|
||||
// https://github.com/milvus-io/milvus/23654
|
||||
// legacy entry num = 0
|
||||
if err := loader.patchEntryNumber(ctx, segment, loadInfo); err != nil {
|
||||
@ -956,6 +957,10 @@ func (loader *segmentLoader) loadSegment(ctx context.Context,
|
||||
|
||||
schemaHelper, _ := typeutil.CreateSchemaHelper(collection.Schema())
|
||||
|
||||
if err := segment.AddFieldDataInfo(ctx, loadInfo.GetNumOfRows(), loadInfo.GetBinlogPaths()); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
log.Info("load fields...",
|
||||
zap.Int64s("indexedFields", lo.Keys(indexedFieldInfos)),
|
||||
)
|
||||
@ -978,9 +983,6 @@ func (loader *segmentLoader) loadSegment(ctx context.Context,
|
||||
if err := loader.loadSealedSegmentFields(ctx, segment, fieldBinlogs, loadInfo.GetNumOfRows()); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := segment.AddFieldDataInfo(ctx, loadInfo.GetNumOfRows(), loadInfo.GetBinlogPaths()); err != nil {
|
||||
return err
|
||||
}
|
||||
// https://github.com/milvus-io/milvus/23654
|
||||
// legacy entry num = 0
|
||||
if err := loader.patchEntryNumber(ctx, segment, loadInfo); err != nil {
|
||||
|
@ -1804,7 +1804,8 @@ type queryNodeConfig struct {
|
||||
MmapDirPath ParamItem `refreshable:"false"`
|
||||
|
||||
// chunk cache
|
||||
ReadAheadPolicy ParamItem `refreshable:"false"`
|
||||
ReadAheadPolicy ParamItem `refreshable:"false"`
|
||||
ChunkCacheWarmingUp ParamItem `refreshable:"true"`
|
||||
|
||||
GroupEnabled ParamItem `refreshable:"true"`
|
||||
MaxReceiveChanSize ParamItem `refreshable:"false"`
|
||||
@ -2013,6 +2014,19 @@ func (p *queryNodeConfig) init(base *BaseTable) {
|
||||
}
|
||||
p.ReadAheadPolicy.Init(base.mgr)
|
||||
|
||||
p.ChunkCacheWarmingUp = ParamItem{
|
||||
Key: "queryNode.cache.warmup",
|
||||
Version: "2.3.6",
|
||||
DefaultValue: "async",
|
||||
Doc: `options: async, sync, off.
|
||||
Specifies the necessity for warming up the chunk cache.
|
||||
1. If set to "sync" or "async," the original vector data will be synchronously/asynchronously loaded into the
|
||||
chunk cache during the load process. This approach has the potential to substantially reduce query/search latency
|
||||
for a specific duration post-load, albeit accompanied by a concurrent increase in disk usage;
|
||||
2. If set to "off," original vector data will only be loaded into the chunk cache during search/query.`,
|
||||
}
|
||||
p.ChunkCacheWarmingUp.Init(base.mgr)
|
||||
|
||||
p.GroupEnabled = ParamItem{
|
||||
Key: "queryNode.grouping.enabled",
|
||||
Version: "2.0.0",
|
||||
|
@ -316,6 +316,7 @@ func TestComponentParam(t *testing.T) {
|
||||
|
||||
// chunk cache
|
||||
assert.Equal(t, "willneed", Params.ReadAheadPolicy.GetValue())
|
||||
assert.Equal(t, "async", Params.ChunkCacheWarmingUp.GetValue())
|
||||
|
||||
// test small indexNlist/NProbe default
|
||||
params.Remove("queryNode.segcore.smallIndex.nlist")
|
||||
|
Loading…
Reference in New Issue
Block a user