feat: cachinglayer: various improvements (#41546)

issue: https://github.com/milvus-io/milvus/issues/41435

this PR is based on https://github.com/milvus-io/milvus/pull/41436. 

Improvements include:

- Lazy Load support for Storage v1
- Use Low/High watermark to control eviction
- Caching Layer related config changes
- Removed ChunkCache related configs and code in golang
- Add `PinAllCells` helper method to CacheSlot class
- Modified ValueAt, RawAt, PrimitiveRawAt to Bulk version, to reduce
caching layer overhead
- Removed some unclear templated bulk_subscript methods
- CachedSearchIterator to store PinWrapper when searching on
ChunkedColumn, and removed unused contrustor.

---------

Signed-off-by: Buqian Zheng <zhengbuqian@gmail.com>
This commit is contained in:
Buqian Zheng 2025-05-10 09:19:16 +08:00 committed by GitHub
parent ee659d50db
commit ff5c2770e5
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
63 changed files with 1684 additions and 1010 deletions

View File

@ -430,9 +430,31 @@ queryNode:
buildParallelRate: 0.5 # the ratio of building interim index parallel matched with cpu num
multipleChunkedEnable: true # Deprecated. Enable multiple chunked search
tieredStorage:
enableGlobally: true # Whether or not to turn on Tiered Storage globally in this cluster.
memoryAllocationRatio: 0.8 # The ratio of memory allocation for Tiered Storage.
diskAllocationRatio: 0.8 # The ratio of disk allocation for Tiered Storage.
warmup:
# options: sync, async, disable.
# Specifies the timing for warming up the Tiered Storage cache.
# - "sync": data will be loaded into the cache before a segment is considered loaded.
# - "async": data will be loaded asynchronously into the cache after a segment is loaded.
# - "disable": data will not be proactively loaded into the cache, and loaded only if needed by search/query tasks.
# Defaults to "sync", except for vector field which defaults to "disable".
scalarField: sync
scalarIndex: sync
vectorField: disable # cache warmup for vector field raw data is by default disabled.
vectorIndex: sync
# If evictionEnabled is true, a background thread will run every evictionIntervalMs to determine if an
# eviction is necessary and the amount of data to evict from memory/disk.
# - The max ratio is the max amount of memory/disk that can be used for cache.
# - If the current memory/disk usage exceeds the high watermark, an eviction will be triggered to evict data from memory/disk
# until the memory/disk usage is below the low watermark.
memoryLowWatermarkRatio: 0.6
memoryHighWatermarkRatio: 0.8
memoryMaxRatio: 0.9
diskLowWatermarkRatio: 0.6
diskHighWatermarkRatio: 0.8
diskMaxRatio: 0.9
# Enable eviction for Tiered Storage. Defaults to false.
# Note that if eviction is enabled, cache data loaded during sync/async warmup is also subject to eviction.
evictionEnabled: false
knowhereScoreConsistency: false # Enable knowhere strong consistency score computation logic
jsonKeyStatsCommitInterval: 200 # the commit interval for the JSON key Stats to commit
loadMemoryUsageFactor: 1 # The multiply factor of calculating the memory usage while loading segments
@ -441,19 +463,11 @@ queryNode:
cache:
memoryLimit: 2147483648 # Deprecated: 2 GB, 2 * 1024 *1024 *1024
readAheadPolicy: willneed # The read ahead policy of chunk cache, options: `normal, random, sequential, willneed, dontneed`
# options: async, sync, disable.
# 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 "disable" original vector data will only be loaded into the chunk cache during search/query.
warmup: disable
mmap:
vectorField: false # Enable mmap for loading vector data
vectorIndex: false # Enable mmap for loading vector index
scalarField: false # Enable mmap for loading scalar data
scalarIndex: false # Enable mmap for loading scalar index
chunkCache: true # Enable mmap for chunk cache (raw vector retrieving).
# Enable memory mapping (mmap) to optimize the handling of growing raw data.
# By activating this feature, the memory overhead associated with newly added or modified data will be significantly minimized.
# However, this optimization may come at the cost of a slight decrease in query latency for the affected data segments.

View File

@ -26,8 +26,11 @@
#include "cachinglayer/lrucache/ListNode.h"
#include "cachinglayer/Translator.h"
#include "cachinglayer/Utils.h"
#include "common/EasyAssert.h"
#include "common/type_c.h"
#include "log/Log.h"
#include "monitor/prometheus_client.h"
#include "storage/ThreadPools.h"
namespace milvus::cachinglayer {
@ -68,6 +71,49 @@ class CacheSlot final : public std::enable_shared_from_this<CacheSlot<CellT>> {
CacheSlot&
operator=(CacheSlot&&) = delete;
void
Warmup() {
auto warmup_policy = translator_->meta()->cache_warmup_policy;
if (warmup_policy == CacheWarmupPolicy::CacheWarmupPolicy_Disable) {
return;
}
std::vector<cid_t> cids;
cids.reserve(translator_->num_cells());
for (cid_t i = 0; i < translator_->num_cells(); ++i) {
cids.push_back(i);
}
switch (warmup_policy) {
case CacheWarmupPolicy::CacheWarmupPolicy_Sync:
SemiInlineGet(PinCells(std::move(cids)));
break;
case CacheWarmupPolicy::CacheWarmupPolicy_Async:
auto& pool = milvus::ThreadPools::GetThreadPool(
milvus::ThreadPoolPriority::MIDDLE);
pool.Submit([this, cids = std::move(cids)]() mutable {
SemiInlineGet(PinCells(std::move(cids)));
});
break;
}
}
folly::SemiFuture<std::shared_ptr<CellAccessor<CellT>>>
PinAllCells() {
return folly::makeSemiFuture().deferValue([this](auto&&) {
size_t index = 0;
return PinInternal(
[this, index]() mutable -> std::pair<cid_t, bool> {
if (index >= cells_.size()) {
return std::make_pair(cells_.size(), true);
}
return std::make_pair(index++, false);
},
cells_.size());
});
}
folly::SemiFuture<std::shared_ptr<CellAccessor<CellT>>>
PinCells(std::vector<uid_t> uids) {
return folly::makeSemiFuture().deferValue([this,
@ -92,34 +138,31 @@ class CacheSlot final : public std::enable_shared_from_this<CacheSlot<CellT>> {
}
involved_cids.insert(cid);
}
auto reserve_size = involved_cids.size();
std::vector<folly::SemiFuture<internal::ListNode::NodePin>> futures;
std::unordered_set<cid_t> need_load_cids;
futures.reserve(involved_cids.size());
need_load_cids.reserve(involved_cids.size());
for (auto cid : involved_cids) {
auto [need_load, future] = cells_[cid].pin();
futures.push_back(std::move(future));
if (need_load) {
need_load_cids.insert(cid);
// must be captured by value.
// theoretically, we can initialize it outside, and it will not be invalidated
// even though we moved involved_cids afterwards, but for safety we initialize it
// inside the lambda.
decltype(involved_cids.begin()) it;
bool initialized = false;
return PinInternal(
[this,
cids = std::move(involved_cids),
it,
initialized]() mutable -> std::pair<cid_t, bool> {
if (!initialized) {
it = cids.begin();
initialized = true;
}
if (it == cids.end()) {
return std::make_pair(0, true);
}
auto load_future = folly::makeSemiFuture();
if (!need_load_cids.empty()) {
load_future = RunLoad(std::move(need_load_cids));
}
return std::move(load_future)
.deferValue(
[this, futures = std::move(futures)](auto&&) mutable
-> folly::SemiFuture<std::shared_ptr<CellAccessor<CellT>>> {
return folly::collect(futures).deferValue(
[this](std::vector<internal::ListNode::NodePin>&&
pins) mutable
-> std::shared_ptr<CellAccessor<CellT>> {
return std::make_shared<CellAccessor<CellT>>(
this->shared_from_this(), std::move(pins));
});
});
auto cid = *it++;
return std::make_pair(cid, false);
},
reserve_size);
});
}
@ -148,6 +191,40 @@ class CacheSlot final : public std::enable_shared_from_this<CacheSlot<CellT>> {
private:
friend class CellAccessor<CellT>;
template <typename Fn>
folly::SemiFuture<std::shared_ptr<CellAccessor<CellT>>>
PinInternal(Fn&& cid_iterator, size_t reserve_size) {
std::vector<folly::SemiFuture<internal::ListNode::NodePin>> futures;
std::unordered_set<cid_t> need_load_cids;
futures.reserve(reserve_size);
need_load_cids.reserve(reserve_size);
auto [cid, end] = cid_iterator();
while (!end) {
auto [need_load, future] = cells_[cid].pin();
futures.push_back(std::move(future));
if (need_load) {
need_load_cids.insert(cid);
}
std::tie(cid, end) = cid_iterator();
}
auto load_future = folly::makeSemiFuture();
if (!need_load_cids.empty()) {
load_future = RunLoad(std::move(need_load_cids));
}
return std::move(load_future)
.deferValue(
[this, futures = std::move(futures)](auto&&) mutable
-> folly::SemiFuture<std::shared_ptr<CellAccessor<CellT>>> {
return folly::collect(futures).deferValue(
[this](std::vector<internal::ListNode::NodePin>&&
pins) mutable
-> std::shared_ptr<CellAccessor<CellT>> {
return std::make_shared<CellAccessor<CellT>>(
this->shared_from_this(), std::move(pins));
});
});
}
cid_t
cell_id_of(uid_t uid) const {
return translator_->cell_id_of(uid);
@ -336,8 +413,8 @@ class PinWrapper {
PinWrapper&
operator=(PinWrapper&& other) noexcept {
if (this != &other) {
raii_ = std::move(other.raii_);
content_ = std::move(other.content_);
std::swap(raii_, other.raii_);
std::swap(content_, other.content_);
}
return *this;
}

View File

@ -23,34 +23,66 @@ Manager::GetInstance() {
return instance;
}
bool
Manager::ConfigureTieredStorage(bool enabled_globally,
int64_t memory_limit_bytes,
int64_t disk_limit_bytes) {
void
Manager::ConfigureTieredStorage(CacheWarmupPolicies warmup_policies,
CacheLimit cache_limit,
bool evictionEnabled,
EvictionConfig eviction_config) {
static std::once_flag once;
std::call_once(once, [&]() {
Manager& manager = GetInstance();
if (enabled_globally) {
if (manager.dlist_ != nullptr) {
return manager.dlist_->UpdateLimit(
{memory_limit_bytes, disk_limit_bytes});
} else {
ResourceUsage limit{memory_limit_bytes, disk_limit_bytes};
internal::DList::TouchConfig touch_config{std::chrono::seconds(10)};
manager.dlist_ =
std::make_unique<internal::DList>(limit, touch_config);
}
manager.warmup_policies_ = warmup_policies;
manager.cache_limit_ = cache_limit;
manager.evictionEnabled_ = evictionEnabled;
if (!evictionEnabled) {
LOG_INFO(
"Configured Tiered Storage manager with memory limit: {} bytes "
"({:.2f} GB), disk "
"limit: {} bytes ({:.2f} GB)",
memory_limit_bytes,
memory_limit_bytes / (1024.0 * 1024.0 * 1024.0),
disk_limit_bytes,
disk_limit_bytes / (1024.0 * 1024.0 * 1024.0));
} else {
manager.dlist_ = nullptr;
LOG_INFO("Tiered Storage is disabled");
"Tiered Storage manager is configured with disabled eviction");
return;
}
return true;
ResourceUsage max{cache_limit.memory_max_bytes,
cache_limit.disk_max_bytes};
ResourceUsage low_watermark{cache_limit.memory_low_watermark_bytes,
cache_limit.disk_low_watermark_bytes};
ResourceUsage high_watermark{cache_limit.memory_high_watermark_bytes,
cache_limit.disk_high_watermark_bytes};
AssertInfo(
low_watermark.GEZero(),
"Milvus Caching Layer: low watermark must be greater than 0");
AssertInfo((high_watermark - low_watermark).GEZero(),
"Milvus Caching Layer: high watermark must be greater than "
"low watermark");
AssertInfo(
(max - high_watermark).GEZero(),
"Milvus Caching Layer: max must be greater than high watermark");
manager.dlist_ = std::make_unique<internal::DList>(
max, low_watermark, high_watermark, eviction_config);
LOG_INFO(
"Configured Tiered Storage manager with memory watermark: low {} "
"bytes ({:.2} GB), high {} bytes ({:.2} GB), max {} bytes "
"({:.2} GB), disk watermark: low "
"{} bytes ({:.2} GB), high {} bytes ({:.2} GB), max {} bytes "
"({:.2} GB), cache touch "
"window: {} ms, eviction interval: {} ms",
low_watermark.memory_bytes,
low_watermark.memory_bytes / (1024.0 * 1024.0 * 1024.0),
high_watermark.memory_bytes,
high_watermark.memory_bytes / (1024.0 * 1024.0 * 1024.0),
max.memory_bytes,
max.memory_bytes / (1024.0 * 1024.0 * 1024.0),
low_watermark.file_bytes,
low_watermark.file_bytes / (1024.0 * 1024.0 * 1024.0),
high_watermark.file_bytes,
high_watermark.file_bytes / (1024.0 * 1024.0 * 1024.0),
max.file_bytes,
max.file_bytes / (1024.0 * 1024.0 * 1024.0),
eviction_config.cache_touch_window.count(),
eviction_config.eviction_interval.count());
});
}
size_t

View File

@ -14,8 +14,10 @@
#include <memory>
#include "cachinglayer/CacheSlot.h"
#include "cachinglayer/Translator.h"
#include "cachinglayer/lrucache/DList.h"
#include "cachinglayer/Translator.h"
#include "cachinglayer/Utils.h"
#include "common/type_c.h"
namespace milvus::cachinglayer {
@ -24,12 +26,14 @@ class Manager {
static Manager&
GetInstance();
// This function is not thread safe, must be called before any CacheSlot is created.
// This function is not thread safe, must be called exactly once before any CacheSlot is created,
// and before any Manager instance method is called.
// TODO(tiered storage 4): support dynamic update.
static bool
ConfigureTieredStorage(bool enabled_globally,
int64_t memory_limit_bytes,
int64_t disk_limit_bytes);
static void
ConfigureTieredStorage(CacheWarmupPolicies warmup_policies,
CacheLimit cache_limit,
bool evictionEnabled,
EvictionConfig eviction_config);
Manager(const Manager&) = delete;
Manager&
@ -43,24 +47,59 @@ class Manager {
template <typename CellT>
std::shared_ptr<CacheSlot<CellT>>
CreateCacheSlot(std::unique_ptr<Translator<CellT>> translator) {
return std::make_shared<CacheSlot<CellT>>(std::move(translator),
dlist_.get());
// if eviction is disabled, pass nullptr dlist to CacheSlot, so pinned cells
// in this CacheSlot will not be evicted.
auto dlist = (translator->meta()->support_eviction && evictionEnabled_)
? dlist_.get()
: nullptr;
auto cache_slot =
std::make_shared<CacheSlot<CellT>>(std::move(translator), dlist);
cache_slot->Warmup();
return cache_slot;
}
// memory overhead for managing all cache slots/cells/translators/policies.
size_t
memory_overhead() const;
CacheWarmupPolicy
getScalarFieldCacheWarmupPolicy() const {
return warmup_policies_.scalarFieldCacheWarmupPolicy;
}
CacheWarmupPolicy
getVectorFieldCacheWarmupPolicy() const {
return warmup_policies_.vectorFieldCacheWarmupPolicy;
}
CacheWarmupPolicy
getScalarIndexCacheWarmupPolicy() const {
return warmup_policies_.scalarIndexCacheWarmupPolicy;
}
CacheWarmupPolicy
getVectorIndexCacheWarmupPolicy() const {
return warmup_policies_.vectorIndexCacheWarmupPolicy;
}
bool
isEvictionEnabled() const {
return evictionEnabled_;
}
private:
friend void
ConfigureTieredStorage(bool enabled_globally,
int64_t memory_limit_bytes,
int64_t disk_limit_bytes);
ConfigureTieredStorage(CacheWarmupPolicies warmup_policies,
CacheLimit cache_limit,
bool evictionEnabled,
EvictionConfig eviction_config);
Manager() = default; // Private constructor
std::unique_ptr<internal::DList> dlist_{nullptr};
bool enable_global_tiered_storage_{false};
CacheWarmupPolicies warmup_policies_{};
bool evictionEnabled_{false};
CacheLimit cache_limit_{};
}; // class Manager
} // namespace milvus::cachinglayer

View File

@ -16,12 +16,22 @@
#include <utility>
#include "cachinglayer/Utils.h"
#include "common/type_c.h"
namespace milvus::cachinglayer {
struct Meta {
// This storage type is currently used only by metrics to distinguish the slot type.
// In actual resource reservation, we use the actual size of the cell to determine the type.
StorageType storage_type;
explicit Meta(StorageType storage_type) : storage_type(storage_type) {
CacheWarmupPolicy cache_warmup_policy;
bool support_eviction;
explicit Meta(StorageType storage_type,
CacheWarmupPolicy cache_warmup_policy,
bool support_eviction)
: storage_type(storage_type),
cache_warmup_policy(cache_warmup_policy),
support_eviction(support_eviction) {
}
};

View File

@ -106,16 +106,18 @@ struct ResourceUsage {
std::string
ToString() const {
return fmt::format("ResourceUsage{memory_bytes={}, file_bytes={}}",
return fmt::format(
"memory {} bytes ({:.2} GB), disk {} bytes ({:.2} GB)",
memory_bytes,
file_bytes);
memory_bytes / 1024.0 / 1024.0 / 1024.0,
file_bytes,
file_bytes / 1024.0 / 1024.0 / 1024.0);
}
};
inline std::ostream&
operator<<(std::ostream& os, const ResourceUsage& usage) {
os << "ResourceUsage{memory_bytes=" << usage.memory_bytes
<< ", file_bytes=" << usage.file_bytes << "}";
os << "memory=" << usage.memory_bytes << ", disk=" << usage.file_bytes;
return os;
}
@ -139,6 +141,82 @@ operator-=(std::atomic<ResourceUsage>& atomic_lhs, const ResourceUsage& rhs) {
} while (!atomic_lhs.compare_exchange_weak(current, new_value));
}
// helper struct for ConfigureTieredStorage, so the list of arguments is not too long.
struct CacheWarmupPolicies {
CacheWarmupPolicy scalarFieldCacheWarmupPolicy;
CacheWarmupPolicy vectorFieldCacheWarmupPolicy;
CacheWarmupPolicy scalarIndexCacheWarmupPolicy;
CacheWarmupPolicy vectorIndexCacheWarmupPolicy;
CacheWarmupPolicies()
: scalarFieldCacheWarmupPolicy(
CacheWarmupPolicy::CacheWarmupPolicy_Sync),
vectorFieldCacheWarmupPolicy(
CacheWarmupPolicy::CacheWarmupPolicy_Disable),
scalarIndexCacheWarmupPolicy(
CacheWarmupPolicy::CacheWarmupPolicy_Sync),
vectorIndexCacheWarmupPolicy(
CacheWarmupPolicy::CacheWarmupPolicy_Sync) {
}
CacheWarmupPolicies(CacheWarmupPolicy scalarFieldCacheWarmupPolicy,
CacheWarmupPolicy vectorFieldCacheWarmupPolicy,
CacheWarmupPolicy scalarIndexCacheWarmupPolicy,
CacheWarmupPolicy vectorIndexCacheWarmupPolicy)
: scalarFieldCacheWarmupPolicy(scalarFieldCacheWarmupPolicy),
vectorFieldCacheWarmupPolicy(vectorFieldCacheWarmupPolicy),
scalarIndexCacheWarmupPolicy(scalarIndexCacheWarmupPolicy),
vectorIndexCacheWarmupPolicy(vectorIndexCacheWarmupPolicy) {
}
};
struct CacheLimit {
int64_t memory_low_watermark_bytes;
int64_t memory_high_watermark_bytes;
int64_t memory_max_bytes;
int64_t disk_low_watermark_bytes;
int64_t disk_high_watermark_bytes;
int64_t disk_max_bytes;
CacheLimit()
: memory_low_watermark_bytes(0),
memory_high_watermark_bytes(0),
memory_max_bytes(0),
disk_low_watermark_bytes(0),
disk_high_watermark_bytes(0),
disk_max_bytes(0) {
}
CacheLimit(int64_t memory_low_watermark_bytes,
int64_t memory_high_watermark_bytes,
int64_t memory_max_bytes,
int64_t disk_low_watermark_bytes,
int64_t disk_high_watermark_bytes,
int64_t disk_max_bytes)
: memory_low_watermark_bytes(memory_low_watermark_bytes),
memory_high_watermark_bytes(memory_high_watermark_bytes),
memory_max_bytes(memory_max_bytes),
disk_low_watermark_bytes(disk_low_watermark_bytes),
disk_high_watermark_bytes(disk_high_watermark_bytes),
disk_max_bytes(disk_max_bytes) {
}
};
struct EvictionConfig {
// Touch a node means to move it to the head of the list, which requires locking the entire list.
// Use cache_touch_window_ms to reduce the frequency of touching and reduce contention.
std::chrono::milliseconds cache_touch_window;
std::chrono::milliseconds eviction_interval;
EvictionConfig()
: cache_touch_window(std::chrono::milliseconds(0)),
eviction_interval(std::chrono::milliseconds(0)) {
}
EvictionConfig(int64_t cache_touch_window_ms, int64_t eviction_interval_ms)
: cache_touch_window(std::chrono::milliseconds(cache_touch_window_ms)),
eviction_interval(std::chrono::milliseconds(eviction_interval_ms)) {
}
};
namespace internal {
inline prometheus::Gauge&

View File

@ -19,6 +19,7 @@
#include "cachinglayer/Utils.h"
#include "cachinglayer/lrucache/ListNode.h"
#include "monitor/prometheus_client.h"
#include "log/Log.h"
namespace milvus::cachinglayer::internal {
@ -30,15 +31,79 @@ DList::reserveMemory(const ResourceUsage& size) {
used_memory_ += size;
return true;
}
if (tryEvict(used + size - max_memory_)) {
// try to evict so that used + size <= low watermark, but if that is not possible,
// evict enough for the current reservation.
if (tryEvict(used + size - low_watermark_, used + size - max_memory_)) {
used_memory_ += size;
return true;
}
return false;
}
void
DList::evictionLoop() {
while (true) {
std::unique_lock<std::mutex> lock(list_mtx_);
if (eviction_thread_cv_.wait_for(
lock, eviction_config_.eviction_interval, [this] {
return stop_eviction_loop_.load();
})) {
break;
}
auto used = used_memory_.load();
// if usage is above high watermark, evict until low watermark is reached.
if (used.memory_bytes >= high_watermark_.memory_bytes ||
used.file_bytes >= high_watermark_.file_bytes) {
tryEvict(
{
used.memory_bytes >= high_watermark_.memory_bytes
? used.memory_bytes - low_watermark_.memory_bytes
: 0,
used.file_bytes >= high_watermark_.file_bytes
? used.file_bytes - low_watermark_.file_bytes
: 0,
},
// in eviction loop, we always evict as much as possible until low watermark.
{0, 0});
}
}
}
std::string
DList::usageInfo(const ResourceUsage& actively_pinned) const {
auto used = used_memory_.load();
static double precision = 100.0;
return fmt::format(
"low_watermark_: {}, "
"high_watermark_: {} , "
"max_memory_: {} , "
"used_memory_: {} {:.2}% of max, {:.2}% of "
"high_watermark memory, {:.2}% of max, {:.2}% of "
"high_watermark disk, "
"actively_pinned: {} {:.2}% of used memory, {:.2}% of used disk",
low_watermark_.ToString(),
high_watermark_.ToString(),
max_memory_.ToString(),
used.ToString(),
static_cast<double>(used.memory_bytes) / max_memory_.memory_bytes *
precision,
static_cast<double>(used.memory_bytes) / high_watermark_.memory_bytes *
precision,
static_cast<double>(used.file_bytes) / max_memory_.file_bytes *
precision,
static_cast<double>(used.file_bytes) / high_watermark_.file_bytes *
precision,
actively_pinned.ToString(),
static_cast<double>(actively_pinned.memory_bytes) / used.memory_bytes *
precision,
static_cast<double>(actively_pinned.file_bytes) / used.file_bytes *
precision);
}
bool
DList::tryEvict(const ResourceUsage& expected_eviction) {
DList::tryEvict(const ResourceUsage& expected_eviction,
const ResourceUsage& min_eviction) {
std::vector<ListNode*> to_evict;
// items are evicted because they are not used for a while, thus it should be ok to lock them
// a little bit longer.
@ -55,6 +120,9 @@ DList::tryEvict(const ResourceUsage& expected_eviction) {
(need_disk && size.file_bytes > 0);
};
ResourceUsage actively_pinned{0, 0};
// accumulate victims using expected_eviction.
for (auto it = tail_; it != nullptr; it = it->next_) {
if (!would_help(it->size())) {
continue;
@ -72,11 +140,26 @@ DList::tryEvict(const ResourceUsage& expected_eviction) {
// if we grabbed the lock only to find that the ListNode is pinned; or if we failed to lock
// the ListNode, we do not evict this ListNode.
item_locks.pop_back();
actively_pinned += it->size();
}
}
if (!size_to_evict.CanHold(expected_eviction)) {
if (!size_to_evict.CanHold(min_eviction)) {
LOG_WARN(
"Milvus Caching Layer: cannot evict even min_eviction {}, "
"giving up eviction. Current usage: {}",
min_eviction.ToString(),
usageInfo(actively_pinned));
return false;
}
LOG_INFO(
"Milvus Caching Layer: cannot evict expected_eviction {}, "
"evicting as much({}) as possible. Current usage: {}",
expected_eviction.ToString(),
size_to_evict.ToString(),
usageInfo(actively_pinned));
}
for (auto* list_node : to_evict) {
auto size = list_node->size();
internal::cache_eviction_count(size.storage_type()).Increment();
@ -108,17 +191,17 @@ DList::tryEvict(const ResourceUsage& expected_eviction) {
bool
DList::UpdateLimit(const ResourceUsage& new_limit) {
if (!new_limit.GEZero()) {
throw std::invalid_argument(
"Milvus Caching Layer: memory and disk usage limit must be greater "
"than 0");
}
AssertInfo(new_limit.GEZero(),
"Milvus Caching Layer: memory and disk usage limit must be "
"greater than 0");
std::unique_lock<std::mutex> list_lock(list_mtx_);
auto used = used_memory_.load();
if (!new_limit.CanHold(used)) {
// positive means amount owed
auto deficit = used - new_limit;
if (!tryEvict(deficit)) {
// deficit is the hard limit of eviction, if we cannot evict deficit, we give
// up the limit change.
if (!tryEvict(deficit, deficit)) {
return false;
}
}
@ -130,6 +213,22 @@ DList::UpdateLimit(const ResourceUsage& new_limit) {
return true;
}
void
DList::UpdateLowWatermark(const ResourceUsage& new_low_watermark) {
std::unique_lock<std::mutex> list_lock(list_mtx_);
AssertInfo(new_low_watermark.GEZero(),
"Milvus Caching Layer: low watermark must be greater than 0");
low_watermark_ = new_low_watermark;
}
void
DList::UpdateHighWatermark(const ResourceUsage& new_high_watermark) {
std::unique_lock<std::mutex> list_lock(list_mtx_);
AssertInfo(new_high_watermark.GEZero(),
"Milvus Caching Layer: high watermark must be greater than 0");
high_watermark_ = new_high_watermark;
}
void
DList::releaseMemory(const ResourceUsage& size) {
// safe to substract on atomic without lock

View File

@ -11,25 +11,36 @@
#pragma once
#include <atomic>
#include <condition_variable>
#include <mutex>
#include <folly/futures/Future.h>
#include <folly/futures/SharedPromise.h>
#include "cachinglayer/lrucache/ListNode.h"
#include "cachinglayer/Utils.h"
namespace milvus::cachinglayer::internal {
class DList {
public:
// Touch a node means to move it to the head of the list, which requires locking the entire list.
// Use TouchConfig to reduce the frequency of touching and reduce contention.
struct TouchConfig {
std::chrono::seconds refresh_window = std::chrono::seconds(10);
};
DList(ResourceUsage max_memory,
ResourceUsage low_watermark,
ResourceUsage high_watermark,
EvictionConfig eviction_config)
: max_memory_(max_memory),
low_watermark_(low_watermark),
high_watermark_(high_watermark),
eviction_config_(eviction_config) {
eviction_thread_ = std::thread(&DList::evictionLoop, this);
}
DList(ResourceUsage max_memory, TouchConfig touch_config)
: max_memory_(max_memory), touch_config_(touch_config) {
~DList() {
stop_eviction_loop_ = true;
eviction_thread_cv_.notify_all();
if (eviction_thread_.joinable()) {
eviction_thread_.join();
}
}
// If after evicting all unpinned items, the used_memory_ is still larger than new_limit, false will be returned
@ -38,6 +49,13 @@ class DList {
bool
UpdateLimit(const ResourceUsage& new_limit);
// Update low/high watermark does not trigger eviction, thus will not fail.
void
UpdateLowWatermark(const ResourceUsage& new_low_watermark);
void
UpdateHighWatermark(const ResourceUsage& new_high_watermark);
// True if no nodes in the list.
bool
IsEmpty() const;
@ -66,19 +84,24 @@ class DList {
void
removeItem(ListNode* list_node, ResourceUsage size);
const TouchConfig&
touch_config() const {
return touch_config_;
const EvictionConfig&
eviction_config() const {
return eviction_config_;
}
private:
friend class DListTestFriend;
// Try to evict some items so that the evicted items are larger than expected_eviction.
// If we cannot achieve the goal, nothing will be evicted and false will be returned.
void
evictionLoop();
// Try to evict some items so that the resources of evicted items are larger than expected_eviction.
// If we cannot achieve the goal, but we can evict min_eviction, we will still perform eviction.
// If we cannot even evict min_eviction, nothing will be evicted and false will be returned.
// Must be called under the lock of list_mtx_.
bool
tryEvict(const ResourceUsage& expected_eviction);
tryEvict(const ResourceUsage& expected_eviction,
const ResourceUsage& min_eviction);
// Must be called under the lock of list_mtx_ and list_node->mtx_.
// ListNode is guaranteed to be not in the list.
@ -91,6 +114,9 @@ class DList {
bool
popItem(ListNode* list_node);
std::string
usageInfo(const ResourceUsage& actively_pinned) const;
// head_ is the most recently used item, tail_ is the least recently used item.
// tail_ -> next -> ... -> head_
// tail_ <- prev <- ... <- head_
@ -101,8 +127,14 @@ class DList {
mutable std::mutex list_mtx_;
// access to used_memory_ and max_memory_ must be done under the lock of list_mtx_
std::atomic<ResourceUsage> used_memory_{};
ResourceUsage low_watermark_;
ResourceUsage high_watermark_;
ResourceUsage max_memory_;
const TouchConfig touch_config_;
const EvictionConfig eviction_config_;
std::thread eviction_thread_;
std::condition_variable eviction_thread_cv_;
std::atomic<bool> stop_eviction_loop_{false};
};
} // namespace milvus::cachinglayer::internal

View File

@ -50,7 +50,7 @@ ListNode::NodePin::operator=(NodePin&& other) {
ListNode::ListNode(DList* dlist, ResourceUsage size)
: last_touch_(dlist ? (std::chrono::high_resolution_clock::now() -
2 * dlist->touch_config().refresh_window)
2 * dlist->eviction_config().cache_touch_window)
: std::chrono::high_resolution_clock::now()),
dlist_(dlist),
size_(size),
@ -181,7 +181,8 @@ ListNode::unpin() {
void
ListNode::touch(bool update_used_memory) {
auto now = std::chrono::high_resolution_clock::now();
if (dlist_ && now - last_touch_ > dlist_->touch_config().refresh_window) {
if (dlist_ &&
now - last_touch_ > dlist_->eviction_config().cache_touch_window) {
std::optional<ResourceUsage> size = std::nullopt;
if (update_used_memory) {
size = size_;
@ -197,7 +198,7 @@ ListNode::clear_data() {
// the cell should be inserted into the cache again.
if (dlist_) {
last_touch_ = std::chrono::high_resolution_clock::now() -
2 * dlist_->touch_config().refresh_window;
2 * dlist_->eviction_config().cache_touch_window;
}
unload();
state_ = State::NOT_LOADED;

View File

@ -166,7 +166,7 @@ class StringChunk : public Chunk {
std::string_view
operator[](const int i) const {
if (i < 0 || i > row_nums_) {
if (i < 0 || i >= row_nums_) {
PanicInfo(ErrorCode::OutOfRange, "index out of range");
}

View File

@ -32,6 +32,14 @@ enum SegmentType {
typedef enum SegmentType SegmentType;
enum CacheWarmupPolicy {
CacheWarmupPolicy_Disable = 0,
CacheWarmupPolicy_Sync = 1,
CacheWarmupPolicy_Async = 2,
};
typedef enum CacheWarmupPolicy CacheWarmupPolicy;
// pure C don't support that we use schemapb.DataType directly.
// Note: the value of all enumerations must match the corresponding schemapb.DataType.
// TODO: what if there are increments in schemapb.DataType.

View File

@ -12,8 +12,9 @@
#pragma once
#include <unordered_map>
#include "mmap/ChunkedColumnGroup.h"
#include "common/Types.h"
#include "mmap/ChunkedColumnInterface.h"
namespace milvus {
@ -98,11 +99,10 @@ class SkipIndex {
const bool* valid_data,
int64_t count);
template <typename T>
void
LoadString(milvus::FieldId field_id,
int64_t chunk_id,
const T& var_column) {
const ChunkedColumnInterface& var_column) {
int num_rows = var_column.NumRows();
auto chunkMetrics = std::make_unique<FieldChunkMetrics>();
if (num_rows > 0) {
@ -289,52 +289,35 @@ class SkipIndex {
return {minValue, maxValue, null_count};
}
template <typename T>
metricInfo<std::string>
ProcessStringFieldMetrics(const T& var_column) {
int num_rows = var_column.NumRows();
// find first not null value
size_t start = 0;
for (size_t i = start; i < num_rows; i++) {
if (!var_column.IsValid(i)) {
start++;
continue;
}
break;
}
if (start > num_rows - 1) {
return {std::string(), std::string(), num_rows};
}
ProcessStringFieldMetrics(const ChunkedColumnInterface& var_column) {
// all captured by reference
bool has_first_valid = false;
std::string_view min_string;
std::string_view max_string;
int64_t null_count = 0;
// Handle both ChunkedVariableColumn and ProxyChunkColumn cases
std::string min_string;
if constexpr (std::is_same_v<T, ProxyChunkColumn>) {
min_string = var_column.template RawAt<std::string>(start);
} else {
min_string = var_column.RawAt(start);
}
std::string max_string = min_string;
int64_t null_count = start;
for (size_t i = start; i < num_rows; i++) {
if (!var_column.IsValid(i)) {
var_column.BulkRawStringAt(
[&](std::string_view value, size_t offset, bool is_valid) {
if (!is_valid) {
null_count++;
continue;
return;
}
std::string val;
if constexpr (std::is_same_v<T, ProxyChunkColumn>) {
val = var_column.template RawAt<std::string>(i);
if (!has_first_valid) {
min_string = value;
max_string = value;
has_first_valid = true;
} else {
val = var_column.RawAt(i);
if (value < min_string) {
min_string = value;
}
if (val < min_string) {
min_string = val;
}
if (val > max_string) {
max_string = val;
if (value > max_string) {
max_string = value;
}
}
// The field data may be released, so we need to copy the string to avoid invalid memory access.
return {min_string, max_string, null_count};
});
// The field data may later be released, so we need to copy the string to avoid invalid memory access.
return {std::string(min_string), std::string(max_string), null_count};
}
private:

View File

@ -16,6 +16,8 @@
#pragma once
#include "common/Array.h"
#include "common/VectorTrait.h"
#include "common/Utils.h"
#include "storage/MmapManager.h"
namespace milvus {

View File

@ -15,6 +15,8 @@
// limitations under the License.
#pragma once
#include "common/Utils.h"
#include "common/Span.h"
#include "mmap/ChunkData.h"
#include "storage/MmapManager.h"

View File

@ -26,6 +26,7 @@
#include <vector>
#include <math.h>
#include "cachinglayer/CacheSlot.h"
#include "cachinglayer/Manager.h"
#include "cachinglayer/Translator.h"
#include "cachinglayer/Utils.h"
@ -58,7 +59,6 @@ std::pair<size_t, size_t> inline GetChunkIDByOffset(
class ChunkedColumnBase : public ChunkedColumnInterface {
public:
// memory mode ctor
explicit ChunkedColumnBase(std::unique_ptr<Translator<Chunk>> translator,
const FieldMeta& field_meta)
: nullable_(field_meta.is_nullable()),
@ -82,18 +82,45 @@ class ChunkedColumnBase : public ChunkedColumnInterface {
return true;
}
auto [chunk_id, offset_in_chunk] = GetChunkIDByOffset(offset);
return IsValid(chunk_id, offset_in_chunk);
}
bool
IsValid(int64_t chunk_id, int64_t offset) const override {
if (nullable_) {
auto ca =
SemiInlineGet(slot_->PinCells({static_cast<cid_t>(chunk_id)}));
auto chunk = ca->get_cell_of(chunk_id);
return chunk->isValid(offset);
return chunk->isValid(offset_in_chunk);
}
void
BulkIsValid(std::function<void(bool, size_t)> fn,
const int64_t* offsets = nullptr,
int64_t count = 0) const override {
if (!nullable_) {
if (offsets == nullptr) {
for (int64_t i = 0; i < num_rows_; i++) {
fn(true, i);
}
} else {
for (int64_t i = 0; i < count; i++) {
fn(true, i);
}
}
}
// nullable:
if (offsets == nullptr) {
auto ca = SemiInlineGet(slot_->PinAllCells());
for (int64_t i = 0; i < num_rows_; i++) {
auto [cid, offset_in_chunk] = GetChunkIDByOffset(i);
auto chunk = ca->get_cell_of(cid);
auto valid = chunk->isValid(offset_in_chunk);
fn(valid, i);
}
} else {
auto [cids, offsets_in_chunk] = ToChunkIdAndOffset(offsets, count);
auto ca = SemiInlineGet(slot_->PinCells(cids));
for (int64_t i = 0; i < count; i++) {
auto chunk = ca->get_cell_of(cids[i]);
auto valid = chunk->isValid(offsets_in_chunk[i]);
fn(valid, i);
}
}
return true;
}
bool
@ -127,14 +154,22 @@ class ChunkedColumnBase : public ChunkedColumnInterface {
GetNumRowsUntilChunk(chunk_id);
}
virtual PinWrapper<SpanBase>
PinWrapper<SpanBase>
Span(int64_t chunk_id) const override {
PanicInfo(ErrorCode::Unsupported,
"Span only supported for ChunkedColumn");
}
virtual PinWrapper<
std::pair<std::vector<std::string_view>, FixedVector<bool>>>
void
BulkValueAt(std::function<void(const char*, size_t)> fn,
const int64_t* offsets,
int64_t count) override {
PanicInfo(ErrorCode::Unsupported,
"BulkValueAt only supported for ChunkedColumn and "
"ProxyChunkColumn");
}
PinWrapper<std::pair<std::vector<std::string_view>, FixedVector<bool>>>
StringViews(int64_t chunk_id,
std::optional<std::pair<int64_t, int64_t>> offset_len =
std::nullopt) const override {
@ -142,7 +177,7 @@ class ChunkedColumnBase : public ChunkedColumnInterface {
"StringViews only supported for VariableColumn");
}
virtual PinWrapper<std::pair<std::vector<ArrayView>, FixedVector<bool>>>
PinWrapper<std::pair<std::vector<ArrayView>, FixedVector<bool>>>
ArrayViews(
int64_t chunk_id,
std::optional<std::pair<int64_t, int64_t>> offset_len) const override {
@ -150,8 +185,7 @@ class ChunkedColumnBase : public ChunkedColumnInterface {
"ArrayViews only supported for ArrayChunkedColumn");
}
virtual PinWrapper<
std::pair<std::vector<std::string_view>, FixedVector<bool>>>
PinWrapper<std::pair<std::vector<std::string_view>, FixedVector<bool>>>
ViewsByOffsets(int64_t chunk_id,
const FixedVector<int32_t>& offsets) const override {
PanicInfo(ErrorCode::Unsupported,
@ -187,12 +221,6 @@ class ChunkedColumnBase : public ChunkedColumnInterface {
return meta->num_rows_until_chunk_;
}
virtual const char*
ValueAt(int64_t offset) override {
PanicInfo(ErrorCode::Unsupported,
"ValueAt only supported for ChunkedColumn");
}
protected:
bool nullable_{false};
size_t num_rows_{0};
@ -208,14 +236,15 @@ class ChunkedColumn : public ChunkedColumnBase {
: ChunkedColumnBase(std::move(translator), field_meta) {
}
// TODO(tiered storage 1): this method should be replaced with a bulk access method.
const char*
ValueAt(int64_t offset) override {
auto [chunk_id, offset_in_chunk] = GetChunkIDByOffset(offset);
auto ca =
SemiInlineGet(slot_->PinCells({static_cast<cid_t>(chunk_id)}));
auto chunk = ca->get_cell_of(chunk_id);
return chunk->ValueAt(offset_in_chunk);
void
BulkValueAt(std::function<void(const char*, size_t)> fn,
const int64_t* offsets,
int64_t count) override {
auto [cids, offsets_in_chunk] = ToChunkIdAndOffset(offsets, count);
auto ca = SemiInlineGet(slot_->PinCells(cids));
for (int64_t i = 0; i < count; i++) {
fn(ca->get_cell_of(cids[i])->ValueAt(offsets_in_chunk[i]), i);
}
}
PinWrapper<SpanBase>
@ -262,14 +291,50 @@ class ChunkedVariableColumn : public ChunkedColumnBase {
ca, static_cast<StringChunk*>(chunk)->ViewsByOffsets(offsets));
}
// TODO(tiered storage 1): this method should be replaced with a bulk access method.
// RawAt is called in three cases:
// 1. bulk_subscript, pass in an offset array, access the specified rows.
// 2. load, create skip index or text index, access all rows. (SkipIndex.h and CreateTextIndex)
// 3. GetJsonData, json related index will use this, see if it can be modified to batch access or batch pin.
T
RawAt(const size_t i) const {
if (i < 0 || i > num_rows_) {
void
BulkRawStringAt(std::function<void(std::string_view, size_t, bool)> fn,
const int64_t* offsets,
int64_t count) const override {
if constexpr (!std::is_same_v<T, std::string>) {
PanicInfo(ErrorCode::Unsupported,
"BulkRawStringAt only supported for "
"ChunkedVariableColumn<std::string>");
}
std::shared_ptr<CellAccessor<Chunk>> ca{nullptr};
if (offsets == nullptr) {
ca = SemiInlineGet(slot_->PinAllCells());
for (int64_t i = 0; i < num_rows_; i++) {
auto [cid, offset_in_chunk] = GetChunkIDByOffset(i);
auto chunk = ca->get_cell_of(cid);
auto valid = nullable_ ? chunk->isValid(offset_in_chunk) : true;
fn(static_cast<StringChunk*>(chunk)->operator[](
offset_in_chunk),
i,
valid);
}
} else {
auto [cids, offsets_in_chunk] = ToChunkIdAndOffset(offsets, count);
ca = SemiInlineGet(slot_->PinCells(cids));
for (int64_t i = 0; i < count; i++) {
auto chunk = ca->get_cell_of(cids[i]);
auto valid =
nullable_ ? chunk->isValid(offsets_in_chunk[i]) : true;
fn(static_cast<StringChunk*>(chunk)->operator[](
offsets_in_chunk[i]),
i,
valid);
}
}
}
Json
RawJsonAt(size_t i) const override {
if constexpr (!std::is_same_v<T, Json>) {
PanicInfo(
ErrorCode::Unsupported,
"RawJsonAt only supported for ChunkedVariableColumn<Json>");
}
if (i < 0 || i >= num_rows_) {
PanicInfo(ErrorCode::OutOfRange, "index out of range");
}
@ -279,7 +344,7 @@ class ChunkedVariableColumn : public ChunkedColumnBase {
auto chunk = ca->get_cell_of(chunk_id);
std::string_view str_view =
static_cast<StringChunk*>(chunk)->operator[](offset_in_chunk);
return T(str_view.data(), str_view.size());
return Json(str_view.data(), str_view.size());
}
};
@ -291,16 +356,18 @@ class ChunkedArrayColumn : public ChunkedColumnBase {
: ChunkedColumnBase(std::move(translator), field_meta) {
}
// TODO(tiered storage 1): this method should be replaced with a bulk access method.
ScalarArray
PrimitivieRawAt(const int i) const override {
auto [chunk_id, offset_in_chunk] = GetChunkIDByOffset(i);
auto ca =
SemiInlineGet(slot_->PinCells({static_cast<cid_t>(chunk_id)}));
auto chunk = ca->get_cell_of(chunk_id);
return static_cast<ArrayChunk*>(chunk)
->View(offset_in_chunk)
void
BulkArrayAt(std::function<void(ScalarArray&&, size_t)> fn,
const int64_t* offsets,
int64_t count) const override {
auto [cids, offsets_in_chunk] = ToChunkIdAndOffset(offsets, count);
auto ca = SemiInlineGet(slot_->PinCells(cids));
for (int64_t i = 0; i < count; i++) {
auto array = static_cast<ArrayChunk*>(ca->get_cell_of(cids[i]))
->View(offsets_in_chunk[i])
.output_data();
fn(std::move(array), i);
}
}
PinWrapper<std::pair<std::vector<ArrayView>, FixedVector<bool>>>

View File

@ -32,11 +32,9 @@
#include "common/Array.h"
#include "common/Chunk.h"
#include "common/GroupChunk.h"
#include "common/Common.h"
#include "common/EasyAssert.h"
#include "common/Span.h"
#include "common/Array.h"
#include "mmap/ChunkedColumn.h"
#include "mmap/ChunkedColumnInterface.h"
#include "segcore/storagev2translator/GroupCTMeta.h"
@ -71,6 +69,11 @@ class ChunkedColumnGroup {
return PinWrapper<GroupChunk*>(ca, chunk);
}
std::shared_ptr<CellAccessor<GroupChunk>>
GetGroupChunks(std::vector<int64_t> chunk_ids) {
return SemiInlineGet(slot_->PinCells(chunk_ids));
}
int64_t
NumRows() const {
return num_rows_;
@ -123,11 +126,44 @@ class ProxyChunkColumn : public ChunkedColumnInterface {
return chunk->isValid(offset_in_chunk);
}
bool
IsValid(int64_t chunk_id, int64_t offset) const override {
auto group_chunk = group_->GetGroupChunk(chunk_id);
void
BulkIsValid(std::function<void(bool, size_t)> fn,
const int64_t* offsets = nullptr,
int64_t count = 0) const override {
if (!field_meta_.is_nullable()) {
if (offsets == nullptr) {
for (int64_t i = 0; i < group_->NumRows(); i++) {
fn(true, i);
}
} else {
for (int64_t i = 0; i < count; i++) {
fn(true, i);
}
}
}
// nullable:
if (offsets == nullptr) {
int64_t current_offset = 0;
for (cid_t cid = 0; cid < num_chunks(); ++cid) {
auto group_chunk = group_->GetGroupChunk(cid);
auto chunk = group_chunk.get()->GetChunk(field_id_);
return chunk->isValid(offset);
auto chunk_rows = chunk->RowNums();
for (int64_t i = 0; i < chunk_rows; ++i) {
auto valid = chunk->isValid(i);
fn(valid, current_offset + i);
}
current_offset += chunk_rows;
}
} else {
auto [cids, offsets_in_chunk] = ToChunkIdAndOffset(offsets, count);
auto ca = group_->GetGroupChunks(cids);
for (int64_t i = 0; i < count; i++) {
auto* group_chunk = ca->get_cell_of(cids[i]);
auto chunk = group_chunk->GetChunk(field_id_);
auto valid = chunk->isValid(offsets_in_chunk[i]);
fn(valid, i);
}
}
}
bool
@ -251,38 +287,92 @@ class ProxyChunkColumn : public ChunkedColumnInterface {
return group_->GetNumRowsUntilChunk();
}
const char*
ValueAt(int64_t offset) override {
auto [chunk_id, offset_in_chunk] = GetChunkIDByOffset(offset);
auto chunk = GetChunk(chunk_id);
return chunk.get()->ValueAt(offset_in_chunk);
void
BulkValueAt(std::function<void(const char*, size_t)> fn,
const int64_t* offsets,
int64_t count) override {
auto [cids, offsets_in_chunk] = ToChunkIdAndOffset(offsets, count);
auto ca = group_->GetGroupChunks(cids);
for (int64_t i = 0; i < count; i++) {
auto* group_chunk = ca->get_cell_of(cids[i]);
auto chunk = group_chunk->GetChunk(field_id_);
fn(chunk->ValueAt(offsets_in_chunk[i]), i);
}
}
template <typename T>
T
RawAt(size_t i) const {
if (!IsChunkedVariableColumnDataType(data_type_)) {
void
BulkRawStringAt(std::function<void(std::string_view, size_t, bool)> fn,
const int64_t* offsets = nullptr,
int64_t count = 0) const override {
if (!IsChunkedVariableColumnDataType(data_type_) ||
data_type_ == DataType::JSON) {
PanicInfo(ErrorCode::Unsupported,
"RawAt only supported for ChunkedVariableColumn");
"BulkRawStringAt only supported for ProxyChunkColumn of "
"variable length type(except Json)");
}
if (offsets == nullptr) {
int64_t current_offset = 0;
for (cid_t cid = 0; cid < num_chunks(); ++cid) {
auto group_chunk = group_->GetGroupChunk(cid);
auto chunk = group_chunk.get()->GetChunk(field_id_);
auto chunk_rows = chunk->RowNums();
for (int64_t i = 0; i < chunk_rows; ++i) {
auto valid = chunk->isValid(i);
auto value =
static_cast<StringChunk*>(chunk.get())->operator[](i);
fn(value, current_offset + i, valid);
}
current_offset += chunk_rows;
}
} else {
auto [cids, offsets_in_chunk] = ToChunkIdAndOffset(offsets, count);
auto ca = group_->GetGroupChunks(cids);
for (int64_t i = 0; i < count; i++) {
auto* group_chunk = ca->get_cell_of(cids[i]);
auto chunk = group_chunk->GetChunk(field_id_);
auto valid = chunk->isValid(offsets_in_chunk[i]);
auto value = static_cast<StringChunk*>(chunk.get())
->
operator[](offsets_in_chunk[i]);
fn(value, i, valid);
}
}
}
// TODO(tiered storage 2): replace with Bulk version
Json
RawJsonAt(size_t i) const override {
if (data_type_ != DataType::JSON) {
PanicInfo(
ErrorCode::Unsupported,
"RawJsonAt only supported for ProxyChunkColumn of Json type");
}
auto [chunk_id, offset_in_chunk] = GetChunkIDByOffset(i);
auto chunk = group_->GetGroupChunk(chunk_id).get()->GetChunk(field_id_);
auto group_chunk = group_->GetGroupChunk(chunk_id);
auto chunk = group_chunk.get()->GetChunk(field_id_);
std::string_view str_view =
static_cast<StringChunk*>(chunk.get())->operator[](offset_in_chunk);
return T(str_view.data(), str_view.size());
return Json(str_view.data(), str_view.size());
}
ScalarArray
PrimitivieRawAt(const int i) const override {
void
BulkArrayAt(std::function<void(ScalarArray&&, size_t)> fn,
const int64_t* offsets,
int64_t count) const override {
if (!IsChunkedArrayColumnDataType(data_type_)) {
PanicInfo(ErrorCode::Unsupported,
"PrimitivieRawAt only supported for ChunkedArrayColumn");
"BulkArrayAt only supported for ChunkedArrayColumn");
}
auto [chunk_id, offset_in_chunk] = GetChunkIDByOffset(i);
auto chunk = group_->GetGroupChunk(chunk_id).get()->GetChunk(field_id_);
return static_cast<ArrayChunk*>(chunk.get())
->View(offset_in_chunk)
auto [cids, offsets_in_chunk] = ToChunkIdAndOffset(offsets, count);
auto ca = group_->GetGroupChunks(cids);
for (int64_t i = 0; i < count; i++) {
auto* group_chunk = ca->get_cell_of(cids[i]);
auto chunk = group_chunk->GetChunk(field_id_);
auto array = static_cast<ArrayChunk*>(chunk.get())
->View(offsets_in_chunk[i])
.output_data();
fn(std::move(array), i);
}
}
private:

View File

@ -34,8 +34,14 @@ class ChunkedColumnInterface {
virtual bool
IsValid(size_t offset) const = 0;
virtual bool
IsValid(int64_t chunk_id, int64_t offset) const = 0;
// fn: (bool is_valid, size_t offset) -> void
// If offsets is nullptr, this function will iterate over all rows.
// Only BulkRawStringAt and BulkIsValid allow offsets to be nullptr.
// Other Bulk* methods can also support nullptr offsets, but not added at this moment.
virtual void
BulkIsValid(std::function<void(bool, size_t)> fn,
const int64_t* offsets = nullptr,
int64_t count = 0) const = 0;
// Check if the column can contain null values
virtual bool
@ -90,22 +96,37 @@ class ChunkedColumnInterface {
virtual const std::vector<int64_t>&
GetNumRowsUntilChunk() const = 0;
virtual const char*
ValueAt(int64_t offset) = 0;
virtual void
BulkValueAt(std::function<void(const char*, size_t)> fn,
const int64_t* offsets,
int64_t count) = 0;
// Get raw value at given offset for variable length types (string/json)
template <typename T>
T
RawAt(size_t offset) const {
// fn: (std::string_view value, size_t offset, bool is_valid) -> void
// If offsets is nullptr, this function will iterate over all rows.
// Only BulkRawStringAt and BulkIsValid allow offsets to be nullptr.
// Other Bulk* methods can also support nullptr offsets, but not added at this moment.
virtual void
BulkRawStringAt(std::function<void(std::string_view, size_t, bool)> fn,
const int64_t* offsets = nullptr,
int64_t count = 0) const {
PanicInfo(ErrorCode::Unsupported,
"RawAt only supported for ChunkedVariableColumn");
"BulkRawStringAt only supported for ChunkColumnInterface of "
"variable length type");
}
// Get raw value at given offset for array type
virtual ScalarArray
PrimitivieRawAt(int offset) const {
virtual Json
RawJsonAt(size_t offset) const {
PanicInfo(
ErrorCode::Unsupported,
"RawJsonAt only supported for ChunkColumnInterface of Json type");
}
virtual void
BulkArrayAt(std::function<void(ScalarArray&&, size_t)> fn,
const int64_t* offsets,
int64_t count) const {
PanicInfo(ErrorCode::Unsupported,
"PrimitivieRawAt only supported for ChunkedArrayColumn");
"BulkArrayAt only supported for ChunkedArrayColumn");
}
static bool
@ -125,6 +146,23 @@ class ChunkedColumnInterface {
return !IsChunkedVariableColumnDataType(data_type) &&
!IsChunkedArrayColumnDataType(data_type);
}
protected:
std::pair<std::vector<milvus::cachinglayer::cid_t>, std::vector<int64_t>>
ToChunkIdAndOffset(const int64_t* offsets, int64_t count) const {
AssertInfo(offsets != nullptr, "Offsets cannot be nullptr");
std::vector<milvus::cachinglayer::cid_t> cids;
cids.reserve(count);
std::vector<int64_t> offsets_in_chunk;
offsets_in_chunk.reserve(count);
for (int64_t i = 0; i < count; i++) {
auto [chunk_id, offset_in_chunk] = GetChunkIDByOffset(offsets[i]);
cids.push_back(chunk_id);
offsets_in_chunk.push_back(offset_in_chunk);
}
return std::make_pair(std::move(cids), std::move(offsets_in_chunk));
}
};
} // namespace milvus

View File

@ -11,12 +11,12 @@
#include <algorithm>
#include "mmap/ChunkedColumn.h"
#include "query/CachedSearchIterator.h"
#include "query/SearchBruteForce.h"
namespace milvus::query {
// For sealed segment with vector index
CachedSearchIterator::CachedSearchIterator(
const milvus::index::VectorIndex& index,
const knowhere::DataSetPtr& query_ds,
@ -43,26 +43,6 @@ CachedSearchIterator::CachedSearchIterator(
}
}
CachedSearchIterator::CachedSearchIterator(
const dataset::SearchDataset& query_ds,
const dataset::RawDataset& raw_ds,
const SearchInfo& search_info,
const std::map<std::string, std::string>& index_info,
const BitsetView& bitset,
const milvus::DataType& data_type) {
nq_ = query_ds.num_queries;
Init(search_info);
auto expected_iterators = GetBruteForceSearchIterators(
query_ds, raw_ds, search_info, index_info, bitset, data_type);
if (expected_iterators.has_value()) {
iterators_ = std::move(expected_iterators.value());
} else {
PanicInfo(ErrorCode::UnexpectedError,
"Failed to create iterators from index");
}
}
void
CachedSearchIterator::InitializeChunkedIterators(
const dataset::SearchDataset& query_ds,
@ -74,10 +54,9 @@ CachedSearchIterator::InitializeChunkedIterators(
int64_t offset = 0;
chunked_heaps_.resize(nq_);
for (int64_t chunk_id = 0; chunk_id < num_chunks_; ++chunk_id) {
// TODO(tiered storage 1): this should store PinWrapper. Double check all places.
auto [chunk_data, chunk_size] = get_chunk_data(chunk_id);
auto sub_data = query::dataset::RawDataset{
offset, query_ds.dim, chunk_size, chunk_data.get()};
offset, query_ds.dim, chunk_size, chunk_data};
auto expected_iterators = GetBruteForceSearchIterators(
query_ds, sub_data, search_info, index_info, bitset, data_type);
@ -94,6 +73,7 @@ CachedSearchIterator::InitializeChunkedIterators(
}
}
// For growing segment with chunked data, BF
CachedSearchIterator::CachedSearchIterator(
const dataset::SearchDataset& query_ds,
const segcore::VectorBase* vec_data,
@ -126,13 +106,14 @@ CachedSearchIterator::CachedSearchIterator(
data_type,
[&vec_data, vec_size_per_chunk, row_count](int64_t chunk_id) {
const void* chunk_data = vec_data->get_chunk_data(chunk_id);
auto pw = milvus::cachinglayer::PinWrapper<const void*>(chunk_data);
// no need to store a PinWrapper for growing, because vec_data is guaranteed to not be evicted.
int64_t chunk_size = std::min(
vec_size_per_chunk, row_count - chunk_id * vec_size_per_chunk);
return std::make_pair(pw, chunk_size);
return std::make_pair(chunk_data, chunk_size);
});
}
// For sealed segment with chunked data, BF
CachedSearchIterator::CachedSearchIterator(
ChunkedColumnInterface* column,
const dataset::SearchDataset& query_ds,
@ -150,17 +131,22 @@ CachedSearchIterator::CachedSearchIterator(
Init(search_info);
iterators_.reserve(nq_ * num_chunks_);
pin_wrappers_.reserve(num_chunks_);
InitializeChunkedIterators(
query_ds,
search_info,
index_info,
bitset,
data_type,
[column](int64_t chunk_id) {
[this, column](int64_t chunk_id) {
auto pw = column->DataOfChunk(chunk_id).transform<const void*>(
[](const auto& x) { return static_cast<const void*>(x); });
int64_t chunk_size = column->chunk_row_nums(chunk_id);
return std::make_pair(pw, chunk_size);
// pw guarantees chunk_data is kept alive.
auto chunk_data = pw.get();
pin_wrappers_.emplace_back(std::move(pw));
return std::make_pair(chunk_data, chunk_size);
});
}

View File

@ -20,7 +20,7 @@
#include "query/helper.h"
#include "segcore/ConcurrentVector.h"
#include "index/VectorIndex.h"
#include "mmap/ChunkedColumn.h"
#include "mmap/ChunkedColumnInterface.h"
namespace milvus::query {
@ -28,7 +28,7 @@ namespace milvus::query {
// search iterators and filter the results based on the last_bound,
// radius and range_filter.
// It provides a number of constructors to support different scenarios,
// including growing/sealed, chunked/non-chunked.
// including growing/sealed, with/without index.
//
// It does not care about TopK in search_info
// The topk in SearchResult will be set to the batch_size for compatibility
@ -43,14 +43,6 @@ class CachedSearchIterator {
const SearchInfo& search_info,
const BitsetView& bitset);
// For sealed segment, BF
CachedSearchIterator(const dataset::SearchDataset& dataset,
const dataset::RawDataset& raw_ds,
const SearchInfo& search_info,
const std::map<std::string, std::string>& index_info,
const BitsetView& bitset,
const milvus::DataType& data_type);
// For growing segment with chunked data, BF
CachedSearchIterator(const dataset::SearchDataset& dataset,
const segcore::VectorBase* vec_data,
@ -86,9 +78,10 @@ class CachedSearchIterator {
using IterIdx = size_t;
using IterIdDisIdPair = std::pair<IterIdx, DisIdPair>;
using GetChunkDataFunc =
std::function<std::pair<milvus::cachinglayer::PinWrapper<const void*>,
int64_t>(int64_t)>;
std::function<std::pair<const void*, int64_t>(int64_t)>;
// used only for sealed segment with chunked data
std::vector<milvus::cachinglayer::PinWrapper<const void*>> pin_wrappers_;
int64_t batch_size_ = 0;
std::vector<knowhere::IndexNode::IteratorPtr> iterators_;
int8_t sign_ = 1;
@ -174,6 +167,7 @@ class CachedSearchIterator {
void
Init(const SearchInfo& search_info);
// must call get_chunk_data from chunk 0 to chunk num_chunks_ - 1 in order.
void
InitializeChunkedIterators(
const dataset::SearchDataset& dataset,

View File

@ -170,64 +170,4 @@ SearchOnSealedColumn(const Schema& schema,
result.total_nq_ = query_dataset.num_queries;
}
void
SearchOnSealedData(const Schema& schema,
const void* vec_data,
const SearchInfo& search_info,
const std::map<std::string, std::string>& index_info,
const void* query_data,
int64_t num_queries,
int64_t row_count,
const BitsetView& bitset,
SearchResult& result) {
auto field_id = search_info.field_id_;
auto& field = schema[field_id];
// TODO(SPARSE): see todo in PlanImpl.h::PlaceHolder.
auto dim = field.get_data_type() == DataType::VECTOR_SPARSE_FLOAT
? 0
: field.get_dim();
query::dataset::SearchDataset query_dataset{search_info.metric_type_,
num_queries,
search_info.topk_,
search_info.round_decimal_,
dim,
query_data};
auto data_type = field.get_data_type();
CheckBruteForceSearchParam(field, search_info);
auto raw_dataset = query::dataset::RawDataset{0, dim, row_count, vec_data};
if (milvus::exec::UseVectorIterator(search_info)) {
auto sub_qr = PackBruteForceSearchIteratorsIntoSubResult(query_dataset,
raw_dataset,
search_info,
index_info,
bitset,
data_type);
result.AssembleChunkVectorIterators(
num_queries, 1, {0}, sub_qr.chunk_iterators());
} else if (search_info.iterator_v2_info_.has_value()) {
CachedSearchIterator cached_iter(query_dataset,
raw_dataset,
search_info,
index_info,
bitset,
data_type);
cached_iter.NextBatch(search_info, result);
return;
} else {
auto sub_qr = BruteForceSearch(query_dataset,
raw_dataset,
search_info,
index_info,
bitset,
data_type);
result.distances_ = std::move(sub_qr.mutable_distances());
result.seg_offsets_ = std::move(sub_qr.mutable_seg_offsets());
}
result.unity_topK_ = query_dataset.topk;
result.total_nq_ = query_dataset.num_queries;
}
} // namespace milvus::query

View File

@ -38,15 +38,4 @@ SearchOnSealedColumn(const Schema& schema,
const BitsetView& bitset,
SearchResult& result);
void
SearchOnSealedData(const Schema& schema,
const void* vec_data,
const SearchInfo& search_info,
const std::map<std::string, std::string>& index_info,
const void* query_data,
int64_t num_queries,
int64_t row_count,
const BitsetView& bitset,
SearchResult& result);
} // namespace milvus::query

View File

@ -304,8 +304,6 @@ ChunkedSegmentSealedImpl::load_field_data_internal(
AssertInfo(info.row_count > 0, "The row count of field data is 0");
auto field_id = FieldId(id);
auto insert_files = info.insert_files;
storage::SortByPath(insert_files);
auto field_data_info =
FieldDataInfo(field_id.get(), num_rows, load_info.mmap_dir_path);
@ -315,6 +313,8 @@ ChunkedSegmentSealedImpl::load_field_data_internal(
num_rows);
if (SystemProperty::Instance().IsSystem(field_id)) {
auto insert_files = info.insert_files;
storage::SortByPath(insert_files);
auto parallel_degree = static_cast<uint64_t>(
DEFAULT_FIELD_MAX_MEMORY_LIMIT / FILE_SLICE_SIZE);
field_data_info.arrow_reader_channel->set_capacity(parallel_degree *
@ -333,13 +333,21 @@ ChunkedSegmentSealedImpl::load_field_data_internal(
this->get_segment_id(),
field_id.get());
} else {
std::vector<std::pair<std::string, int64_t>>
insert_files_with_entries_nums;
for (int i = 0; i < info.insert_files.size(); i++) {
insert_files_with_entries_nums.emplace_back(
info.insert_files[i], info.entries_nums[i]);
}
storage::SortByPath(insert_files_with_entries_nums);
auto field_meta = schema_->operator[](field_id);
std::unique_ptr<Translator<milvus::Chunk>> translator =
std::make_unique<storagev1translator::ChunkTranslator>(
this->get_segment_id(),
field_meta,
field_data_info,
insert_files,
std::move(insert_files_with_entries_nums),
info.enable_mmap);
auto data_type = field_meta.get_data_type();
@ -966,25 +974,35 @@ ChunkedSegmentSealedImpl::bulk_subscript_impl(ChunkedColumnInterface* field,
int64_t count,
T* dst) {
static_assert(IsScalar<T>);
auto column = reinterpret_cast<ChunkedColumn*>(field);
for (int64_t i = 0; i < count; ++i) {
auto offset = seg_offsets[i];
dst[i] = *static_cast<const S*>(
static_cast<const void*>(column->ValueAt(offset)));
}
field->BulkValueAt(
[dst](const char* value, size_t i) {
dst[i] = *static_cast<const S*>(static_cast<const void*>(value));
},
seg_offsets,
count);
}
template <typename S, typename T>
template <typename S>
void
ChunkedSegmentSealedImpl::bulk_subscript_ptr_impl(
ChunkedColumnInterface* column,
const int64_t* seg_offsets,
int64_t count,
google::protobuf::RepeatedPtrField<T>* dst) {
auto field = reinterpret_cast<ChunkedVariableColumn<S>*>(column);
google::protobuf::RepeatedPtrField<std::string>* dst) {
if constexpr (std::is_same_v<S, Json>) {
for (int64_t i = 0; i < count; ++i) {
auto offset = seg_offsets[i];
dst->at(i) = std::move(T(field->RawAt(offset)));
Json json = column->RawJsonAt(offset);
dst->at(i) = std::move(std::string(json.data()));
}
} else {
static_assert(std::is_same_v<S, std::string>);
column->BulkRawStringAt(
[dst](std::string_view value, size_t offset, bool is_valid) {
dst->at(offset) = std::move(std::string(value));
},
seg_offsets,
count);
}
}
@ -995,11 +1013,10 @@ ChunkedSegmentSealedImpl::bulk_subscript_array_impl(
const int64_t* seg_offsets,
int64_t count,
google::protobuf::RepeatedPtrField<T>* dst) {
auto field = reinterpret_cast<ChunkedArrayColumn*>(column);
for (int64_t i = 0; i < count; ++i) {
auto offset = seg_offsets[i];
dst->at(i) = std::move(field->PrimitivieRawAt(offset));
}
column->BulkArrayAt(
[dst](ScalarArray&& array, size_t i) { dst->at(i) = std::move(array); },
seg_offsets,
count);
}
// for dense vector
@ -1010,13 +1027,13 @@ ChunkedSegmentSealedImpl::bulk_subscript_impl(int64_t element_sizeof,
int64_t count,
void* dst_raw) {
auto dst_vec = reinterpret_cast<char*>(dst_raw);
auto column = reinterpret_cast<ChunkedColumn*>(field);
for (int64_t i = 0; i < count; ++i) {
auto offset = seg_offsets[i];
auto src = column->ValueAt(offset);
field->BulkValueAt(
[&](const char* value, size_t i) {
auto dst = dst_vec + i * element_sizeof;
memcpy(dst, src, element_sizeof);
}
memcpy(dst, value, element_sizeof);
},
seg_offsets,
count);
}
void
@ -1077,18 +1094,10 @@ ChunkedSegmentSealedImpl::CreateTextIndex(FieldId field_id) {
// build
auto iter = fields_.find(field_id);
if (iter != fields_.end()) {
auto column =
std::dynamic_pointer_cast<ChunkedVariableColumn<std::string>>(
iter->second);
AssertInfo(
column != nullptr,
"failed to create text index, field is not of text type: {}",
field_id.get());
auto n = column->NumRows();
for (size_t i = 0; i < n; i++) {
index->AddText(
std::string(column->RawAt(i)), column->IsValid(i), i);
}
iter->second->BulkRawStringAt(
[&](std::string_view value, size_t offset, bool is_valid) {
index->AddText(std::string(value), is_valid, offset);
});
} else { // fetch raw data from index.
auto field_index_iter = scalar_indexings_.find(field_id);
AssertInfo(field_index_iter != scalar_indexings_.end(),
@ -1150,10 +1159,10 @@ ChunkedSegmentSealedImpl::get_raw_data(FieldId field_id,
auto ret = fill_with_empty(field_id, count);
if (column->IsNullable()) {
auto dst = ret->mutable_valid_data()->mutable_data();
for (int64_t i = 0; i < count; ++i) {
auto offset = seg_offsets[i];
dst[i] = column->IsValid(offset);
}
column->BulkIsValid(
[&](bool is_valid, size_t offset) { dst[offset] = is_valid; },
seg_offsets,
count);
}
switch (field_meta.get_data_type()) {
case DataType::VARCHAR:
@ -1168,7 +1177,7 @@ ChunkedSegmentSealedImpl::get_raw_data(FieldId field_id,
}
case DataType::JSON: {
bulk_subscript_ptr_impl<Json, std::string>(
bulk_subscript_ptr_impl<Json>(
column.get(),
seg_offsets,
count,
@ -1295,17 +1304,26 @@ ChunkedSegmentSealedImpl::get_raw_data(FieldId field_id,
}
case DataType::VECTOR_SPARSE_FLOAT: {
auto dst = ret->mutable_vectors()->mutable_sparse_float_vector();
auto col = reinterpret_cast<ChunkedColumn*>(column.get());
SparseRowsToProto(
[&](size_t i) {
int64_t max_dim = 0;
column->BulkValueAt(
[&](const char* value, size_t i) mutable {
auto offset = seg_offsets[i];
auto row =
static_cast<const knowhere::sparse::SparseRow<float>*>(
static_cast<const void*>(col->ValueAt(offset)));
return offset != INVALID_SEG_OFFSET ? row : nullptr;
offset != INVALID_SEG_OFFSET
? static_cast<
const knowhere::sparse::SparseRow<float>*>(
static_cast<const void*>(value))
: nullptr;
if (row == nullptr) {
dst->add_contents();
return;
}
max_dim = std::max(max_dim, row->dim());
dst->add_contents(row->data(), row->data_byte_size());
},
count,
dst);
seg_offsets,
count);
dst->set_dim(max_dim);
ret->mutable_vectors()->set_dim(dst->dim());
break;
}
@ -1389,17 +1407,17 @@ ChunkedSegmentSealedImpl::bulk_subscript(
auto ret = fill_with_empty(field_id, count);
if (column->IsNullable()) {
auto dst = ret->mutable_valid_data()->mutable_data();
for (int64_t i = 0; i < count; ++i) {
auto offset = seg_offsets[i];
dst[i] = column->IsValid(offset);
}
column->BulkIsValid(
[&](bool is_valid, size_t offset) { dst[offset] = is_valid; },
seg_offsets,
count);
}
auto dst = ret->mutable_scalars()->mutable_json_data()->mutable_data();
auto field = reinterpret_cast<ChunkedVariableColumn<Json>*>(column.get());
for (int64_t i = 0; i < count; ++i) {
auto offset = seg_offsets[i];
dst->at(i) = ExtractSubJson(std::string(field->RawAt(offset)),
dynamic_field_names);
Json json = column->RawJsonAt(offset);
dst->at(i) =
ExtractSubJson(std::string(json.data()), dynamic_field_names);
}
return ret;
}
@ -1662,9 +1680,9 @@ ChunkedSegmentSealedImpl::generate_interim_index(const FieldId field_id) {
segcore_config_,
SegmentType::Sealed,
is_sparse));
// if (row_count < field_binlog_config->GetBuildThreshold()) {
// return false;
// }
if (row_count < field_binlog_config->GetBuildThreshold()) {
return false;
}
std::shared_ptr<ChunkedColumnInterface> vec_data{};
{
std::shared_lock lck(mutex_);
@ -1750,19 +1768,7 @@ ChunkedSegmentSealedImpl::load_field_data_common(
stats_.mem_size += column->DataByteSize();
if (IsVariableDataType(data_type)) {
if (IsStringDataType(data_type)) {
if (!is_proxy_column) {
auto var_column = std::dynamic_pointer_cast<
ChunkedVariableColumn<std::string>>(column);
AssertInfo(var_column != nullptr,
"column is not of variable type");
LoadStringSkipIndex(field_id, 0, *var_column);
} else {
auto var_column =
std::dynamic_pointer_cast<ProxyChunkColumn>(column);
AssertInfo(var_column != nullptr,
"column is not of variable type");
LoadStringSkipIndex(field_id, 0, *var_column);
}
LoadStringSkipIndex(field_id, 0, *column);
}
// update average row data size
SegmentInternalInterface::set_field_avg_size(

View File

@ -122,14 +122,12 @@ class ChunkedSegmentSealedImpl : public SegmentSealed {
// TODO(tiered storage 1): should return a PinWrapper
std::pair<milvus::Json, bool>
GetJsonData(FieldId field_id, size_t offset) const override {
auto column =
std::dynamic_pointer_cast<ChunkedVariableColumn<milvus::Json>>(
fields_.at(field_id));
auto column = fields_.at(field_id);
bool is_valid = column->IsValid(offset);
if (!is_valid) {
return std::make_pair(milvus::Json(), false);
}
return std::make_pair(milvus::Json(column->RawAt(offset)), is_valid);
return std::make_pair(column->RawJsonAt(offset), is_valid);
}
void
@ -307,12 +305,13 @@ class ChunkedSegmentSealedImpl : public SegmentSealed {
int64_t count,
T* dst_raw);
template <typename S, typename T = S>
template <typename S>
static void
bulk_subscript_ptr_impl(ChunkedColumnInterface* field,
bulk_subscript_ptr_impl(
ChunkedColumnInterface* field,
const int64_t* seg_offsets,
int64_t count,
google::protobuf::RepeatedPtrField<T>* dst_raw);
google::protobuf::RepeatedPtrField<std::string>* dst_raw);
template <typename T>
static void

View File

@ -415,7 +415,6 @@ struct InsertRecord {
protected:
storage::MmapChunkDescriptorPtr mmap_descriptor_;
std::unordered_map<FieldId, std::unique_ptr<VectorBase>> data_{};
mutable std::shared_mutex shared_mutex_{};
};
@ -708,6 +707,7 @@ struct InsertRecord<false> : public InsertRecord<true> {
AckResponder ack_responder_;
private:
std::unordered_map<FieldId, std::unique_ptr<VectorBase>> data_{};
std::unordered_map<FieldId, ThreadSafeValidDataPtr> valid_data_{};
};

View File

@ -27,6 +27,7 @@
#include "common/EasyAssert.h"
#include "common/FieldData.h"
#include "common/Schema.h"
#include "common/Json.h"
#include "common/Types.h"
#include "common/Common.h"
#include "fmt/format.h"
@ -812,7 +813,7 @@ SegmentGrowingImpl::bulk_subscript(FieldId field_id,
break;
}
case DataType::JSON: {
bulk_subscript_ptr_impl<Json, std::string>(
bulk_subscript_ptr_impl<Json>(
vec_ptr,
seg_offsets,
count,
@ -878,21 +879,21 @@ SegmentGrowingImpl::bulk_subscript_sparse_float_vector_impl(
indexing_record_.GetDataFromIndex(field_id, seg_offsets, count, 0, output);
}
template <typename S, typename T>
template <typename S>
void
SegmentGrowingImpl::bulk_subscript_ptr_impl(
const VectorBase* vec_raw,
const int64_t* seg_offsets,
int64_t count,
google::protobuf::RepeatedPtrField<T>* dst) const {
google::protobuf::RepeatedPtrField<std::string>* dst) const {
auto vec = dynamic_cast<const ConcurrentVector<S>*>(vec_raw);
auto& src = *vec;
for (int64_t i = 0; i < count; ++i) {
auto offset = seg_offsets[i];
if (IsVariableTypeSupportInChunk<S> && mmap_descriptor_ != nullptr) {
dst->at(i) = std::move(T(src.view_element(offset)));
dst->at(i) = std::move(std::string(src.view_element(offset)));
} else {
dst->at(i) = std::move(T(src[offset]));
dst->at(i) = std::move(std::string(src[offset]));
}
}
}

View File

@ -202,12 +202,13 @@ class SegmentGrowingImpl : public SegmentGrowing {
int64_t count,
T* output) const;
template <typename S, typename T = S>
template <typename S>
void
bulk_subscript_ptr_impl(const VectorBase* vec_raw,
bulk_subscript_ptr_impl(
const VectorBase* vec_raw,
const int64_t* seg_offsets,
int64_t count,
google::protobuf::RepeatedPtrField<T>* dst) const;
google::protobuf::RepeatedPtrField<std::string>* dst) const;
// for scalar array vectors
template <typename T>

View File

@ -335,12 +335,11 @@ class SegmentInternalInterface : public SegmentInterface {
const bool* valid_data,
int64_t count);
template <typename T>
void
LoadStringSkipIndex(FieldId field_id,
int64_t chunk_id,
const T& var_column) {
skip_index_.LoadString(field_id, chunk_id, var_column);
const ChunkedColumnInterface& column) {
skip_index_.LoadString(field_id, chunk_id, column);
}
virtual DataType

View File

@ -17,6 +17,8 @@
#include <string>
#include <vector>
#include "cachinglayer/Manager.h"
#include "common/type_c.h"
#include "common/Common.h"
#include "common/FieldData.h"
#include "common/Types.h"
@ -973,4 +975,17 @@ upper_bound(const ConcurrentVector<Timestamp>& timestamps,
}
return first;
}
// Get the globally configured cache warmup policy for the given content type.
CacheWarmupPolicy
getCacheWarmupPolicy(bool is_vector, bool is_index) {
auto& manager = milvus::cachinglayer::Manager::GetInstance();
if (is_index) {
return is_vector ? manager.getVectorIndexCacheWarmupPolicy()
: manager.getScalarIndexCacheWarmupPolicy();
} else {
return is_vector ? manager.getVectorFieldCacheWarmupPolicy()
: manager.getScalarFieldCacheWarmupPolicy();
}
}
} // namespace milvus::segcore

View File

@ -17,6 +17,7 @@
#include <vector>
#include "common/FieldData.h"
#include "common/type_c.h"
#include "common/Types.h"
#include "index/Index.h"
#include "segcore/ConcurrentVector.h"
@ -128,4 +129,8 @@ upper_bound(const ConcurrentVector<Timestamp>& timestamps,
int64_t first,
int64_t last,
Timestamp value);
CacheWarmupPolicy
getCacheWarmupPolicy(bool is_vector, bool is_index);
} // namespace milvus::segcore

View File

@ -16,7 +16,7 @@
#include "segcore/SegcoreConfig.h"
#include "segcore/segcore_init_c.h"
#include "cachinglayer/Manager.h"
#include "cachinglayer/Utils.h"
namespace milvus::segcore {
std::once_flag close_glog_once;
@ -122,11 +122,32 @@ SetThreadName(const char* name) {
}
extern "C" void
ConfigureTieredStorage(const bool enabled_globally,
const int64_t memory_limit_bytes,
const int64_t disk_limit_bytes) {
ConfigureTieredStorage(const CacheWarmupPolicy scalarFieldCacheWarmupPolicy,
const CacheWarmupPolicy vectorFieldCacheWarmupPolicy,
const CacheWarmupPolicy scalarIndexCacheWarmupPolicy,
const CacheWarmupPolicy vectorIndexCacheWarmupPolicy,
const int64_t memory_low_watermark_bytes,
const int64_t memory_high_watermark_bytes,
const int64_t memory_max_bytes,
const int64_t disk_low_watermark_bytes,
const int64_t disk_high_watermark_bytes,
const int64_t disk_max_bytes,
const bool evictionEnabled,
const int64_t cache_touch_window_ms,
const int64_t eviction_interval_ms) {
milvus::cachinglayer::Manager::ConfigureTieredStorage(
enabled_globally, memory_limit_bytes, disk_limit_bytes);
{scalarFieldCacheWarmupPolicy,
vectorFieldCacheWarmupPolicy,
scalarIndexCacheWarmupPolicy,
vectorIndexCacheWarmupPolicy},
{memory_low_watermark_bytes,
memory_high_watermark_bytes,
memory_max_bytes,
disk_low_watermark_bytes,
disk_high_watermark_bytes,
disk_max_bytes},
evictionEnabled,
{cache_touch_window_ms, eviction_interval_ms});
}
} // namespace milvus::segcore

View File

@ -14,6 +14,8 @@
#include <stdbool.h>
#include <stdint.h>
#include "common/type_c.h"
#ifdef __cplusplus
extern "C" {
#endif
@ -66,9 +68,19 @@ void
SetThreadName(const char*);
void
ConfigureTieredStorage(const bool enabled_globally,
const int64_t memory_limit_bytes,
const int64_t disk_limit_bytes);
ConfigureTieredStorage(const CacheWarmupPolicy scalarFieldCacheWarmupPolicy,
const CacheWarmupPolicy vectorFieldCacheWarmupPolicy,
const CacheWarmupPolicy scalarIndexCacheWarmupPolicy,
const CacheWarmupPolicy vectorIndexCacheWarmupPolicy,
const int64_t memory_low_watermark_bytes,
const int64_t memory_high_watermark_bytes,
const int64_t memory_max_bytes,
const int64_t disk_low_watermark_bytes,
const int64_t disk_high_watermark_bytes,
const int64_t disk_max_bytes,
const bool evictionEnabled,
const int64_t cache_touch_window_ms,
const int64_t eviction_interval_ms);
#ifdef __cplusplus
}

View File

@ -562,13 +562,6 @@ AddFieldDataInfoForSealed(CSegmentInterface c_segment,
}
}
CStatus
WarmupChunkCache(CSegmentInterface c_segment,
int64_t field_id,
bool mmap_enabled) {
return milvus::SuccessCStatus();
}
void
RemoveFieldFile(CSegmentInterface c_segment, int64_t field_id) {
auto segment = reinterpret_cast<milvus::segcore::SegmentSealed*>(c_segment);

View File

@ -142,11 +142,6 @@ CStatus
AddFieldDataInfoForSealed(CSegmentInterface c_segment,
CLoadFieldDataInfo c_load_field_data_info);
CStatus
WarmupChunkCache(CSegmentInterface c_segment,
int64_t field_id,
bool mmap_enabled);
////////////////////////////// interfaces for SegmentInterface //////////////////////////////
CStatus
ExistPk(CSegmentInterface c_segment,

View File

@ -27,105 +27,107 @@
namespace milvus::segcore::storagev1translator {
ChunkTranslator::ChunkTranslator(int64_t segment_id,
ChunkTranslator::ChunkTranslator(
int64_t segment_id,
FieldMeta field_meta,
FieldDataInfo field_data_info,
std::vector<std::string> insert_files,
std::vector<std::pair<std::string, int64_t>>&& files_and_rows,
bool use_mmap)
: segment_id_(segment_id),
key_(fmt::format("seg_{}_f_{}", segment_id, field_data_info.field_id)),
field_id_(field_data_info.field_id),
field_meta_(field_meta),
key_(fmt::format("seg_{}_f_{}", segment_id, field_meta.get_id().get())),
use_mmap_(use_mmap),
files_and_rows_(std::move(files_and_rows)),
mmap_dir_path_(field_data_info.mmap_dir_path),
meta_(use_mmap ? milvus::cachinglayer::StorageType::DISK
: milvus::cachinglayer::StorageType::MEMORY) {
chunks_.resize(insert_files.size());
AssertInfo(
!SystemProperty::Instance().IsSystem(FieldId(field_data_info.field_id)),
: milvus::cachinglayer::StorageType::MEMORY,
milvus::segcore::getCacheWarmupPolicy(
IsVectorDataType(field_meta.get_data_type()),
/* is_index */ false),
/* support_eviction */ false) {
AssertInfo(!SystemProperty::Instance().IsSystem(FieldId(field_id_)),
"ChunkTranslator not supported for system field");
auto parallel_degree =
static_cast<uint64_t>(DEFAULT_FIELD_MAX_MEMORY_LIMIT / FILE_SLICE_SIZE);
auto& pool = ThreadPools::GetThreadPool(milvus::ThreadPoolPriority::MIDDLE);
pool.Submit(LoadArrowReaderFromRemote,
insert_files,
field_data_info.arrow_reader_channel);
LOG_INFO("segment {} submits load field {} task to thread pool",
segment_id_,
field_data_info.field_id);
auto data_type = field_meta.get_data_type();
auto cid = 0;
auto row_count = 0;
meta_.num_rows_until_chunk_.push_back(0);
for (auto& [file, rows] : files_and_rows_) {
meta_.num_rows_until_chunk_.push_back(
meta_.num_rows_until_chunk_.back() + rows);
}
AssertInfo(meta_.num_rows_until_chunk_.back() == field_data_info.row_count,
fmt::format("data lost while loading column {}: found "
"num rows {} but expected {}",
field_data_info.field_id,
meta_.num_rows_until_chunk_.back(),
field_data_info.row_count));
}
std::unique_ptr<milvus::Chunk>
ChunkTranslator::load_chunk(milvus::cachinglayer::cid_t cid) {
auto& pool = ThreadPools::GetThreadPool(milvus::ThreadPoolPriority::MIDDLE);
auto channel = std::make_shared<ArrowReaderChannel>();
pool.Submit(LoadArrowReaderFromRemote,
std::vector<std::string>{files_and_rows_[cid].first},
channel);
LOG_DEBUG("segment {} submits load field {} chunk {} task to thread pool",
segment_id_,
field_id_,
cid);
auto data_type = field_meta_.get_data_type();
if (!use_mmap_) {
std::shared_ptr<milvus::ArrowDataWrapper> r;
while (field_data_info.arrow_reader_channel->pop(r)) {
while (channel->pop(r)) {
arrow::ArrayVector array_vec =
read_single_column_batches(r->reader);
auto chunk =
create_chunk(field_meta,
return create_chunk(field_meta_,
IsVectorDataType(data_type) &&
!IsSparseFloatVectorDataType(data_type)
? field_meta.get_dim()
? field_meta_.get_dim()
: 1,
array_vec)
.release();
chunks_[cid] = chunk;
row_count += chunk->RowNums();
meta_.num_rows_until_chunk_.push_back(row_count);
cid++;
array_vec);
}
} else {
auto filepath = std::filesystem::path(field_data_info.mmap_dir_path) /
// we don't know the resulting file size beforehand, thus using a separate file for each chunk.
auto filepath = std::filesystem::path(mmap_dir_path_) /
std::to_string(segment_id_) /
std::to_string(field_data_info.field_id);
auto dir = filepath.parent_path();
std::filesystem::create_directories(dir);
std::to_string(field_id_) / std::to_string(cid);
LOG_INFO("segment {} mmaping field {} chunk {} to path {}",
segment_id_,
field_id_,
cid,
filepath.string());
std::filesystem::create_directories(filepath.parent_path());
auto file = File::Open(filepath.string(), O_CREAT | O_TRUNC | O_RDWR);
std::shared_ptr<milvus::ArrowDataWrapper> r;
size_t file_offset = 0;
std::vector<std::shared_ptr<Chunk>> chunks;
while (field_data_info.arrow_reader_channel->pop(r)) {
while (channel->pop(r)) {
arrow::ArrayVector array_vec =
read_single_column_batches(r->reader);
auto chunk =
create_chunk(field_meta,
auto chunk = create_chunk(field_meta_,
IsVectorDataType(data_type) &&
!IsSparseFloatVectorDataType(data_type)
? field_meta.get_dim()
? field_meta_.get_dim()
: 1,
file,
file_offset,
array_vec)
.release();
chunks_[cid] = chunk;
row_count += chunk->RowNums();
meta_.num_rows_until_chunk_.push_back(row_count);
cid++;
file_offset += chunk->Size();
}
}
AssertInfo(row_count == field_data_info.row_count,
fmt::format("data lost while loading column {}: loaded "
"num rows {} but expected {}",
field_data_info.field_id,
row_count,
field_data_info.row_count));
}
ChunkTranslator::~ChunkTranslator() {
for (auto chunk : chunks_) {
if (chunk != nullptr) {
// let the Chunk to be deleted by the unique_ptr
auto chunk_ptr = std::unique_ptr<Chunk>(chunk);
/*file_offset*/ 0,
array_vec);
auto ok = unlink(filepath.c_str());
AssertInfo(ok == 0,
fmt::format("failed to unlink mmap data file {}, err: {}",
filepath.c_str(),
strerror(errno)));
return chunk;
}
}
}
size_t
ChunkTranslator::num_cells() const {
return chunks_.size();
return files_and_rows_.size();
}
milvus::cachinglayer::cid_t
@ -151,14 +153,9 @@ ChunkTranslator::get_cells(
std::vector<
std::pair<milvus::cachinglayer::cid_t, std::unique_ptr<milvus::Chunk>>>
cells;
cells.reserve(cids.size());
for (auto cid : cids) {
AssertInfo(chunks_[cid] != nullptr,
"ChunkTranslator::get_cells called again on cell {} of "
"CacheSlot {}.",
cid,
key_);
cells.emplace_back(cid, std::unique_ptr<milvus::Chunk>(chunks_[cid]));
chunks_[cid] = nullptr;
cells.emplace_back(cid, load_chunk(cid));
}
return cells;
}

View File

@ -17,31 +17,31 @@
#include "cachinglayer/Translator.h"
#include "cachinglayer/Utils.h"
#include "common/Chunk.h"
#include "common/type_c.h"
#include "mmap/Types.h"
namespace milvus::segcore::storagev1translator {
struct CTMeta : public milvus::cachinglayer::Meta {
std::vector<int64_t> num_rows_until_chunk_;
CTMeta(milvus::cachinglayer::StorageType storage_type)
: milvus::cachinglayer::Meta(storage_type) {
CTMeta(milvus::cachinglayer::StorageType storage_type,
CacheWarmupPolicy cache_warmup_policy,
bool support_eviction)
: milvus::cachinglayer::Meta(
storage_type, cache_warmup_policy, support_eviction) {
}
};
// This class will load all cells(Chunks) in ctor, and move them out during get_cells.
// This should be used only in storagev1(no eviction allowed), thus trying to get a
// same cell a second time will result in exception.
// For this translator each Chunk is a CacheCell, cid_t and uid_ is the same.
// For this translator each Chunk is a CacheCell, cid_t == uid_t.
class ChunkTranslator : public milvus::cachinglayer::Translator<milvus::Chunk> {
public:
ChunkTranslator(int64_t segment_id,
ChunkTranslator(
int64_t segment_id,
FieldMeta field_meta,
FieldDataInfo field_data_info,
std::vector<std::string> insert_files,
std::vector<std::pair<std::string, int64_t>>&& files_and_rows,
bool use_mmap);
~ChunkTranslator() override;
size_t
num_cells() const override;
milvus::cachinglayer::cid_t
@ -54,18 +54,23 @@ class ChunkTranslator : public milvus::cachinglayer::Translator<milvus::Chunk> {
std::pair<milvus::cachinglayer::cid_t, std::unique_ptr<milvus::Chunk>>>
get_cells(const std::vector<milvus::cachinglayer::cid_t>& cids) override;
// TODO: info other than get_cels() should all be in meta()
milvus::cachinglayer::Meta*
meta() override {
return &meta_;
}
private:
std::unique_ptr<milvus::Chunk>
load_chunk(milvus::cachinglayer::cid_t cid);
std::vector<std::pair<std::string, int64_t>> files_and_rows_;
int64_t segment_id_;
int64_t field_id_;
std::string key_;
bool use_mmap_;
std::vector<milvus::Chunk*> chunks_;
CTMeta meta_;
FieldMeta field_meta_;
std::string mmap_dir_path_;
};
} // namespace milvus::segcore::storagev1translator

View File

@ -12,6 +12,7 @@
#include "segcore/storagev1translator/DefaultValueChunkTranslator.h"
#include "common/ChunkWriter.h"
#include "segcore/Utils.h"
#include "storage/Util.h"
namespace milvus::segcore::storagev1translator {
@ -26,7 +27,11 @@ DefaultValueChunkTranslator::DefaultValueChunkTranslator(
use_mmap_(use_mmap),
field_meta_(field_meta),
meta_(use_mmap ? milvus::cachinglayer::StorageType::DISK
: milvus::cachinglayer::StorageType::MEMORY) {
: milvus::cachinglayer::StorageType::MEMORY,
milvus::segcore::getCacheWarmupPolicy(
IsVectorDataType(field_meta.get_data_type()),
/* is_index */ false),
/* support_eviction */ false) {
meta_.num_rows_until_chunk_.push_back(0);
meta_.num_rows_until_chunk_.push_back(field_data_info.row_count);
}

View File

@ -1,5 +1,6 @@
#include "segcore/storagev1translator/InterimSealedIndexTranslator.h"
#include "index/VectorMemIndex.h"
#include "segcore/Utils.h"
namespace milvus::segcore::storagev1translator {
@ -19,7 +20,11 @@ InterimSealedIndexTranslator::InterimSealedIndexTranslator(
dim_(dim),
is_sparse_(is_sparse),
index_key_(fmt::format("seg_{}_ii_{}", segment_id, field_id)),
meta_(milvus::cachinglayer::StorageType::MEMORY) {
meta_(milvus::cachinglayer::StorageType::MEMORY,
milvus::segcore::getCacheWarmupPolicy(
/* is_vector */ true,
/* is_index */ true),
/* support_eviction */ false) {
}
size_t

View File

@ -1,6 +1,7 @@
#include "segcore/storagev1translator/SealedIndexTranslator.h"
#include "index/IndexFactory.h"
#include "segcore/load_index_c.h"
#include "segcore/Utils.h"
#include <utility>
namespace milvus::segcore::storagev1translator {
@ -27,8 +28,13 @@ SealedIndexTranslator::SealedIndexTranslator(
std::to_string(load_index_info->index_id),
std::to_string(load_index_info->segment_id),
std::to_string(load_index_info->field_id)}),
// TODO : how to express a index use both memory and disk
meta_(milvus::cachinglayer::StorageType::MEMORY) {
meta_(load_index_info->enable_mmap
? milvus::cachinglayer::StorageType::DISK
: milvus::cachinglayer::StorageType::MEMORY,
milvus::segcore::getCacheWarmupPolicy(
IsVectorDataType(load_index_info->field_type),
/* is_index */ true),
/* support_eviction */ false) {
}
size_t

View File

@ -1,6 +1,7 @@
#include "segcore/storagev1translator/V1SealedIndexTranslator.h"
#include "index/IndexFactory.h"
#include "segcore/load_index_c.h"
#include "segcore/Utils.h"
#include <utility>
#include "storage/RemoteChunkManagerSingleton.h"
@ -28,7 +29,13 @@ V1SealedIndexTranslator::V1SealedIndexTranslator(
key_(fmt::format("seg_{}_si_{}",
load_index_info->segment_id,
load_index_info->field_id)),
meta_(milvus::cachinglayer::StorageType::MEMORY) {
meta_(load_index_info->enable_mmap
? milvus::cachinglayer::StorageType::DISK
: milvus::cachinglayer::StorageType::MEMORY,
milvus::segcore::getCacheWarmupPolicy(
IsVectorDataType(load_index_info->field_type),
/* is_index */ true),
/* support_eviction */ false) {
}
size_t

View File

@ -22,8 +22,11 @@ namespace milvus::segcore::storagev2translator {
struct GroupCTMeta : public milvus::cachinglayer::Meta {
std::vector<int64_t> num_rows_until_chunk_;
std::vector<int64_t> chunk_memory_size_;
GroupCTMeta(milvus::cachinglayer::StorageType storage_type)
: milvus::cachinglayer::Meta(storage_type) {
GroupCTMeta(milvus::cachinglayer::StorageType storage_type,
CacheWarmupPolicy cache_warmup_policy,
bool support_eviction)
: milvus::cachinglayer::Meta(
storage_type, cache_warmup_policy, support_eviction) {
}
};

View File

@ -31,6 +31,7 @@
#include "arrow/type_fwd.h"
#include "cachinglayer/Utils.h"
#include "common/ChunkWriter.h"
#include "segcore/Utils.h"
namespace milvus::segcore::storagev2translator {
@ -50,8 +51,14 @@ GroupChunkTranslator::GroupChunkTranslator(
use_mmap_(use_mmap),
row_group_meta_list_(row_group_meta_list),
field_id_list_(field_id_list),
meta_(use_mmap ? milvus::cachinglayer::StorageType::DISK
: milvus::cachinglayer::StorageType::MEMORY) {
meta_(
use_mmap ? milvus::cachinglayer::StorageType::DISK
: milvus::cachinglayer::StorageType::MEMORY,
// TODO(tiered storage 2): vector may be of small size and mixed with scalar, do we force it
// to use the warm up policy of scalar field?
milvus::segcore::getCacheWarmupPolicy(/* is_vector */ false,
/* is_index */ false),
/* support_eviction */ true) {
AssertInfo(insert_files_.size() == row_group_meta_list_.size(),
"Number of insert files must match number of row group metas");
meta_.num_rows_until_chunk_.push_back(0);
@ -217,6 +224,13 @@ GroupChunkTranslator::load_column_group_in_mmap() {
process_batch(table, files, file_offsets, row_counts);
}
}
for (size_t i = 0; i < files.size(); ++i) {
auto ok = unlink(files[i].c_str());
AssertInfo(ok == 0,
fmt::format("failed to unlink mmap data file {}, err: {}",
files[i].c_str(),
strerror(errno)));
}
}
void

View File

@ -220,6 +220,19 @@ SortByPath(std::vector<std::string>& paths) {
});
}
// same as SortByPath, but with pair of path and entries_nums
inline void
SortByPath(std::vector<std::pair<std::string, int64_t>>& paths) {
std::sort(
paths.begin(),
paths.end(),
[](const std::pair<std::string, int64_t>& a,
const std::pair<std::string, int64_t>& b) {
return std::stol(a.first.substr(a.first.find_last_of("/") + 1)) <
std::stol(b.first.substr(b.first.find_last_of("/") + 1));
});
}
// used only for test
inline std::shared_ptr<ArrowDataWrapper>
ConvertFieldDataToArrowDataWrapper(const FieldDataPtr& field_data) {

View File

@ -12,6 +12,7 @@
#include <gtest/gtest.h>
#include "folly/init/Init.h"
#include "common/type_c.h"
#include "test_utils/Constants.h"
#include "storage/LocalChunkManagerSingleton.h"
#include "storage/RemoteChunkManagerSingleton.h"
@ -28,8 +29,16 @@ main(int argc, char** argv) {
get_default_local_storage_config());
milvus::storage::MmapManager::GetInstance().Init(get_default_mmap_config());
static const int64_t mb = 1024 * 1024;
milvus::cachinglayer::Manager::ConfigureTieredStorage(
true, 1024 * 1024 * 1024, 1024 * 1024 * 1024);
{CacheWarmupPolicy::CacheWarmupPolicy_Disable,
CacheWarmupPolicy::CacheWarmupPolicy_Disable,
CacheWarmupPolicy::CacheWarmupPolicy_Disable,
CacheWarmupPolicy::CacheWarmupPolicy_Disable},
{1024 * mb, 1024 * mb, 1024 * mb, 1024 * mb, 1024 * mb, 1024 * mb},
true,
{10, 30});
return RUN_ALL_TESTS();
}

View File

@ -48,16 +48,10 @@ constexpr size_t kHnswEf = 128;
const MetricType kMetricType = knowhere::metric::L2;
} // namespace
enum class ConstructorType {
VectorIndex = 0,
RawData,
VectorBase,
ChunkedColumn
};
enum class ConstructorType { VectorIndex = 0, VectorBase, ChunkedColumn };
static const std::vector<ConstructorType> kConstructorTypes = {
ConstructorType::VectorIndex,
ConstructorType::RawData,
ConstructorType::VectorBase,
ConstructorType::ChunkedColumn,
};
@ -122,15 +116,6 @@ class CachedSearchIteratorTest
search_info,
bitset);
case ConstructorType::RawData:
return std::make_unique<CachedSearchIterator>(
search_dataset_,
dataset::RawDataset{0, dim_, nb_, base_dataset_.data()},
search_info,
std::map<std::string, std::string>{},
bitset,
data_type_);
case ConstructorType::VectorBase:
return std::make_unique<CachedSearchIterator>(
search_dataset_,
@ -706,16 +691,6 @@ TEST_P(CachedSearchIteratorTest, ConstructorWithInvalidParams) {
search_info,
nullptr),
SegcoreError);
} else if (std::get<0>(GetParam()) == ConstructorType::RawData) {
EXPECT_THROW(
auto iterator = std::make_unique<CachedSearchIterator>(
dataset::SearchDataset{},
dataset::RawDataset{0, dim_, nb_, base_dataset_.data()},
search_info,
std::map<std::string, std::string>{},
nullptr,
data_type_),
SegcoreError);
} else if (std::get<0>(GetParam()) == ConstructorType::VectorBase) {
EXPECT_THROW(auto iterator = std::make_unique<CachedSearchIterator>(
dataset::SearchDataset{},
@ -782,9 +757,6 @@ INSTANTIATE_TEST_SUITE_P(
case ConstructorType::VectorIndex:
constructor_type_str = "VectorIndex";
break;
case ConstructorType::RawData:
constructor_type_str = "RawData";
break;
case ConstructorType::VectorBase:
constructor_type_str = "VectorBase";
break;

View File

@ -20,6 +20,7 @@
#include "cachinglayer/Translator.h"
#include "common/Chunk.h"
#include "common/GroupChunk.h"
#include "common/type_c.h"
#include "segcore/storagev1translator/ChunkTranslator.h"
#include "segcore/storagev2translator/GroupChunkTranslator.h"
#include "cachinglayer/lrucache/DList.h"
@ -36,7 +37,10 @@ class TestChunkTranslator : public Translator<milvus::Chunk> {
: Translator<milvus::Chunk>(),
num_cells_(num_rows_per_chunk.size()),
chunks_(std::move(chunks)),
meta_(segcore::storagev1translator::CTMeta(StorageType::MEMORY)) {
meta_(segcore::storagev1translator::CTMeta(
StorageType::MEMORY,
CacheWarmupPolicy::CacheWarmupPolicy_Disable,
true)) {
meta_.num_rows_until_chunk_.reserve(num_cells_ + 1);
meta_.num_rows_until_chunk_.push_back(0);
int total_rows = 0;
@ -102,8 +106,10 @@ class TestGroupChunkTranslator : public Translator<milvus::GroupChunk> {
: Translator<milvus::GroupChunk>(),
num_cells_(num_rows_per_chunk.size()),
chunks_(std::move(chunks)),
meta_(
segcore::storagev2translator::GroupCTMeta(StorageType::MEMORY)) {
meta_(segcore::storagev2translator::GroupCTMeta(
StorageType::MEMORY,
CacheWarmupPolicy::CacheWarmupPolicy_Disable,
true)) {
meta_.num_rows_until_chunk_.reserve(num_cells_ + 1);
meta_.num_rows_until_chunk_.push_back(0);
for (int i = 0; i < num_cells_; ++i) {
@ -166,7 +172,10 @@ class TestIndexTranslator : public Translator<milvus::index::IndexBase> {
: Translator<milvus::index::IndexBase>(),
key_(key),
index_(std::move(index)),
meta_(milvus::cachinglayer::Meta(StorageType::MEMORY)) {
meta_(milvus::cachinglayer::Meta(
StorageType::MEMORY,
CacheWarmupPolicy::CacheWarmupPolicy_Disable,
false)) {
}
~TestIndexTranslator() override = default;

View File

@ -27,7 +27,6 @@ class MockListNode : public ListNode {
cid_t cid = 0)
: ListNode(dlist, size), mock_key_(fmt::format("{}:{}", key, cid)) {
ON_CALL(*this, clear_data).WillByDefault([this]() {
// Default clear_data calls unload() by default in base, mimic if needed
unload();
state_ = State::NOT_LOADED;
});

View File

@ -46,8 +46,10 @@ class MockTranslator : public Translator<TestCell> {
bool for_concurrent_test = false)
: uid_to_cid_map_(std::move(uid_to_cid_map)),
key_(key),
meta_(storage_type),
num_unique_cids_(cell_sizes.size()) {
meta_(
storage_type, CacheWarmupPolicy::CacheWarmupPolicy_Disable, true),
num_unique_cids_(cell_sizes.size()),
for_concurrent_test_(for_concurrent_test) {
cid_set_.reserve(cell_sizes.size());
cell_sizes_.reserve(cell_sizes.size());
for (const auto& pair : cell_sizes) {
@ -217,8 +219,9 @@ class CacheSlotTest : public ::testing::Test {
void
SetUp() override {
auto limit = ResourceUsage{MEMORY_LIMIT, DISK_LIMIT};
dlist_ = std::make_unique<DList>(
ResourceUsage{MEMORY_LIMIT, DISK_LIMIT}, DList::TouchConfig{});
limit, limit, limit, EvictionConfig{10, 600});
auto temp_translator_uptr = std::make_unique<MockTranslator>(
cell_sizes_, uid_to_cid_map_, SLOT_KEY, StorageType::MEMORY);
@ -509,9 +512,13 @@ TEST_F(CacheSlotTest, TranslatorReturnsExtraCells) {
TEST_F(CacheSlotTest, EvictionTest) {
// Sizes: 0:50, 1:150, 2:100, 3:200
ResourceUsage NEW_LIMIT = ResourceUsage(300, 0);
EXPECT_TRUE(dlist_->UpdateLimit(NEW_LIMIT));
EXPECT_EQ(DListTestFriend::get_max_memory(*dlist_), NEW_LIMIT);
ResourceUsage new_limit = ResourceUsage(300, 0);
ResourceUsage new_high_watermark = ResourceUsage(250, 0);
ResourceUsage new_low_watermark = ResourceUsage(200, 0);
EXPECT_TRUE(dlist_->UpdateLimit(new_limit));
dlist_->UpdateHighWatermark(new_high_watermark);
dlist_->UpdateLowWatermark(new_low_watermark);
EXPECT_EQ(DListTestFriend::get_max_memory(*dlist_), new_limit);
std::vector<cl_uid_t> uids_012 = {10, 20, 30};
std::vector<cid_t> cids_012 = {0, 1, 2};
@ -554,7 +561,7 @@ TEST_F(CacheSlotTest, EvictionTest) {
// Verify eviction happened
ResourceUsage used_after_evict1 = DListTestFriend::get_used_memory(*dlist_);
EXPECT_LE(used_after_evict1.memory_bytes, NEW_LIMIT.memory_bytes);
EXPECT_LE(used_after_evict1.memory_bytes, new_limit.memory_bytes);
EXPECT_GE(used_after_evict1.memory_bytes, size_3.memory_bytes);
EXPECT_LT(
used_after_evict1.memory_bytes,
@ -569,18 +576,26 @@ TEST_P(CacheSlotConcurrentTest, ConcurrentAccessMultipleSlots) {
// Slot 2 Cells: 0-4 (Sizes: 55, 65, 75, 85, 95) -> Total 375
// Total potential size = 350 + 375 = 725
// Set limit lower than total potential size to force eviction
ResourceUsage NEW_LIMIT = ResourceUsage(600, 0);
ASSERT_TRUE(dlist_->UpdateLimit(NEW_LIMIT));
ResourceUsage new_limit = ResourceUsage(700, 0);
ResourceUsage new_high_watermark = ResourceUsage(650, 0);
ResourceUsage new_low_watermark = ResourceUsage(600, 0);
ASSERT_TRUE(dlist_->UpdateLimit(new_limit));
dlist_->UpdateHighWatermark(new_high_watermark);
dlist_->UpdateLowWatermark(new_low_watermark);
EXPECT_EQ(DListTestFriend::get_max_memory(*dlist_).memory_bytes,
NEW_LIMIT.memory_bytes);
new_limit.memory_bytes);
// 1. Setup CacheSlots sharing dlist_
std::vector<std::pair<cid_t, int64_t>> cell_sizes_1 = {
{0, 50}, {1, 60}, {2, 70}, {3, 80}, {4, 90}};
std::unordered_map<cl_uid_t, cid_t> uid_map_1 = {
{1000, 0}, {1001, 1}, {1002, 2}, {1003, 3}, {1004, 4}};
auto translator_1_ptr = std::make_unique<MockTranslator>(
cell_sizes_1, uid_map_1, "slot1", StorageType::MEMORY, true);
auto translator_1_ptr =
std::make_unique<MockTranslator>(cell_sizes_1,
uid_map_1,
"slot1",
StorageType::MEMORY,
/*for_concurrent_test*/ true);
MockTranslator* translator_1 = translator_1_ptr.get();
auto slot1 = std::make_shared<CacheSlot<TestCell>>(
std::move(translator_1_ptr), dlist_.get());
@ -589,8 +604,12 @@ TEST_P(CacheSlotConcurrentTest, ConcurrentAccessMultipleSlots) {
{0, 55}, {1, 65}, {2, 75}, {3, 85}, {4, 95}};
std::unordered_map<cl_uid_t, cid_t> uid_map_2 = {
{2000, 0}, {2001, 1}, {2002, 2}, {2003, 3}, {2004, 4}};
auto translator_2_ptr = std::make_unique<MockTranslator>(
cell_sizes_2, uid_map_2, "slot2", StorageType::MEMORY, true);
auto translator_2_ptr =
std::make_unique<MockTranslator>(cell_sizes_2,
uid_map_2,
"slot2",
StorageType::MEMORY,
/*for_concurrent_test*/ true);
MockTranslator* translator_2 = translator_2_ptr.get();
auto slot2 = std::make_shared<CacheSlot<TestCell>>(
std::move(translator_2_ptr), dlist_.get());
@ -722,11 +741,14 @@ TEST_P(CacheSlotConcurrentTest, ConcurrentAccessMultipleSlots) {
{3000, 0}, {3001, 1}, {3002, 2}, {3003, 3}, {3004, 4}};
std::vector<cl_uid_t> slot3_uids = {3000, 3001, 3002, 3003, 3004};
auto create_new_slot3 = [&]() {
auto translator_3_ptr = std::make_unique<MockTranslator>(
cell_sizes_3, uid_map_3, "slot3", StorageType::MEMORY, true);
auto translator_3_ptr =
std::make_unique<MockTranslator>(cell_sizes_3,
uid_map_3,
"slot3",
StorageType::MEMORY,
/*for_concurrent_test*/ true);
auto sl = std::make_shared<CacheSlot<TestCell>>(
std::move(translator_3_ptr), dlist_ptr);
// std::cout << "Created new SSSSSSslot3 at " << (void*)sl.get() << std::endl;
return sl;
};
std::shared_ptr<CacheSlot<TestCell>> slot3 = create_new_slot3();
@ -783,7 +805,6 @@ TEST_P(CacheSlotConcurrentTest, ConcurrentAccessMultipleSlots) {
}
if (ops_since_recreate >= recreate_interval) {
// std::cout << "Destroying SSSSSSslot3 at " << (void*)slot3.get() << std::endl;
slot3 = nullptr;
int sleep_ms = recreate_sleep_dist(gen);
std::this_thread::sleep_for(
@ -828,9 +849,9 @@ TEST_P(CacheSlotConcurrentTest, ConcurrentAccessMultipleSlots) {
// bonus cell may cause memory usage to exceed the limit
if (!with_bonus_cells) {
EXPECT_LE(final_memory_usage.memory_bytes, NEW_LIMIT.memory_bytes)
EXPECT_LE(final_memory_usage.memory_bytes, new_limit.memory_bytes)
<< "Final memory usage (" << final_memory_usage.memory_bytes
<< ") exceeds the limit (" << NEW_LIMIT.memory_bytes
<< ") exceeds the limit (" << new_limit.memory_bytes
<< ") after concurrent access.";
}

View File

@ -7,6 +7,7 @@
#include "cachinglayer/lrucache/DList.h"
#include "cachinglayer/Utils.h"
#include "common/EasyAssert.h"
#include "mock_list_node.h"
#include "cachinglayer_test_utils.h"
@ -18,14 +19,21 @@ using DLF = DListTestFriend;
class DListTest : public ::testing::Test {
protected:
ResourceUsage initial_limit{100, 50};
DList::TouchConfig touch_config{{std::chrono::seconds(1)}};
// Set watermarks relative to the limit
ResourceUsage low_watermark{80, 40}; // 80%
ResourceUsage high_watermark{90, 45}; // 90%
// Use a very long interval to disable background eviction for most tests
EvictionConfig eviction_config_{10, // cache_touch_window (10 ms)
10}; // eviction_interval (10 ms)
std::unique_ptr<DList> dlist;
// Keep track of nodes to prevent them from being deleted prematurely
std::vector<std::shared_ptr<MockListNode>> managed_nodes;
void
SetUp() override {
dlist = std::make_unique<DList>(initial_limit, touch_config);
dlist = std::make_unique<DList>(
initial_limit, low_watermark, high_watermark, eviction_config_);
managed_nodes.clear();
}
@ -70,19 +78,6 @@ class DListTest : public ::testing::Test {
get_used_memory() const {
return DLF::get_used_memory(*dlist);
}
// void
// DLF::verify_list(dlist.get(), std::vector<MockListNode*> nodes) const {
// EXPECT_EQ(nodes.front(), DLF::get_tail(*dlist));
// EXPECT_EQ(nodes.back(), DLF::get_head(*dlist));
// for (size_t i = 0; i < nodes.size() - 1; ++i) {
// auto current = nodes[i];
// auto expected_prev = i == 0 ? nullptr : nodes[i - 1];
// auto expected_next = i == nodes.size() - 1 ? nullptr : nodes[i + 1];
// EXPECT_EQ(current->test_get_prev(), expected_prev);
// EXPECT_EQ(current->test_get_next(), expected_next);
// }
// }
};
TEST_F(DListTest, Initialization) {
@ -188,8 +183,8 @@ TEST_F(DListTest, UpdateLimitToZero) {
}
TEST_F(DListTest, UpdateLimitInvalid) {
EXPECT_THROW(dlist->UpdateLimit({-10, 0}), std::invalid_argument);
EXPECT_THROW(dlist->UpdateLimit({0, -5}), std::invalid_argument);
EXPECT_THROW(dlist->UpdateLimit({-10, 0}), milvus::SegcoreError);
EXPECT_THROW(dlist->UpdateLimit({0, -5}), milvus::SegcoreError);
}
TEST_F(DListTest, ReserveMemorySufficient) {
@ -210,9 +205,9 @@ TEST_F(DListTest, ReserveMemoryRequiresEviction) {
EXPECT_CALL(*node1, clear_data()).Times(1);
EXPECT_CALL(*node2, clear_data()).Times(0);
ResourceUsage reserve_size{20, 20};
// Current used: 90, 40. Limit: 100, 50. Reserve: 20, 20.
// Potential total: 110, 60. Need to free >= 10 mem, 10 disk.
// Limit: 100, 50, current usage: 90, 40, reserve: 20, 15.
// Potential total: 110, 55. Need to free to low watermark 80, 40.
ResourceUsage reserve_size{20, 15};
// Evicting node1 ({40, 15}) is sufficient.
EXPECT_TRUE(dlist->reserveMemory(reserve_size));
@ -309,8 +304,9 @@ TEST_F(DListTest, TouchItemRefreshWindow) {
}
DLF::verify_list(dlist.get(), {node2, node1});
std::this_thread::sleep_for(touch_config.refresh_window +
std::chrono::milliseconds(100));
// Use eviction_config from dlist
std::this_thread::sleep_for(dlist->eviction_config().cache_touch_window +
std::chrono::milliseconds(10));
{
std::unique_lock node_lock(node1->test_get_mutex());
@ -318,8 +314,9 @@ TEST_F(DListTest, TouchItemRefreshWindow) {
}
DLF::verify_list(dlist.get(), {node2, node1});
std::this_thread::sleep_for(touch_config.refresh_window +
std::chrono::milliseconds(100));
// Use eviction_config from dlist
std::this_thread::sleep_for(dlist->eviction_config().cache_touch_window +
std::chrono::milliseconds(10));
{
std::unique_lock node_lock(node2->test_get_mutex());
@ -340,11 +337,8 @@ TEST_F(DListTest, releaseMemory) {
}
TEST_F(DListTest, ReserveMemoryEvictOnlyMemoryNeeded) {
initial_limit = {100, 100};
EXPECT_TRUE(dlist->UpdateLimit(initial_limit));
MockListNode* node_disk_only = add_and_load_node({0, 50}, "disk_only");
MockListNode* node_mixed = add_and_load_node({50, 50}, "mixed");
MockListNode* node_disk_only = add_and_load_node({0, 20}, "disk_only");
MockListNode* node_mixed = add_and_load_node({50, 10}, "mixed");
ResourceUsage usage_disk = node_disk_only->size();
ResourceUsage usage_mixed = node_mixed->size();
DLF::verify_list(dlist.get(), {node_disk_only, node_mixed});
@ -353,8 +347,8 @@ TEST_F(DListTest, ReserveMemoryEvictOnlyMemoryNeeded) {
EXPECT_CALL(*node_disk_only, clear_data()).Times(0);
EXPECT_CALL(*node_mixed, clear_data()).Times(1);
// node_disk_only is at tail, but it contains no memory, thus evicting it does not help.
// We need to evict node_mixed to free up memory.
// node_disk_only is at tail, but it contains no memory, and disk usage is below low watermark,
// thus evicting it does not help. We need to evict node_mixed to free up memory.
ResourceUsage reserve_size{60, 0};
EXPECT_TRUE(dlist->reserveMemory(reserve_size));
@ -363,33 +357,29 @@ TEST_F(DListTest, ReserveMemoryEvictOnlyMemoryNeeded) {
}
TEST_F(DListTest, ReserveMemoryEvictOnlyDiskNeeded) {
initial_limit = {100, 100};
EXPECT_TRUE(dlist->UpdateLimit(initial_limit));
MockListNode* node_mem_only = add_and_load_node({50, 0}, "mem_only");
MockListNode* node_mixed = add_and_load_node({50, 50}, "mixed");
ResourceUsage usage_mem = node_mem_only->size();
MockListNode* node_memory_only = add_and_load_node({40, 0}, "memory_only");
MockListNode* node_mixed = add_and_load_node({20, 25}, "mixed");
ResourceUsage usage_memory = node_memory_only->size();
ResourceUsage usage_mixed = node_mixed->size();
DLF::verify_list(dlist.get(), {node_mem_only, node_mixed});
ASSERT_EQ(get_used_memory(), usage_mem + usage_mixed);
DLF::verify_list(dlist.get(), {node_memory_only, node_mixed});
ASSERT_EQ(get_used_memory(), usage_memory + usage_mixed);
EXPECT_CALL(*node_mem_only, clear_data()).Times(0);
EXPECT_CALL(*node_memory_only, clear_data()).Times(0);
EXPECT_CALL(*node_mixed, clear_data()).Times(1);
ResourceUsage reserve_size{0, 60};
// node_memory_only is at tail, but it contains no disk, and memory usage is below low watermark,
// thus evicting it does not help. We need to evict node_mixed to free up disk.
ResourceUsage reserve_size{0, 30};
EXPECT_TRUE(dlist->reserveMemory(reserve_size));
EXPECT_EQ(get_used_memory(), usage_mem + reserve_size);
DLF::verify_list(dlist.get(), {node_mem_only});
EXPECT_EQ(get_used_memory(), usage_memory + reserve_size);
DLF::verify_list(dlist.get(), {node_memory_only});
}
TEST_F(DListTest, ReserveMemoryEvictBothNeeded) {
initial_limit = {100, 100};
EXPECT_TRUE(dlist->UpdateLimit(initial_limit));
MockListNode* node1 = add_and_load_node({30, 10}, "node1");
MockListNode* node2 = add_and_load_node({10, 30}, "node2");
MockListNode* node3 = add_and_load_node({50, 50}, "node3");
MockListNode* node1 = add_and_load_node({30, 5}, "node1");
MockListNode* node2 = add_and_load_node({10, 15}, "node2");
MockListNode* node3 = add_and_load_node({50, 25}, "node3");
ResourceUsage usage1 = node1->size();
ResourceUsage usage2 = node2->size();
ResourceUsage usage3 = node3->size();
@ -400,13 +390,56 @@ TEST_F(DListTest, ReserveMemoryEvictBothNeeded) {
EXPECT_CALL(*node2, clear_data()).Times(1);
EXPECT_CALL(*node3, clear_data()).Times(0);
ResourceUsage reserve_size{50, 50};
ResourceUsage reserve_size{10, 15};
EXPECT_TRUE(dlist->reserveMemory(reserve_size));
EXPECT_EQ(get_used_memory(), usage3 + reserve_size);
DLF::verify_list(dlist.get(), {node3});
}
TEST_F(DListTest, ReserveToAboveLowWatermarkNoEviction) {
// initial 40, 20
MockListNode* node1 = add_and_load_node({30, 5}, "node1");
MockListNode* node2 = add_and_load_node({10, 15}, "node2");
ResourceUsage usage1 = node1->size();
ResourceUsage usage2 = node2->size();
DLF::verify_list(dlist.get(), {node1, node2});
ASSERT_EQ(get_used_memory(), usage1 + usage2);
// after reserve, 45, 22, end up in 85, 42, no eviction
ResourceUsage reserve_size{45, 22};
EXPECT_TRUE(dlist->reserveMemory(reserve_size));
EXPECT_EQ(get_used_memory(), usage1 + usage2 + reserve_size);
DLF::verify_list(dlist.get(), {node1, node2});
}
TEST_F(DListTest, ReserveToAboveHighWatermarkNoEvictionThenAutoEviction) {
// initial 40, 20
MockListNode* node1 = add_and_load_node({30, 15}, "node1");
MockListNode* node2 = add_and_load_node({10, 5}, "node2");
ResourceUsage usage1 = node1->size();
ResourceUsage usage2 = node2->size();
DLF::verify_list(dlist.get(), {node1, node2});
ASSERT_EQ(get_used_memory(), usage1 + usage2);
// after reserve, 55, 26, end up in 95, 46, above high watermark, no eviction
ResourceUsage reserve_size{55, 26};
EXPECT_TRUE(dlist->reserveMemory(reserve_size));
EXPECT_EQ(get_used_memory(), usage1 + usage2 + reserve_size);
DLF::verify_list(dlist.get(), {node1, node2});
// wait for background eviction to run, current usage 95, 46, above high watermark.
// reserved 55, 26 is considered pinned, thus evict node 1, resulting in 65, 31, below low watermark
EXPECT_CALL(*node1, clear_data()).Times(1);
std::this_thread::sleep_for(dlist->eviction_config().eviction_interval +
std::chrono::milliseconds(10));
EXPECT_EQ(get_used_memory(), usage2 + reserve_size);
DLF::verify_list(dlist.get(), {node2});
}
TEST_F(DListTest, ReserveMemoryFailsAllPinned) {
MockListNode* node1 = add_and_load_node({40, 15}, "key1", 0, 1);
MockListNode* node2 = add_and_load_node({50, 25}, "key2", 0, 1);
@ -495,8 +528,9 @@ TEST_F(DListTest, TouchItemHeadOutsideWindow) {
MockListNode* node2 = add_and_load_node({10, 0}, "key2");
DLF::verify_list(dlist.get(), {node1, node2});
std::this_thread::sleep_for(touch_config.refresh_window +
std::chrono::milliseconds(100));
// Use eviction_config from dlist
std::this_thread::sleep_for(dlist->eviction_config().cache_touch_window +
std::chrono::milliseconds(10));
{
std::unique_lock node_lock(node2->test_get_mutex());
@ -608,3 +642,64 @@ TEST_F(DListTest, NodeInListDestroyed) {
EXPECT_EQ(get_used_memory(), memory_before_destroy - usage1);
DLF::verify_list(dlist.get(), {node2});
}
// New tests for watermark updates
TEST_F(DListTest, UpdateWatermarksValid) {
ResourceUsage new_low{70, 30};
ResourceUsage new_high{85, 40};
// Check initial watermarks (optional, could use friend class if needed)
// EXPECT_EQ(DLF::get_low_watermark(*dlist), low_watermark);
// EXPECT_EQ(DLF::get_high_watermark(*dlist), high_watermark);
EXPECT_NO_THROW(dlist->UpdateLowWatermark(new_low));
EXPECT_NO_THROW(dlist->UpdateHighWatermark(new_high));
// Verify new watermarks (requires friend class accessors)
// EXPECT_EQ(DLF::get_low_watermark(*dlist), new_low);
// EXPECT_EQ(DLF::get_high_watermark(*dlist), new_high);
// Verify no change in list state or usage
EXPECT_TRUE(dlist->IsEmpty());
EXPECT_EQ(get_used_memory(), ResourceUsage{});
}
TEST_F(DListTest, UpdateWatermarksInvalid) {
EXPECT_THROW(dlist->UpdateLowWatermark({-10, 0}), milvus::SegcoreError);
EXPECT_THROW(dlist->UpdateLowWatermark({0, -5}), milvus::SegcoreError);
EXPECT_THROW(dlist->UpdateHighWatermark({-10, 0}), milvus::SegcoreError);
EXPECT_THROW(dlist->UpdateHighWatermark({0, -5}), milvus::SegcoreError);
}
TEST_F(DListTest, ReserveMemoryUsesLowWatermark) {
// Set up: Limit 100/100, Low 80/80, High 90/90
initial_limit = {100, 100};
low_watermark = {80, 80};
high_watermark = {90, 90};
EXPECT_TRUE(dlist->UpdateLimit(initial_limit));
dlist->UpdateLowWatermark(low_watermark);
dlist->UpdateHighWatermark(high_watermark);
// Add nodes totaling 95/95 usage (above high watermark)
MockListNode* node1 = add_and_load_node({45, 45}, "node1"); // Tail
MockListNode* node2 = add_and_load_node({50, 50}, "node2"); // Head
ResourceUsage usage1 = node1->size();
ResourceUsage usage2 = node2->size();
DLF::verify_list(dlist.get(), {node1, node2});
ASSERT_EQ(get_used_memory(), usage1 + usage2); // 95, 95
EXPECT_CALL(*node1, clear_data())
.Times(1); // Evict node1 to get below low watermark
EXPECT_CALL(*node2, clear_data()).Times(0);
// Reserve 10/10. Current usage 95/95. New potential usage 105/105.
// Max limit 100/100. Min eviction needed: 5/5.
// Expected eviction (target low watermark): 95/95 + 10/10 - 80/80 = 25/25.
// Evicting node1 (45/45) satisfies both min and expected.
ResourceUsage reserve_size{10, 10};
EXPECT_TRUE(dlist->reserveMemory(reserve_size));
// Expected usage: usage2 + reserve_size = (50,50) + (10,10) = (60,60)
EXPECT_EQ(get_used_memory(), usage2 + reserve_size);
DLF::verify_list(dlist.get(), {node2});
}

View File

@ -261,9 +261,15 @@ TEST_F(ChunkedColumnGroupTest, ProxyChunkColumn) {
EXPECT_EQ(proxy_int64->num_chunks(), 1);
EXPECT_FALSE(proxy_int64->IsNullable());
EXPECT_NE(proxy_int64->DataOfChunk(0).get(), nullptr);
EXPECT_NE(proxy_int64->ValueAt(0), nullptr);
EXPECT_TRUE(proxy_int64->IsValid(0));
EXPECT_TRUE(proxy_int64->IsValid(0, 0));
int64_t offset = 0;
proxy_int64->BulkValueAt(
[&](const char* value, size_t size) { EXPECT_NE(value, nullptr); },
&offset,
1);
proxy_int64->BulkIsValid(
[&](bool is_valid, size_t offset) { EXPECT_TRUE(is_valid); },
&offset,
1);
// Test string proxy
auto proxy_string = std::make_shared<ProxyChunkColumn>(

View File

@ -148,12 +148,12 @@ TEST_P(TestGroupChunkTranslator, TestWithMmap) {
std::string mmap_dir = std::to_string(segment_id_);
EXPECT_TRUE(std::filesystem::exists(mmap_dir));
// Verify each field has a corresponding file
for (size_t i = 0; i < field_id_list.size(); ++i) {
auto field_id = field_id_list.Get(i);
std::string field_file = mmap_dir + "/" + std::to_string(field_id);
EXPECT_TRUE(std::filesystem::exists(field_file));
}
// DO NOT Verify each field has a corresponding file: files are unlinked immediately after being mmaped.
// for (size_t i = 0; i < field_id_list.size(); ++i) {
// auto field_id = field_id_list.Get(i);
// std::string field_file = mmap_dir + "/" + std::to_string(field_id);
// EXPECT_TRUE(std::filesystem::exists(field_file));
// }
}
}

View File

@ -111,7 +111,6 @@ func (sd *shardDelegator) ProcessInsert(insertRecords map[int64]*InsertData) {
DeltaPosition: insertData.StartPosition,
Level: datapb.SegmentLevel_L1,
},
nil,
)
if err != nil {
log.Error("failed to create new segment",

View File

@ -68,7 +68,6 @@ func (s *ManagerSuite) SetupTest() {
InsertChannel: s.channels[i],
Level: s.levels[i],
},
nil,
)
s.Require().NoError(err)
s.segments = append(s.segments, segment)

View File

@ -98,7 +98,6 @@ func (suite *RetrieveSuite) SetupTest() {
InsertChannel: fmt.Sprintf("by-dev-rootcoord-dml_0_%dv0", suite.collectionID),
Level: datapb.SegmentLevel_Legacy,
},
nil,
)
suite.Require().NoError(err)
@ -127,7 +126,6 @@ func (suite *RetrieveSuite) SetupTest() {
InsertChannel: fmt.Sprintf("by-dev-rootcoord-dml_0_%dv0", suite.collectionID),
Level: datapb.SegmentLevel_Legacy,
},
nil,
)
suite.Require().NoError(err)

View File

@ -57,6 +57,8 @@ func (suite *SearchSuite) SetupTest() {
chunkManagerFactory := storage.NewChunkManagerFactoryWithParam(paramtable.Get())
suite.chunkManager, _ = chunkManagerFactory.NewPersistentStorageChunkManager(ctx)
initcore.InitRemoteChunkManager(paramtable.Get())
initcore.InitLocalChunkManager(suite.T().Name())
initcore.InitMmapManager(paramtable.Get())
suite.collectionID = 100
suite.partitionID = 10
@ -88,7 +90,6 @@ func (suite *SearchSuite) SetupTest() {
InsertChannel: fmt.Sprintf("by-dev-rootcoord-dml_0_%dv0", suite.collectionID),
Level: datapb.SegmentLevel_Legacy,
},
nil,
)
suite.Require().NoError(err)
@ -117,7 +118,6 @@ func (suite *SearchSuite) SetupTest() {
InsertChannel: fmt.Sprintf("by-dev-rootcoord-dml_0_%dv0", suite.collectionID),
Level: datapb.SegmentLevel_Legacy,
},
nil,
)
suite.Require().NoError(err)

View File

@ -29,8 +29,6 @@ import "C"
import (
"context"
"fmt"
"strings"
"sync"
"time"
"unsafe"
@ -296,7 +294,6 @@ type LocalSegment struct {
lastDeltaTimestamp *atomic.Uint64
fields *typeutil.ConcurrentMap[int64, *FieldInfo]
fieldIndexes *typeutil.ConcurrentMap[int64, *IndexedFieldInfo] // indexID -> IndexedFieldInfo
warmupDispatcher *AsyncWarmupDispatcher
fieldJSONStats []int64
}
@ -305,7 +302,6 @@ func NewSegment(ctx context.Context,
segmentType SegmentType,
version int64,
loadInfo *querypb.SegmentLoadInfo,
warmupDispatcher *AsyncWarmupDispatcher,
) (Segment, error) {
log := log.Ctx(ctx)
/*
@ -367,7 +363,6 @@ func NewSegment(ctx context.Context,
memSize: atomic.NewInt64(-1),
rowNum: atomic.NewInt64(-1),
insertCount: atomic.NewInt64(0),
warmupDispatcher: warmupDispatcher,
}
if err := segment.initializeSegment(); err != nil {
@ -1090,15 +1085,10 @@ func (s *LocalSegment) innerLoadIndex(ctx context.Context,
return nil
}
// 4.
mmapChunkCache := paramtable.Get().QueryNodeCfg.MmapChunkCache.GetAsBool()
s.WarmupChunkCache(ctx, indexInfo.GetFieldID(), mmapChunkCache)
warmupChunkCacheSpan := tr.RecordSpan()
log.Info("Finish loading index",
zap.Duration("newLoadIndexInfoSpan", newLoadIndexInfoSpan),
zap.Duration("appendLoadIndexInfoSpan", appendLoadIndexInfoSpan),
zap.Duration("updateIndexInfoSpan", updateIndexInfoSpan),
zap.Duration("warmupChunkCacheSpan", warmupChunkCacheSpan),
)
return nil
})
@ -1224,62 +1214,6 @@ func (s *LocalSegment) UpdateIndexInfo(ctx context.Context, indexInfo *querypb.F
return nil
}
func (s *LocalSegment) WarmupChunkCache(ctx context.Context, fieldID int64, mmapEnabled bool) {
log := log.Ctx(ctx).With(
zap.Int64("collectionID", s.Collection()),
zap.Int64("partitionID", s.Partition()),
zap.Int64("segmentID", s.ID()),
zap.Int64("fieldID", fieldID),
zap.Bool("mmapEnabled", mmapEnabled),
)
if !s.ptrLock.PinIf(state.IsNotReleased) {
return
}
defer s.ptrLock.Unpin()
var status C.CStatus
warmingUp := strings.ToLower(paramtable.Get().QueryNodeCfg.ChunkCacheWarmingUp.GetValue())
switch warmingUp {
case "sync":
GetWarmupPool().Submit(func() (any, error) {
cFieldID := C.int64_t(fieldID)
cMmapEnabled := C.bool(mmapEnabled)
status = C.WarmupChunkCache(s.ptr, cFieldID, cMmapEnabled)
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":
task := func() (any, error) {
// failed to wait for state update, return directly
if !s.ptrLock.BlockUntilDataLoadedOrReleased() {
return nil, nil
}
if s.PinIfNotReleased() != nil {
return nil, nil
}
defer s.Unpin()
cFieldID := C.int64_t(fieldID)
cMmapEnabled := C.bool(mmapEnabled)
status = C.WarmupChunkCache(s.ptr, cFieldID, cMmapEnabled)
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
}
s.warmupDispatcher.AddTask(task)
default:
// no warming up
}
}
func (s *LocalSegment) UpdateFieldRawDataSize(ctx context.Context, numRows int64, fieldBinlog *datapb.FieldBinlog) error {
var status C.CStatus
fieldID := fieldBinlog.FieldID
@ -1445,58 +1379,6 @@ func (s *LocalSegment) indexNeedLoadRawData(schema *schemapb.CollectionSchema, i
return !typeutil.IsVectorType(fieldSchema.DataType) && s.HasRawData(indexInfo.IndexInfo.FieldID), nil
}
type (
WarmupTask = func() (any, error)
AsyncWarmupDispatcher struct {
mu sync.RWMutex
tasks []WarmupTask
notify chan struct{}
}
)
func NewWarmupDispatcher() *AsyncWarmupDispatcher {
return &AsyncWarmupDispatcher{
notify: make(chan struct{}, 1),
}
}
func (d *AsyncWarmupDispatcher) AddTask(task func() (any, error)) {
d.mu.Lock()
d.tasks = append(d.tasks, task)
d.mu.Unlock()
select {
case d.notify <- struct{}{}:
default:
}
}
func (d *AsyncWarmupDispatcher) Run(ctx context.Context) {
for {
select {
case <-ctx.Done():
return
case <-d.notify:
d.mu.RLock()
tasks := make([]WarmupTask, len(d.tasks))
copy(tasks, d.tasks)
d.mu.RUnlock()
for _, task := range tasks {
select {
case <-ctx.Done():
return
default:
GetWarmupPool().Submit(task)
}
}
d.mu.Lock()
d.tasks = d.tasks[len(tasks):]
d.mu.Unlock()
}
}
}
func (s *LocalSegment) GetFieldJSONIndexStats() []int64 {
return s.fieldJSONStats
}

View File

@ -176,15 +176,12 @@ func NewLoader(
duf := NewDiskUsageFetcher(ctx)
go duf.Start()
warmupDispatcher := NewWarmupDispatcher()
go warmupDispatcher.Run(ctx)
loader := &segmentLoader{
manager: manager,
cm: cm,
loadingSegments: typeutil.NewConcurrentMap[int64, *loadResult](),
committedResourceNotifier: syncutil.NewVersionedNotifier(),
duf: duf,
warmupDispatcher: warmupDispatcher,
}
return loader
@ -228,7 +225,6 @@ type segmentLoader struct {
committedResourceNotifier *syncutil.VersionedNotifier
duf *diskUsageFetcher
warmupDispatcher *AsyncWarmupDispatcher
}
var _ Loader = (*segmentLoader)(nil)
@ -311,7 +307,6 @@ func (loader *segmentLoader) Load(ctx context.Context,
segmentType,
version,
loadInfo,
loader.warmupDispatcher,
)
if err != nil {
log.Warn("load segment failed when create new segment",
@ -1556,8 +1551,8 @@ func getResourceUsageEstimateOfSegment(schema *schemapb.CollectionSchema, loadIn
fieldIndexInfo.GetBuildID())
}
if metricType != metric.BM25 {
mmapChunkCache := paramtable.Get().QueryNodeCfg.MmapChunkCache.GetAsBool()
if mmapChunkCache {
mmapVectorField := paramtable.Get().QueryNodeCfg.MmapVectorField.GetAsBool()
if mmapVectorField {
segmentDiskSize += binlogSize
} else {
segmentMemorySize += binlogSize

View File

@ -628,64 +628,6 @@ func (suite *SegmentLoaderSuite) TestLoadWithMmap() {
suite.NoError(err)
}
func (suite *SegmentLoaderSuite) TestPatchEntryNum() {
ctx := context.Background()
msgLength := 100
segmentID := suite.segmentID
binlogs, statsLogs, err := mock_segcore.SaveBinLog(ctx,
suite.collectionID,
suite.partitionID,
segmentID,
msgLength,
suite.schema,
suite.chunkManager,
)
suite.NoError(err)
vecFields := funcutil.GetVecFieldIDs(suite.schema)
indexInfo, err := mock_segcore.GenAndSaveIndex(
suite.collectionID,
suite.partitionID,
segmentID,
vecFields[0],
msgLength,
mock_segcore.IndexFaissIVFFlat,
metric.L2,
suite.chunkManager,
)
suite.NoError(err)
loadInfo := &querypb.SegmentLoadInfo{
SegmentID: segmentID,
PartitionID: suite.partitionID,
CollectionID: suite.collectionID,
BinlogPaths: binlogs,
Statslogs: statsLogs,
IndexInfos: []*querypb.FieldIndexInfo{indexInfo},
NumOfRows: int64(msgLength),
InsertChannel: fmt.Sprintf("by-dev-rootcoord-dml_0_%dv0", suite.collectionID),
}
// mock legacy binlog entry num is zero case
for _, fieldLog := range binlogs {
for _, binlog := range fieldLog.GetBinlogs() {
binlog.EntriesNum = 0
}
}
segments, err := suite.loader.Load(ctx, suite.collectionID, SegmentTypeSealed, 0, loadInfo)
suite.Require().NoError(err)
suite.Require().Equal(1, len(segments))
segment := segments[0]
info := segment.GetIndex(vecFields[0])
suite.Require().NotNil(info)
for _, binlog := range info[0].FieldBinlog.GetBinlogs() {
suite.Greater(binlog.EntriesNum, int64(0))
}
}
func (suite *SegmentLoaderSuite) TestRunOutMemory() {
ctx := context.Background()
paramtable.Get().Save(paramtable.Get().QueryNodeCfg.OverloadedMemoryThresholdPercentage.Key, "0")

View File

@ -5,11 +5,8 @@ import (
"fmt"
"path/filepath"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/suite"
"go.uber.org/atomic"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/mocks/util/mock_segcore"
@ -94,7 +91,6 @@ func (suite *SegmentSuite) SetupTest() {
},
},
},
nil,
)
suite.Require().NoError(err)
@ -126,7 +122,6 @@ func (suite *SegmentSuite) SetupTest() {
InsertChannel: fmt.Sprintf("by-dev-rootcoord-dml_0_%dv0", suite.collectionID),
Level: datapb.SegmentLevel_Legacy,
},
nil,
)
suite.Require().NoError(err)
@ -227,22 +222,3 @@ func (suite *SegmentSuite) TestSegmentReleased() {
func TestSegment(t *testing.T) {
suite.Run(t, new(SegmentSuite))
}
func TestWarmupDispatcher(t *testing.T) {
d := NewWarmupDispatcher()
ctx := context.Background()
go d.Run(ctx)
completed := atomic.NewInt64(0)
taskCnt := 10000
for i := 0; i < taskCnt; i++ {
d.AddTask(func() (any, error) {
completed.Inc()
return nil, nil
})
}
assert.Eventually(t, func() bool {
return completed.Load() == int64(taskCnt)
}, 10*time.Second, time.Second)
}

View File

@ -271,11 +271,72 @@ func (node *QueryNode) InitSegcore() error {
return err
}
tieredStorageEnabledGlobally := C.bool(paramtable.Get().QueryNodeCfg.TieredStorageEnableGlobal.GetAsBool())
tieredStorageMemoryLimit := C.int64_t(paramtable.Get().QueryNodeCfg.TieredStorageMemoryAllocationRatio.GetAsFloat() * float64(hardware.GetMemoryCount()))
diskCapacity := paramtable.Get().QueryNodeCfg.DiskCapacityLimit.GetAsInt64()
tieredStorageDiskLimit := C.int64_t(paramtable.Get().QueryNodeCfg.TieredStorageDiskAllocationRatio.GetAsFloat() * float64(diskCapacity))
C.ConfigureTieredStorage(tieredStorageEnabledGlobally, tieredStorageMemoryLimit, tieredStorageDiskLimit)
// init tiered storage
scalarFieldCacheWarmupPolicy, err := segcore.ConvertCacheWarmupPolicy(paramtable.Get().QueryNodeCfg.TieredWarmupScalarField.GetValue())
if err != nil {
return err
}
vectorFieldCacheWarmupPolicy, err := segcore.ConvertCacheWarmupPolicy(paramtable.Get().QueryNodeCfg.TieredWarmupVectorField.GetValue())
if err != nil {
return err
}
scalarIndexCacheWarmupPolicy, err := segcore.ConvertCacheWarmupPolicy(paramtable.Get().QueryNodeCfg.TieredWarmupScalarIndex.GetValue())
if err != nil {
return err
}
vectorIndexCacheWarmupPolicy, err := segcore.ConvertCacheWarmupPolicy(paramtable.Get().QueryNodeCfg.TieredWarmupVectorIndex.GetValue())
if err != nil {
return err
}
osMemBytes := hardware.GetMemoryCount()
osDiskBytes := paramtable.Get().QueryNodeCfg.DiskCapacityLimit.GetAsInt64()
memoryLowWatermarkRatio := paramtable.Get().QueryNodeCfg.TieredMemoryLowWatermarkRatio.GetAsFloat()
memoryHighWatermarkRatio := paramtable.Get().QueryNodeCfg.TieredMemoryHighWatermarkRatio.GetAsFloat()
memoryMaxRatio := paramtable.Get().QueryNodeCfg.TieredMemoryMaxRatio.GetAsFloat()
diskLowWatermarkRatio := paramtable.Get().QueryNodeCfg.TieredDiskLowWatermarkRatio.GetAsFloat()
diskHighWatermarkRatio := paramtable.Get().QueryNodeCfg.TieredDiskHighWatermarkRatio.GetAsFloat()
diskMaxRatio := paramtable.Get().QueryNodeCfg.TieredDiskMaxRatio.GetAsFloat()
if memoryLowWatermarkRatio > memoryHighWatermarkRatio {
return errors.New("memoryLowWatermarkRatio should not be greater than memoryHighWatermarkRatio")
}
if memoryHighWatermarkRatio > memoryMaxRatio {
return errors.New("memoryHighWatermarkRatio should not be greater than memoryMaxRatio")
}
if memoryMaxRatio >= 1 {
return errors.New("memoryMaxRatio should not be greater than 1")
}
if diskLowWatermarkRatio > diskHighWatermarkRatio {
return errors.New("diskLowWatermarkRatio should not be greater than diskHighWatermarkRatio")
}
if diskHighWatermarkRatio > diskMaxRatio {
return errors.New("diskHighWatermarkRatio should not be greater than diskMaxRatio")
}
if diskMaxRatio >= 1 {
return errors.New("diskMaxRatio should not be greater than 1")
}
memoryLowWatermarkBytes := C.int64_t(memoryLowWatermarkRatio * float64(osMemBytes))
memoryHighWatermarkBytes := C.int64_t(memoryHighWatermarkRatio * float64(osMemBytes))
memoryMaxBytes := C.int64_t(memoryMaxRatio * float64(osMemBytes))
diskLowWatermarkBytes := C.int64_t(diskLowWatermarkRatio * float64(osDiskBytes))
diskHighWatermarkBytes := C.int64_t(diskHighWatermarkRatio * float64(osDiskBytes))
diskMaxBytes := C.int64_t(diskMaxRatio * float64(osDiskBytes))
evictionEnabled := C.bool(paramtable.Get().QueryNodeCfg.TieredEvictionEnabled.GetAsBool())
cacheTouchWindowMs := C.int64_t(paramtable.Get().QueryNodeCfg.TieredCacheTouchWindowMs.GetAsInt64())
evictionIntervalMs := C.int64_t(paramtable.Get().QueryNodeCfg.TieredEvictionIntervalMs.GetAsInt64())
C.ConfigureTieredStorage(C.CacheWarmupPolicy(scalarFieldCacheWarmupPolicy),
C.CacheWarmupPolicy(vectorFieldCacheWarmupPolicy),
C.CacheWarmupPolicy(scalarIndexCacheWarmupPolicy),
C.CacheWarmupPolicy(vectorIndexCacheWarmupPolicy),
memoryLowWatermarkBytes, memoryHighWatermarkBytes, memoryMaxBytes,
diskLowWatermarkBytes, diskHighWatermarkBytes, diskMaxBytes,
evictionEnabled, cacheTouchWindowMs, evictionIntervalMs)
initcore.InitTraceConfig(paramtable.Get())
C.InitExecExpressionFunctionFactory()
@ -432,7 +493,6 @@ func (node *QueryNode) Start() error {
mmapVectorField := paramtable.Get().QueryNodeCfg.MmapVectorField.GetAsBool()
mmapScalarIndex := paramtable.Get().QueryNodeCfg.MmapScalarIndex.GetAsBool()
mmapScalarField := paramtable.Get().QueryNodeCfg.MmapScalarField.GetAsBool()
mmapChunkCache := paramtable.Get().QueryNodeCfg.MmapChunkCache.GetAsBool()
node.UpdateStateCode(commonpb.StateCode_Healthy)
@ -446,7 +506,6 @@ func (node *QueryNode) Start() error {
zap.Bool("mmapVectorField", mmapVectorField),
zap.Bool("mmapScalarIndex", mmapScalarIndex),
zap.Bool("mmapScalarField", mmapScalarField),
zap.Bool("mmapChunkCache", mmapChunkCache),
)
})

View File

@ -238,7 +238,6 @@ func (suite *QueryNodeSuite) TestStop() {
Level: datapb.SegmentLevel_Legacy,
InsertChannel: fmt.Sprintf("by-dev-rootcoord-dml_0_%dv0", 1),
},
nil,
)
suite.NoError(err)
suite.node.manager.Segment.Put(context.Background(), segments.SegmentTypeSealed, segment)

View File

@ -293,3 +293,16 @@ func (s *cSegmentImpl) FinishLoad() error {
func (s *cSegmentImpl) Release() {
C.DeleteSegment(s.ptr)
}
func ConvertCacheWarmupPolicy(policy string) (C.CacheWarmupPolicy, error) {
switch policy {
case "sync":
return C.CacheWarmupPolicy_Sync, nil
case "async":
return C.CacheWarmupPolicy_Async, nil
case "disable":
return C.CacheWarmupPolicy_Disable, nil
default:
return C.CacheWarmupPolicy_Disable, fmt.Errorf("invalid Tiered Storage cache warmup policy: %s", policy)
}
}

View File

@ -2686,10 +2686,21 @@ type queryNodeConfig struct {
InterimIndexMemExpandRate ParamItem `refreshable:"false"`
InterimIndexBuildParallelRate ParamItem `refreshable:"false"`
MultipleChunkedEnable ParamItem `refreshable:"false"` // Deprecated
// TODO this should be refreshable?
TieredStorageEnableGlobal ParamItem `refreshable:"false"`
TieredStorageMemoryAllocationRatio ParamItem `refreshable:"false"`
TieredStorageDiskAllocationRatio ParamItem `refreshable:"false"`
// TODO(tiered storage 2) this should be refreshable?
TieredWarmupScalarField ParamItem `refreshable:"false"`
TieredWarmupScalarIndex ParamItem `refreshable:"false"`
TieredWarmupVectorField ParamItem `refreshable:"false"`
TieredWarmupVectorIndex ParamItem `refreshable:"false"`
TieredMemoryLowWatermarkRatio ParamItem `refreshable:"false"`
TieredMemoryHighWatermarkRatio ParamItem `refreshable:"false"`
TieredMemoryMaxRatio ParamItem `refreshable:"false"`
TieredDiskLowWatermarkRatio ParamItem `refreshable:"false"`
TieredDiskHighWatermarkRatio ParamItem `refreshable:"false"`
TieredDiskMaxRatio ParamItem `refreshable:"false"`
TieredEvictionEnabled ParamItem `refreshable:"false"`
TieredCacheTouchWindowMs ParamItem `refreshable:"false"`
TieredEvictionIntervalMs ParamItem `refreshable:"false"`
KnowhereScoreConsistency ParamItem `refreshable:"false"`
@ -2713,7 +2724,6 @@ type queryNodeConfig struct {
MmapVectorIndex ParamItem `refreshable:"false"`
MmapScalarField ParamItem `refreshable:"false"`
MmapScalarIndex ParamItem `refreshable:"false"`
MmapChunkCache ParamItem `refreshable:"false"`
GrowingMmapEnabled ParamItem `refreshable:"false"`
FixedFileSizeForMmapManager ParamItem `refreshable:"false"`
MaxMmapDiskPercentageForMmapManager ParamItem `refreshable:"false"`
@ -2727,9 +2737,7 @@ type queryNodeConfig struct {
IndexOffsetCacheEnabled ParamItem `refreshable:"true"`
// chunk cache
ReadAheadPolicy ParamItem `refreshable:"false"`
ChunkCacheWarmingUp ParamItem `refreshable:"true"`
MaxReceiveChanSize ParamItem `refreshable:"false"`
MaxUnsolvedQueueSize ParamItem `refreshable:"true"`
@ -2827,46 +2835,181 @@ func (p *queryNodeConfig) init(base *BaseTable) {
}
p.StatsPublishInterval.Init(base.mgr)
p.TieredStorageEnableGlobal = ParamItem{
Key: "queryNode.segcore.tieredStorage.enableGlobally",
p.TieredWarmupScalarField = ParamItem{
Key: "queryNode.segcore.tieredStorage.warmup.scalarField",
Version: "2.6.0",
DefaultValue: "sync",
Doc: `options: sync, async, disable.
Specifies the timing for warming up the Tiered Storage cache.
- "sync": data will be loaded into the cache before a segment is considered loaded.
- "async": data will be loaded asynchronously into the cache after a segment is loaded.
- "disable": data will not be proactively loaded into the cache, and loaded only if needed by search/query tasks.
Defaults to "sync", except for vector field which defaults to "disable".`,
Export: true,
}
p.TieredWarmupScalarField.Init(base.mgr)
p.TieredWarmupScalarIndex = ParamItem{
Key: "queryNode.segcore.tieredStorage.warmup.scalarIndex",
Version: "2.6.0",
DefaultValue: "sync",
Export: true,
}
p.TieredWarmupScalarIndex.Init(base.mgr)
p.TieredWarmupVectorField = ParamItem{
Key: "queryNode.segcore.tieredStorage.warmup.vectorField",
Version: "2.6.0",
DefaultValue: "disable",
Doc: `cache warmup for vector field raw data is by default disabled.`,
Export: true,
}
p.TieredWarmupVectorField.Init(base.mgr)
p.TieredWarmupVectorIndex = ParamItem{
Key: "queryNode.segcore.tieredStorage.warmup.vectorIndex",
Version: "2.6.0",
DefaultValue: "sync",
Export: true,
}
p.TieredWarmupVectorIndex.Init(base.mgr)
p.TieredEvictionEnabled = ParamItem{
Key: "queryNode.segcore.tieredStorage.evictionEnabled",
Version: "2.6.0",
DefaultValue: "false",
Doc: "Whether or not to turn on Tiered Storage globally in this cluster.",
Doc: `Enable eviction for Tiered Storage. Defaults to false.
Note that if eviction is enabled, cache data loaded during sync/async warmup is also subject to eviction.`,
Export: true,
}
p.TieredStorageEnableGlobal.Init(base.mgr)
p.TieredEvictionEnabled.Init(base.mgr)
p.TieredStorageMemoryAllocationRatio = ParamItem{
Key: "queryNode.segcore.tieredStorage.memoryAllocationRatio",
p.TieredMemoryLowWatermarkRatio = ParamItem{
Key: "queryNode.segcore.tieredStorage.memoryLowWatermarkRatio",
Version: "2.6.0",
DefaultValue: "0.5",
DefaultValue: "0.6",
Formatter: func(v string) string {
ratio := getAsFloat(v)
if ratio < 0 || ratio > 1 {
return "0.5"
return "0.6"
}
return fmt.Sprintf("%f", ratio)
},
Doc: "The ratio of memory allocation for Tiered Storage.",
Doc: `If evictionEnabled is true, a background thread will run every evictionIntervalMs to determine if an
eviction is necessary and the amount of data to evict from memory/disk.
- The max ratio is the max amount of memory/disk that can be used for cache.
- If the current memory/disk usage exceeds the high watermark, an eviction will be triggered to evict data from memory/disk
until the memory/disk usage is below the low watermark.`,
Export: true,
}
p.TieredStorageMemoryAllocationRatio.Init(base.mgr)
p.TieredMemoryLowWatermarkRatio.Init(base.mgr)
p.TieredStorageDiskAllocationRatio = ParamItem{
Key: "queryNode.segcore.tieredStorage.diskAllocationRatio",
p.TieredMemoryHighWatermarkRatio = ParamItem{
Key: "queryNode.segcore.tieredStorage.memoryHighWatermarkRatio",
Version: "2.6.0",
DefaultValue: "0.5",
DefaultValue: "0.8",
Formatter: func(v string) string {
ratio := getAsFloat(v)
if ratio < 0 || ratio > 1 {
return "0.5"
return "0.8"
}
return fmt.Sprintf("%f", ratio)
},
Doc: "The ratio of disk allocation for Tiered Storage.",
Export: true,
}
p.TieredStorageDiskAllocationRatio.Init(base.mgr)
p.TieredMemoryHighWatermarkRatio.Init(base.mgr)
p.TieredMemoryMaxRatio = ParamItem{
Key: "queryNode.segcore.tieredStorage.memoryMaxRatio",
Version: "2.6.0",
DefaultValue: "0.9",
Formatter: func(v string) string {
ratio := getAsFloat(v)
if ratio < 0 || ratio > 1 {
return "0.9"
}
return fmt.Sprintf("%f", ratio)
},
Export: true,
}
p.TieredMemoryMaxRatio.Init(base.mgr)
p.TieredDiskLowWatermarkRatio = ParamItem{
Key: "queryNode.segcore.tieredStorage.diskLowWatermarkRatio",
Version: "2.6.0",
DefaultValue: "0.6",
Formatter: func(v string) string {
ratio := getAsFloat(v)
if ratio < 0 || ratio > 1 {
return "0.6"
}
return fmt.Sprintf("%f", ratio)
},
Export: true,
}
p.TieredDiskLowWatermarkRatio.Init(base.mgr)
p.TieredDiskHighWatermarkRatio = ParamItem{
Key: "queryNode.segcore.tieredStorage.diskHighWatermarkRatio",
Version: "2.6.0",
DefaultValue: "0.8",
Formatter: func(v string) string {
ratio := getAsFloat(v)
if ratio < 0 || ratio > 1 {
return "0.8"
}
return fmt.Sprintf("%f", ratio)
},
Export: true,
}
p.TieredDiskHighWatermarkRatio.Init(base.mgr)
p.TieredDiskMaxRatio = ParamItem{
Key: "queryNode.segcore.tieredStorage.diskMaxRatio",
Version: "2.6.0",
DefaultValue: "0.9",
Formatter: func(v string) string {
ratio := getAsFloat(v)
if ratio < 0 || ratio > 1 {
return "0.9"
}
return fmt.Sprintf("%f", ratio)
},
Export: true,
}
p.TieredDiskMaxRatio.Init(base.mgr)
p.TieredCacheTouchWindowMs = ParamItem{
Key: "queryNode.segcore.tieredStorage.cacheTouchWindowMs",
Version: "2.6.0",
DefaultValue: "3000",
Formatter: func(v string) string {
window := getAsInt64(v)
if window < 0 {
return "3000"
}
return fmt.Sprintf("%d", window)
},
Doc: "Min interval in milliseconds for update a cache entry's hotness. If a cache entry is frequently accessed, it will be moved to the head of the cache at most once every cacheTouchWindowMs.",
Export: false,
}
p.TieredCacheTouchWindowMs.Init(base.mgr)
p.TieredEvictionIntervalMs = ParamItem{
Key: "queryNode.segcore.tieredStorage.evictionIntervalMs",
Version: "2.6.0",
DefaultValue: "10000",
Formatter: func(v string) string {
window := getAsInt64(v)
if window < 0 {
return "10000"
}
return fmt.Sprintf("%d", window)
},
Doc: "Interval in milliseconds to run periodic eviction.",
Export: false,
}
p.TieredEvictionIntervalMs.Init(base.mgr)
p.KnowhereThreadPoolSize = ParamItem{
Key: "queryNode.segcore.knowhereThreadPoolNumRatio",
@ -3094,15 +3237,6 @@ This defaults to true, indicating that Milvus creates temporary index for growin
}
p.MmapScalarIndex.Init(base.mgr)
p.MmapChunkCache = ParamItem{
Key: "queryNode.mmap.chunkCache",
Version: "2.4.12",
DefaultValue: "true",
Doc: "Enable mmap for chunk cache (raw vector retrieving).",
Export: true,
}
p.MmapChunkCache.Init(base.mgr)
p.GrowingMmapEnabled = ParamItem{
Key: "queryNode.mmap.growingMmapEnabled",
Version: "2.4.6",
@ -3193,20 +3327,6 @@ However, this optimization may come at the cost of a slight decrease in query la
}
p.ReadAheadPolicy.Init(base.mgr)
p.ChunkCacheWarmingUp = ParamItem{
Key: "queryNode.cache.warmup",
Version: "2.3.6",
DefaultValue: "disable",
Doc: `options: async, sync, disable.
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 "disable" original vector data will only be loaded into the chunk cache during search/query.`,
Export: true,
}
p.ChunkCacheWarmingUp.Init(base.mgr)
p.MaxReceiveChanSize = ParamItem{
Key: "queryNode.scheduler.receiveChanSize",
Version: "2.0.0",

View File

@ -412,7 +412,6 @@ func TestComponentParam(t *testing.T) {
// chunk cache
assert.Equal(t, "willneed", Params.ReadAheadPolicy.GetValue())
assert.Equal(t, "disable", Params.ChunkCacheWarmingUp.GetValue())
// test small indexNlist/NProbe default
params.Remove("queryNode.segcore.smallIndex.nlist")
@ -485,7 +484,6 @@ func TestComponentParam(t *testing.T) {
assert.Equal(t, "/var/lib/milvus/data/mmap", Params.MmapDirPath.GetValue())
assert.Equal(t, true, Params.MmapChunkCache.GetAsBool())
assert.Equal(t, 60*time.Second, Params.DiskSizeFetchInterval.GetAsDuration(time.Second))
})