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

@ -366,7 +366,7 @@ queryCoord:
rowCountFactor: 0.4 # the row count weight used when balancing segments among queryNodes
segmentCountFactor: 0.4 # the segment count weight used when balancing segments among queryNodes
globalSegmentCountFactor: 0.1 # the segment count weight used when balancing segments among queryNodes
# the channel count weight used when balancing channels among queryNodes,
# the channel count weight used when balancing channels among queryNodes,
# A higher value reduces the likelihood of assigning channels from the same collection to the same QueryNode. Set to 1 to disable this feature.
collectionChannelCountFactor: 10
segmentCountMaxSteps: 50 # segment count based plan generator max steps
@ -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,21 +463,13 @@ 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.
# 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.
growingMmapEnabled: false
fixedFileSizeForMmapAlloc: 1 # tmp file size for mmap chunk manager
@ -549,7 +563,7 @@ dataCoord:
# The max idle time of segment in seconds, 10*60.
maxIdleTime: 600
minSizeFromIdleToSealed: 16 # The min size in MB of segment which can be idle from sealed.
# The max number of binlog (which is equal to the binlog file num of primary key) for one segment,
# The max number of binlog (which is equal to the binlog file num of primary key) for one segment,
# the segment will be sealed if the number of binlog file reaches to max value.
maxBinlogFileNumber: 32
smallProportion: 0.5 # The segment is considered as "small segment" when its # of rows is smaller than
@ -562,13 +576,13 @@ dataCoord:
expansionRate: 1.25
sealPolicy:
channel:
# The size threshold in MB, if the total size of growing segments of each shard
# The size threshold in MB, if the total size of growing segments of each shard
# exceeds this threshold, the largest growing segment will be sealed.
growingSegmentsMemSize: 4096
# If the total entry number of l0 logs of each shard
# If the total entry number of l0 logs of each shard
# exceeds this threshold, the earliest growing segments will be sealed.
blockingL0EntryNum: 5000000
# The size threshold in MB, if the total entry number of l0 logs of each shard
# The size threshold in MB, if the total entry number of l0 logs of each shard
# exceeds this threshold, the earliest growing segments will be sealed.
blockingL0SizeInMB: 64
autoUpgradeSegmentIndex: false # whether auto upgrade segment index to index engine's version
@ -578,7 +592,7 @@ dataCoord:
# if param targetVecIndexVersion is not set, the default value is -1, which means no target vec index version, then the vector index will be aligned with index engine's version
targetVecIndexVersion: -1
segmentFlushInterval: 2 # the minimal interval duration(unit: Seconds) between flusing operation on same segment
# Switch value to control if to enable segment compaction.
# Switch value to control if to enable segment compaction.
# Compaction merges small-size segments into a large segment, and clears the entities deleted beyond the rentention duration of Time Travel.
enableCompaction: true
compaction:
@ -586,7 +600,7 @@ dataCoord:
# This configuration takes effect only when dataCoord.enableCompaction is set as true.
enableAutoCompaction: true
indexBasedCompaction: true
# compaction task prioritizer, options: [default, level, mix].
# compaction task prioritizer, options: [default, level, mix].
# default is FIFO.
# level is prioritized by level: L0 compactions first, then mix compactions, then clustering compactions.
# mix is prioritized by level: mix compactions first, then L0 compactions, then clustering compactions.
@ -786,7 +800,7 @@ msgChannel:
# Configures the system log output.
log:
# Milvus log level. Option: debug, info, warn, error, panic, and fatal.
# Milvus log level. Option: debug, info, warn, error, panic, and fatal.
# It is recommended to use debug level under test and development environments, and info level in production environment.
level: info
file:
@ -903,8 +917,8 @@ common:
storage:
scheme: s3
enablev2: false
# Whether to disable the internal time messaging mechanism for the system.
# If disabled (set to false), the system will not allow DML operations, including insertion, deletion, queries, and searches.
# Whether to disable the internal time messaging mechanism for the system.
# If disabled (set to false), the system will not allow DML operations, including insertion, deletion, queries, and searches.
# This helps Milvus-CDC synchronize incremental data
ttMsgEnabled: true
traceLogMode: 0 # trace request info
@ -1163,23 +1177,23 @@ streamingNode:
# Any configuration related to the streaming service.
streaming:
walBalancer:
# The interval of balance task trigger at background, 1 min by default.
# The interval of balance task trigger at background, 1 min by default.
# It's ok to set it into duration string, such as 30s or 1m30s, see time.ParseDuration
triggerInterval: 1m
# The initial interval of balance task trigger backoff, 50 ms by default.
# The initial interval of balance task trigger backoff, 50 ms by default.
# It's ok to set it into duration string, such as 30s or 1m30s, see time.ParseDuration
backoffInitialInterval: 50ms
backoffMultiplier: 2 # The multiplier of balance task trigger backoff, 2 by default
balancePolicy:
name: vchannelFair # The multiplier of balance task trigger backoff, 2 by default
vchannelFair:
# The weight of pchannel count in vchannelFair balance policy,
# The weight of pchannel count in vchannelFair balance policy,
# the pchannel count will more evenly distributed if the weight is greater, 0.4 by default
pchannelWeight: 0.4
# The weight of vchannel count in vchannelFair balance policy,
# The weight of vchannel count in vchannelFair balance policy,
# the vchannel count will more evenly distributed if the weight is greater, 0.3 by default
vchannelWeight: 0.3
# The weight of affinity in vchannelFair balance policy,
# The weight of affinity in vchannelFair balance policy,
# the fewer VChannels belonging to the same Collection between two PChannels, the higher the affinity,
# the vchannel of one collection will more evenly distributed if the weight is greater, 0.01 by default
antiAffinityWeight: 0.01
@ -1187,7 +1201,7 @@ streaming:
# the balance result will be ignored, the lower tolerance, the sensitive rebalance, 0.01 by default
rebalanceTolerance: 0.01
# Indicates how many pchannels will be considered as a batch for rebalancing,
# the larger step, more aggressive and accurate rebalance,
# the larger step, more aggressive and accurate rebalance,
# it also determine the depth of depth first search method that is used to find the best balance result, 3 by default
rebalanceMaxStep: 3
walBroadcaster:

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);
}
}
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));
});
});
// 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 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) {
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);
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();
manager.warmup_policies_ = warmup_policies;
manager.cache_limit_ = cache_limit;
manager.evictionEnabled_ = evictionEnabled;
if (!evictionEnabled) {
LOG_INFO(
"Tiered Storage manager is configured with disabled eviction");
return;
}
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 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");
}
return true;
"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={}}",
memory_bytes,
file_bytes);
return fmt::format(
"memory {} bytes ({:.2} GB), disk {} bytes ({:.2} GB)",
memory_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)) {
return false;
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)) {
null_count++;
continue;
}
std::string val;
if constexpr (std::is_same_v<T, ProxyChunkColumn>) {
val = var_column.template RawAt<std::string>(i);
} else {
val = var_column.RawAt(i);
}
if (val < min_string) {
min_string = val;
}
if (val > max_string) {
max_string = val;
}
}
// 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};
var_column.BulkRawStringAt(
[&](std::string_view value, size_t offset, bool is_valid) {
if (!is_valid) {
null_count++;
return;
}
if (!has_first_valid) {
min_string = value;
max_string = value;
has_first_valid = true;
} else {
if (value < min_string) {
min_string = value;
}
if (value > max_string) {
max_string = value;
}
}
});
// 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);
auto ca =
SemiInlineGet(slot_->PinCells({static_cast<cid_t>(chunk_id)}));
auto chunk = ca->get_cell_of(chunk_id);
return chunk->isValid(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);
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)
.output_data();
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);
auto chunk = group_chunk.get()->GetChunk(field_id_);
return chunk->isValid(offset);
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_);
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 [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);
}
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)
.output_data();
}
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);
for (int64_t i = 0; i < count; ++i) {
auto offset = seg_offsets[i];
dst->at(i) = std::move(T(field->RawAt(offset)));
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];
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);
auto dst = dst_vec + i * element_sizeof;
memcpy(dst, src, element_sizeof);
}
field->BulkValueAt(
[&](const char* value, size_t i) {
auto dst = dst_vec + i * 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,
const int64_t* seg_offsets,
int64_t count,
google::protobuf::RepeatedPtrField<T>* dst_raw);
bulk_subscript_ptr_impl(
ChunkedColumnInterface* field,
const int64_t* seg_offsets,
int64_t count,
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,
const int64_t* seg_offsets,
int64_t count,
google::protobuf::RepeatedPtrField<T>* dst) const;
bulk_subscript_ptr_impl(
const VectorBase* vec_raw,
const int64_t* seg_offsets,
int64_t count,
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,
FieldMeta field_meta,
FieldDataInfo field_data_info,
std::vector<std::string> insert_files,
bool use_mmap)
ChunkTranslator::ChunkTranslator(
int64_t segment_id,
FieldMeta field_meta,
FieldDataInfo field_data_info,
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)),
"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;
: 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");
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,
IsVectorDataType(data_type) &&
!IsSparseFloatVectorDataType(data_type)
? 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++;
return create_chunk(field_meta_,
IsVectorDataType(data_type) &&
!IsSparseFloatVectorDataType(data_type)
? field_meta_.get_dim()
: 1,
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,
IsVectorDataType(data_type) &&
!IsSparseFloatVectorDataType(data_type)
? 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);
auto chunk = create_chunk(field_meta_,
IsVectorDataType(data_type) &&
!IsSparseFloatVectorDataType(data_type)
? field_meta_.get_dim()
: 1,
file,
/*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,30 +17,30 @@
#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,
FieldMeta field_meta,
FieldDataInfo field_data_info,
std::vector<std::string> insert_files,
bool use_mmap);
~ChunkTranslator() override;
ChunkTranslator(
int64_t segment_id,
FieldMeta field_meta,
FieldDataInfo field_data_info,
std::vector<std::pair<std::string, int64_t>>&& files_and_rows,
bool use_mmap);
size_t
num_cells() const override;
@ -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)
/*
@ -364,10 +360,9 @@ func NewSegment(ctx context.Context,
fields: typeutil.NewConcurrentMap[int64, *FieldInfo](),
fieldIndexes: typeutil.NewConcurrentMap[int64, *IndexedFieldInfo](),
memSize: atomic.NewInt64(-1),
rowNum: atomic.NewInt64(-1),
insertCount: atomic.NewInt64(0),
warmupDispatcher: warmupDispatcher,
memSize: atomic.NewInt64(-1),
rowNum: atomic.NewInt64(-1),
insertCount: atomic.NewInt64(0),
}
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
@ -227,8 +224,7 @@ type segmentLoader struct {
committedResource LoadResource
committedResourceNotifier *syncutil.VersionedNotifier
duf *diskUsageFetcher
warmupDispatcher *AsyncWarmupDispatcher
duf *diskUsageFetcher
}
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

@ -827,8 +827,8 @@ like the old password verification when updating the credential`,
Key: "common.ttMsgEnabled",
Version: "2.3.2",
DefaultValue: "true",
Doc: `Whether to disable the internal time messaging mechanism for the system.
If disabled (set to false), the system will not allow DML operations, including insertion, deletion, queries, and searches.
Doc: `Whether to disable the internal time messaging mechanism for the system.
If disabled (set to false), the system will not allow DML operations, including insertion, deletion, queries, and searches.
This helps Milvus-CDC synchronize incremental data`,
Export: true,
}
@ -1255,7 +1255,7 @@ func (l *logConfig) init(base *BaseTable) {
Key: "log.level",
DefaultValue: "info",
Version: "2.0.0",
Doc: `Milvus log level. Option: debug, info, warn, error, panic, and fatal.
Doc: `Milvus log level. Option: debug, info, warn, error, panic, and fatal.
It is recommended to use debug level under test and development environments, and info level in production environment.`,
Export: true,
}
@ -1854,7 +1854,7 @@ please adjust in embedded Milvus: false`,
Key: "proxy.workloadToleranceFactor",
Version: "2.4.12",
DefaultValue: "0.1",
Doc: `tolerance factor for query node workload difference, default to 10%, which means if query node's workload diff is higher than this factor,
Doc: `tolerance factor for query node workload difference, default to 10%, which means if query node's workload diff is higher than this factor,
proxy will compute each querynode's workload score, and assign request to the lowest workload node; otherwise, it will assign request to the node by round robin`,
}
p.WorkloadToleranceFactor.Init(base.mgr)
@ -2202,7 +2202,7 @@ If this parameter is set false, Milvus simply searches the growing segments with
Version: "2.4.18",
DefaultValue: "10",
PanicIfEmpty: true,
Doc: `the channel count weight used when balancing channels among queryNodes,
Doc: `the channel count weight used when balancing channels among queryNodes,
A higher value reduces the likelihood of assigning channels from the same collection to the same QueryNode. Set to 1 to disable this feature.`,
Export: true,
}
@ -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"`
ReadAheadPolicy ParamItem `refreshable:"false"`
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: "false",
Doc: "Whether or not to turn on Tiered Storage globally in this cluster.",
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.TieredStorageEnableGlobal.Init(base.mgr)
p.TieredWarmupScalarIndex.Init(base.mgr)
p.TieredStorageMemoryAllocationRatio = ParamItem{
Key: "queryNode.segcore.tieredStorage.memoryAllocationRatio",
p.TieredWarmupVectorField = ParamItem{
Key: "queryNode.segcore.tieredStorage.warmup.vectorField",
Version: "2.6.0",
DefaultValue: "0.5",
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: `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.TieredEvictionEnabled.Init(base.mgr)
p.TieredMemoryLowWatermarkRatio = ParamItem{
Key: "queryNode.segcore.tieredStorage.memoryLowWatermarkRatio",
Version: "2.6.0",
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,22 +3237,13 @@ 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",
DefaultValue: "false",
FallbackKeys: []string{"queryNode.growingMmapEnabled"},
Doc: `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.
Doc: `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.`,
Export: true,
}
@ -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",
@ -3870,7 +3990,7 @@ The max idle time of segment in seconds, 10*60.`,
Key: "dataCoord.segment.maxBinlogFileNumber",
Version: "2.2.0",
DefaultValue: "32",
Doc: `The max number of binlog (which is equal to the binlog file num of primary key) for one segment,
Doc: `The max number of binlog (which is equal to the binlog file num of primary key) for one segment,
the segment will be sealed if the number of binlog file reaches to max value.`,
Export: true,
}
@ -3880,7 +4000,7 @@ the segment will be sealed if the number of binlog file reaches to max value.`,
Key: "dataCoord.sealPolicy.channel.growingSegmentsMemSize",
Version: "2.4.6",
DefaultValue: "4096",
Doc: `The size threshold in MB, if the total size of growing segments of each shard
Doc: `The size threshold in MB, if the total size of growing segments of each shard
exceeds this threshold, the largest growing segment will be sealed.`,
Export: true,
}
@ -3890,7 +4010,7 @@ exceeds this threshold, the largest growing segment will be sealed.`,
Key: "dataCoord.sealPolicy.channel.blockingL0EntryNum",
Version: "2.5.7",
DefaultValue: "5000000",
Doc: `If the total entry number of l0 logs of each shard
Doc: `If the total entry number of l0 logs of each shard
exceeds this threshold, the earliest growing segments will be sealed.`,
Export: true,
}
@ -3900,7 +4020,7 @@ exceeds this threshold, the earliest growing segments will be sealed.`,
Key: "dataCoord.sealPolicy.channel.blockingL0SizeInMB",
Version: "2.5.7",
DefaultValue: "64",
Doc: `The size threshold in MB, if the total entry number of l0 logs of each shard
Doc: `The size threshold in MB, if the total entry number of l0 logs of each shard
exceeds this threshold, the earliest growing segments will be sealed.`,
Export: true,
}
@ -3910,7 +4030,7 @@ exceeds this threshold, the earliest growing segments will be sealed.`,
Key: "dataCoord.enableCompaction",
Version: "2.0.0",
DefaultValue: "true",
Doc: `Switch value to control if to enable segment compaction.
Doc: `Switch value to control if to enable segment compaction.
Compaction merges small-size segments into a large segment, and clears the entities deleted beyond the rentention duration of Time Travel.`,
Export: true,
}
@ -3938,7 +4058,7 @@ This configuration takes effect only when dataCoord.enableCompaction is set as t
Key: "dataCoord.compaction.taskPrioritizer",
Version: "2.5.0",
DefaultValue: "default",
Doc: `compaction task prioritizer, options: [default, level, mix].
Doc: `compaction task prioritizer, options: [default, level, mix].
default is FIFO.
level is prioritized by level: L0 compactions first, then mix compactions, then clustering compactions.
mix is prioritized by level: mix compactions first, then L0 compactions, then clustering compactions.`,
@ -5279,7 +5399,7 @@ func (p *streamingConfig) init(base *BaseTable) {
p.WALBalancerTriggerInterval = ParamItem{
Key: "streaming.walBalancer.triggerInterval",
Version: "2.6.0",
Doc: `The interval of balance task trigger at background, 1 min by default.
Doc: `The interval of balance task trigger at background, 1 min by default.
It's ok to set it into duration string, such as 30s or 1m30s, see time.ParseDuration`,
DefaultValue: "1m",
Export: true,
@ -5288,7 +5408,7 @@ It's ok to set it into duration string, such as 30s or 1m30s, see time.ParseDura
p.WALBalancerBackoffInitialInterval = ParamItem{
Key: "streaming.walBalancer.backoffInitialInterval",
Version: "2.6.0",
Doc: `The initial interval of balance task trigger backoff, 50 ms by default.
Doc: `The initial interval of balance task trigger backoff, 50 ms by default.
It's ok to set it into duration string, such as 30s or 1m30s, see time.ParseDuration`,
DefaultValue: "50ms",
Export: true,
@ -5315,7 +5435,7 @@ It's ok to set it into duration string, such as 30s or 1m30s, see time.ParseDura
p.WALBalancerPolicyVChannelFairPChannelWeight = ParamItem{
Key: "streaming.walBalancer.balancePolicy.vchannelFair.pchannelWeight",
Version: "2.6.0",
Doc: `The weight of pchannel count in vchannelFair balance policy,
Doc: `The weight of pchannel count in vchannelFair balance policy,
the pchannel count will more evenly distributed if the weight is greater, 0.4 by default`,
DefaultValue: "0.4",
Export: true,
@ -5325,7 +5445,7 @@ the pchannel count will more evenly distributed if the weight is greater, 0.4 by
p.WALBalancerPolicyVChannelFairVChannelWeight = ParamItem{
Key: "streaming.walBalancer.balancePolicy.vchannelFair.vchannelWeight",
Version: "2.6.0",
Doc: `The weight of vchannel count in vchannelFair balance policy,
Doc: `The weight of vchannel count in vchannelFair balance policy,
the vchannel count will more evenly distributed if the weight is greater, 0.3 by default`,
DefaultValue: "0.3",
Export: true,
@ -5335,7 +5455,7 @@ the vchannel count will more evenly distributed if the weight is greater, 0.3 by
p.WALBalancerPolicyVChannelFairAntiAffinityWeight = ParamItem{
Key: "streaming.walBalancer.balancePolicy.vchannelFair.antiAffinityWeight",
Version: "2.6.0",
Doc: `The weight of affinity in vchannelFair balance policy,
Doc: `The weight of affinity in vchannelFair balance policy,
the fewer VChannels belonging to the same Collection between two PChannels, the higher the affinity,
the vchannel of one collection will more evenly distributed if the weight is greater, 0.01 by default`,
DefaultValue: "0.01",
@ -5357,7 +5477,7 @@ the balance result will be ignored, the lower tolerance, the sensitive rebalance
Key: "streaming.walBalancer.balancePolicy.vchannelFair.rebalanceMaxStep",
Version: "2.6.0",
Doc: `Indicates how many pchannels will be considered as a batch for rebalancing,
the larger step, more aggressive and accurate rebalance,
the larger step, more aggressive and accurate rebalance,
it also determine the depth of depth first search method that is used to find the best balance result, 3 by default`,
DefaultValue: "3",
Export: true,

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