feat: add cachinglayer to sealed segment (#41436)

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

---------

Signed-off-by: Buqian Zheng <zhengbuqian@gmail.com>
This commit is contained in:
Buqian Zheng 2025-04-28 10:52:40 +08:00 committed by GitHub
parent 640f526301
commit 3de904c7ea
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
134 changed files with 6262 additions and 4439 deletions

4
.gitignore vendored
View File

@ -9,7 +9,9 @@
**/cmake-build-release/*
**/cmake_build_release/*
**/cmake_build/*
**/CmakeFiles/*
**/CMakeFiles/*
**/.cmake/*
CMakeCache.txt
.cache
coverage_report/

View File

@ -98,7 +98,7 @@ minio:
# minio.address and minio.port together generate the valid access to MinIO or S3 service.
# MinIO preferentially acquires the valid IP address from the environment variable MINIO_ADDRESS when Milvus is started.
# Default value applies when MinIO or S3 is running on the same network with Milvus.
address: localhost
address: localhost:9000
port: 9000 # Port of MinIO or S3 service.
# Access key ID that MinIO or S3 issues to user for authorized access.
# Environment variable: MINIO_ACCESS_KEY_ID or minio.accessKeyID
@ -447,6 +447,10 @@ queryNode:
memExpansionRate: 1.15 # extra memory needed by building interim index
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.
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

View File

@ -8,7 +8,6 @@ SegmentSealed has an extra interface rather than SegmentInterface:
2. `LoadFieldData(loadFieldDataInfo)`: load column data, could be either scalar column or vector column
1. Note: indexes and vector data for the same column may coexist. Indexes are prioritized in the search
3. `DropIndex(fieldId)`: drop and release an existing index of a specified field
4. `DropFieldData(fieldId)`: drop and release existing data for a specified field
Search is executable as long as all the columns involved in the search are loaded.

View File

@ -71,7 +71,7 @@ class MilvusConan(ConanFile):
"aws-sdk-cpp:config": True,
"aws-sdk-cpp:text-to-speech": False,
"aws-sdk-cpp:transfer": False,
"gtest:build_gmock": False,
"gtest:build_gmock": True,
"boost:without_locale": False,
"boost:without_test": True,
"glog:with_gflags": True,

View File

@ -49,6 +49,7 @@ add_subdirectory( clustering )
add_subdirectory( exec )
add_subdirectory( bitset )
add_subdirectory( futures )
add_subdirectory( cachinglayer )
milvus_add_pkg_config("milvus_core")
@ -67,6 +68,7 @@ add_library(milvus_core SHARED
$<TARGET_OBJECTS:milvus_exec>
$<TARGET_OBJECTS:milvus_bitset>
$<TARGET_OBJECTS:milvus_futures>
$<TARGET_OBJECTS:milvus_cachinglayer>
)
set(LINK_TARGETS

View File

@ -0,0 +1,13 @@
# Copyright (C) 2019-2025 Zilliz. All rights reserved.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software distributed under the License
# is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
# or implied. See the License for the specific language governing permissions and limitations under the License
add_source_at_current_directory_recursively()
add_library(milvus_cachinglayer OBJECT ${SOURCE_FILES})

View File

@ -0,0 +1,372 @@
// Copyright (C) 2019-2025 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#pragma once
#include <any>
#include <chrono>
#include <cstddef>
#include <exception>
#include <memory>
#include <type_traits>
#include <vector>
#include <folly/futures/Future.h>
#include <folly/futures/SharedPromise.h>
#include <folly/Synchronized.h>
#include "cachinglayer/lrucache/DList.h"
#include "cachinglayer/lrucache/ListNode.h"
#include "cachinglayer/Translator.h"
#include "cachinglayer/Utils.h"
#include "log/Log.h"
#include "monitor/prometheus_client.h"
namespace milvus::cachinglayer {
template <typename CellT>
class CellAccessor;
// - The action of pinning cells is not started until the returned SemiFuture is scheduled on an executor.
// - Once the future is scheduled, CacheSlot must live until the future is ready.
// - The returned CellAccessor stores a shared_ptr of CacheSlot, thus will keep CacheSlot alive.
template <typename CellT>
class CacheSlot final : public std::enable_shared_from_this<CacheSlot<CellT>> {
public:
// TODO(tiered storage 1): the CellT should return its actual usage, once loaded. And we use this to report metrics.
static_assert(
std::is_same_v<size_t, decltype(std::declval<CellT>().CellByteSize())>,
"CellT must have a CellByteSize() method that returns a size_t "
"representing the memory consumption of the cell");
CacheSlot(std::unique_ptr<Translator<CellT>> translator,
internal::DList* dlist)
: translator_(std::move(translator)),
cells_(translator_->num_cells()),
dlist_(dlist) {
for (cid_t i = 0; i < translator_->num_cells(); ++i) {
new (&cells_[i])
CacheCell(this, i, translator_->estimated_byte_size_of_cell(i));
}
internal::cache_slot_count(translator_->meta()->storage_type)
.Increment();
internal::cache_cell_count(translator_->meta()->storage_type)
.Increment(translator_->num_cells());
}
CacheSlot(const CacheSlot&) = delete;
CacheSlot&
operator=(const CacheSlot&) = delete;
CacheSlot(CacheSlot&&) = delete;
CacheSlot&
operator=(CacheSlot&&) = delete;
folly::SemiFuture<std::shared_ptr<CellAccessor<CellT>>>
PinCells(std::vector<uid_t> uids) {
return folly::makeSemiFuture().deferValue([this,
uids = std::vector<uid_t>(
uids)](auto&&) {
auto count = uids.size();
std::unordered_set<cid_t> involved_cids;
involved_cids.reserve(count);
for (size_t i = 0; i < count; ++i) {
auto uid = uids[i];
auto cid = translator_->cell_id_of(uid);
if (cid >= cells_.size()) {
return folly::makeSemiFuture<
std::shared_ptr<CellAccessor<CellT>>>(
folly::make_exception_wrapper<std::invalid_argument>(
fmt::format(
"CacheSlot {}: translator returned cell_id {} "
"for uid {} which is out of range",
translator_->key(),
cid,
uid)));
}
involved_cids.insert(cid);
}
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));
});
});
});
}
size_t
num_cells() const {
return translator_->num_cells();
}
ResourceUsage
size_of_cell(cid_t cid) const {
return translator_->estimated_byte_size_of_cell(cid);
}
Meta*
meta() {
return translator_->meta();
}
~CacheSlot() {
internal::cache_slot_count(translator_->meta()->storage_type)
.Decrement();
internal::cache_cell_count(translator_->meta()->storage_type)
.Decrement(translator_->num_cells());
}
private:
friend class CellAccessor<CellT>;
cid_t
cell_id_of(uid_t uid) const {
return translator_->cell_id_of(uid);
}
folly::SemiFuture<folly::Unit>
RunLoad(std::unordered_set<cid_t>&& cids) {
return folly::makeSemiFuture().deferValue(
[this,
cids = std::move(cids)](auto&&) -> folly::SemiFuture<folly::Unit> {
try {
auto start = std::chrono::high_resolution_clock::now();
std::vector<cid_t> cids_vec(cids.begin(), cids.end());
auto results = translator_->get_cells(cids_vec);
auto latency =
std::chrono::duration_cast<std::chrono::microseconds>(
std::chrono::high_resolution_clock::now() - start);
for (auto& result : results) {
cells_[result.first].set_cell(
std::move(result.second),
cids.count(result.first) > 0);
internal::cache_load_latency(
translator_->meta()->storage_type)
.Observe(latency.count());
}
internal::cache_cell_loaded_count(
translator_->meta()->storage_type)
.Increment(results.size());
internal::cache_load_count_success(
translator_->meta()->storage_type)
.Increment(results.size());
} catch (...) {
auto exception = std::current_exception();
auto ew = folly::exception_wrapper(exception);
internal::cache_load_count_fail(
translator_->meta()->storage_type)
.Increment(cids.size());
for (auto cid : cids) {
cells_[cid].set_error(ew);
}
return folly::makeSemiFuture<folly::Unit>(ew);
}
return folly::Unit();
});
}
struct CacheCell : internal::ListNode {
public:
CacheCell() = default;
CacheCell(CacheSlot<CellT>* slot, cid_t cid, ResourceUsage size)
: internal::ListNode(slot->dlist_, size), slot_(slot), cid_(cid) {
}
~CacheCell() {
if (state_ == State::LOADING) {
LOG_ERROR("CacheSlot Cell {} destroyed while loading", key());
}
}
CellT*
cell() {
return cell_.get();
}
// Be careful that even though only a single thread can request loading a cell,
// it is still possible that multiple threads call set_cell() concurrently.
// For example, 2 RunLoad() calls tries to download cell 4 and 6, and both decided
// to also download cell 5, if they finished at the same time, they will call set_cell()
// of cell 5 concurrently.
void
set_cell(std::unique_ptr<CellT> cell, bool requesting_thread) {
mark_loaded(
[this, cell = std::move(cell)]() mutable {
cell_ = std::move(cell);
life_start_ = std::chrono::steady_clock::now();
milvus::monitor::internal_cache_used_bytes_memory.Increment(
size_.memory_bytes);
milvus::monitor::internal_cache_used_bytes_disk.Increment(
size_.file_bytes);
},
requesting_thread);
}
void
set_error(folly::exception_wrapper error) {
internal::ListNode::set_error(std::move(error));
}
protected:
void
unload() override {
if (cell_) {
internal::cache_cell_loaded_count(
slot_->translator_->meta()->storage_type)
.Decrement();
auto life_time = std::chrono::steady_clock::now() - life_start_;
auto seconds =
std::chrono::duration_cast<std::chrono::seconds>(life_time)
.count();
internal::cache_item_lifetime_seconds(
slot_->translator_->meta()->storage_type)
.Observe(seconds);
cell_ = nullptr;
milvus::monitor::internal_cache_used_bytes_memory.Decrement(
size_.memory_bytes);
milvus::monitor::internal_cache_used_bytes_disk.Decrement(
size_.file_bytes);
}
}
std::string
key() const override {
return fmt::format("{}:{}", slot_->translator_->key(), cid_);
}
private:
CacheSlot<CellT>* slot_{nullptr};
cid_t cid_{0};
std::unique_ptr<CellT> cell_{nullptr};
std::chrono::steady_clock::time_point life_start_{};
};
const std::unique_ptr<Translator<CellT>> translator_;
// Each CacheCell's cid_t is its index in vector
// Once initialized, cells_ should never be resized.
std::vector<CacheCell> cells_;
internal::DList* dlist_;
};
// - A thin wrapper for accessing cells in a CacheSlot.
// - When this class is created, the cells are loaded and pinned.
// - Accessing cells through this class does not incur any lock overhead.
// - Accessing cells that are not pinned by this CellAccessor is undefined behavior.
template <typename CellT>
class CellAccessor {
public:
CellAccessor(std::shared_ptr<CacheSlot<CellT>> slot,
std::vector<internal::ListNode::NodePin> pins)
: slot_(std::move(slot)), pins_(std::move(pins)) {
}
CellT*
get_cell_of(uid_t uid) {
auto cid = slot_->cell_id_of(uid);
return slot_->cells_[cid].cell();
}
CellT*
get_ith_cell(cid_t cid) {
return slot_->cells_[cid].cell();
}
private:
// pins must be destroyed before slot_ is destroyed, thus
// pins_ should be a member after slot_.
std::shared_ptr<CacheSlot<CellT>> slot_;
std::vector<internal::ListNode::NodePin> pins_;
};
// TODO(tiered storage 4): this class is a temp solution. Later we should modify all usage of this class
// to use folly::SemiFuture instead: all data access should happen within deferValue().
// Current impl requires the T type to be movable/copyable.
template <typename T>
class PinWrapper {
public:
PinWrapper() = default;
PinWrapper(std::any raii, T&& content)
: raii_(std::move(raii)), content_(std::move(content)) {
}
PinWrapper(std::any raii, const T& content)
: raii_(std::move(raii)), content_(content) {
}
// For those that does not need a pin. eg: growing segment, views that actually copies the data, etc.
PinWrapper(T&& content) : raii_(nullptr), content_(std::move(content)) {
}
PinWrapper(const T& content) : raii_(nullptr), content_(content) {
}
PinWrapper(PinWrapper&& other) noexcept
: raii_(std::move(other.raii_)), content_(std::move(other.content_)) {
}
PinWrapper(const PinWrapper& other)
: raii_(other.raii_), content_(other.content_) {
}
PinWrapper&
operator=(PinWrapper&& other) noexcept {
if (this != &other) {
raii_ = std::move(other.raii_);
content_ = std::move(other.content_);
}
return *this;
}
PinWrapper&
operator=(const PinWrapper& other) {
if (this != &other) {
raii_ = other.raii_;
content_ = other.content_;
}
return *this;
}
T&
get() {
return content_;
}
template <typename T2, typename Fn>
PinWrapper<T2>
transform(Fn&& transformer) && {
T2 transformed = transformer(std::move(content_));
return PinWrapper<T2>(std::move(raii_), std::move(transformed));
}
private:
// CellAccessor is templated on CellT, we don't want to enforce that in this class.
std::any raii_{nullptr};
T content_;
};
} // namespace milvus::cachinglayer

View File

@ -0,0 +1,62 @@
// Copyright (C) 2019-2025 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#include "cachinglayer/Manager.h"
#include <memory>
#include "cachinglayer/Utils.h"
#include "log/Log.h"
namespace milvus::cachinglayer {
Manager&
Manager::GetInstance() {
static Manager instance;
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);
}
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;
}
size_t
Manager::memory_overhead() const {
// TODO(tiered storage 2): calculate memory overhead
return 0;
}
} // namespace milvus::cachinglayer

View File

@ -0,0 +1,66 @@
// Copyright (C) 2019-2025 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#pragma once
#include <memory>
#include "cachinglayer/CacheSlot.h"
#include "cachinglayer/Translator.h"
#include "cachinglayer/lrucache/DList.h"
namespace milvus::cachinglayer {
class Manager {
public:
static Manager&
GetInstance();
// This function is not thread safe, must be called before any CacheSlot is created.
// TODO(tiered storage 4): support dynamic update.
static bool
ConfigureTieredStorage(bool enabled_globally,
int64_t memory_limit_bytes,
int64_t disk_limit_bytes);
Manager(const Manager&) = delete;
Manager&
operator=(const Manager&) = delete;
Manager(Manager&&) = delete;
Manager&
operator=(Manager&&) = delete;
~Manager() = default;
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());
}
// memory overhead for managing all cache slots/cells/translators/policies.
size_t
memory_overhead() const;
private:
friend void
ConfigureTieredStorage(bool enabled_globally,
int64_t memory_limit_bytes,
int64_t disk_limit_bytes);
Manager() = default; // Private constructor
std::unique_ptr<internal::DList> dlist_{nullptr};
bool enable_global_tiered_storage_{false};
}; // class Manager
} // namespace milvus::cachinglayer

View File

@ -0,0 +1,60 @@
// Copyright (C) 2019-2025 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#pragma once
#include <memory>
#include <vector>
#include <utility>
#include "cachinglayer/Utils.h"
namespace milvus::cachinglayer {
struct Meta {
StorageType storage_type;
Meta(StorageType storage_type) : storage_type(storage_type) {
}
};
template <typename CellT>
class Translator {
public:
using value_type = CellT;
virtual size_t
num_cells() const = 0;
virtual cid_t
cell_id_of(uid_t uid) const = 0;
// For resource reservation when a cell is about to be loaded.
// If a cell is about to be pinned and loaded, and there are not enough resource for it, EvictionManager
// will try to evict some other cells to make space. Thus this estimation should generally be greater
// than or equal to the actual size. If the estimation is smaller than the actual size, with insufficient
// resource reserved, the load may fail.
virtual ResourceUsage
estimated_byte_size_of_cell(cid_t cid) const = 0;
// must be unique to identify a CacheSlot.
virtual const std::string&
key() const = 0;
virtual Meta*
meta() = 0;
// Translator may choose to fetch more than requested cells.
// TODO(tiered storage 2): This has a problem: when loading, the resource manager will only reserve the size of the
// requested cells, How can translator be sure the extra cells can fit? Currently if bonus cells are returned,
// used memory in cache may exceed the limit. Maybe try to reserve memory for bonus cells, and drop cell if failed.
virtual std::vector<std::pair<cid_t, std::unique_ptr<CellT>>>
get_cells(const std::vector<cid_t>& cids) = 0;
virtual ~Translator() = default;
};
} // namespace milvus::cachinglayer

View File

@ -0,0 +1,300 @@
// Copyright (C) 2019-2025 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#pragma once
#include <atomic>
#include <cstdint>
#include "common/EasyAssert.h"
#include "folly/executors/InlineExecutor.h"
#include <folly/futures/Future.h>
#include <prometheus/counter.h>
#include <prometheus/gauge.h>
#include <prometheus/histogram.h>
#include "monitor/prometheus_client.h"
namespace milvus::cachinglayer {
using uid_t = int64_t;
using cid_t = int64_t;
enum class StorageType {
MEMORY,
DISK,
MIXED,
};
// TODO(tiered storage 4): this is a temporary function to get the result of a future
// by running it on the inline executor. We don't need this once we are fully async.
template <typename T>
T
SemiInlineGet(folly::SemiFuture<T>&& future) {
return std::move(future).via(&folly::InlineExecutor::instance()).get();
}
struct ResourceUsage {
int64_t memory_bytes{0};
int64_t file_bytes{0};
ResourceUsage() noexcept : memory_bytes(0), file_bytes(0) {
}
ResourceUsage(int64_t mem, int64_t file) noexcept
: memory_bytes(mem), file_bytes(file) {
}
ResourceUsage
operator+(const ResourceUsage& rhs) const {
return ResourceUsage(memory_bytes + rhs.memory_bytes,
file_bytes + rhs.file_bytes);
}
void
operator+=(const ResourceUsage& rhs) {
memory_bytes += rhs.memory_bytes;
file_bytes += rhs.file_bytes;
}
ResourceUsage
operator-(const ResourceUsage& rhs) const {
return ResourceUsage(memory_bytes - rhs.memory_bytes,
file_bytes - rhs.file_bytes);
}
void
operator-=(const ResourceUsage& rhs) {
memory_bytes -= rhs.memory_bytes;
file_bytes -= rhs.file_bytes;
}
bool
operator==(const ResourceUsage& rhs) const {
return memory_bytes == rhs.memory_bytes && file_bytes == rhs.file_bytes;
}
bool
operator!=(const ResourceUsage& rhs) const {
return !(*this == rhs);
}
bool
GEZero() const {
return memory_bytes >= 0 && file_bytes >= 0;
}
bool
CanHold(const ResourceUsage& rhs) const {
return memory_bytes >= rhs.memory_bytes && file_bytes >= rhs.file_bytes;
}
StorageType
storage_type() const {
if (memory_bytes > 0 && file_bytes > 0) {
return StorageType::MIXED;
}
return memory_bytes > 0 ? StorageType::MEMORY : StorageType::DISK;
}
std::string
ToString() const {
return fmt::format("ResourceUsage{memory_bytes={}, file_bytes={}}",
memory_bytes,
file_bytes);
}
};
inline std::ostream&
operator<<(std::ostream& os, const ResourceUsage& usage) {
os << "ResourceUsage{memory_bytes=" << usage.memory_bytes
<< ", file_bytes=" << usage.file_bytes << "}";
return os;
}
inline void
operator+=(std::atomic<ResourceUsage>& atomic_lhs, const ResourceUsage& rhs) {
ResourceUsage current = atomic_lhs.load();
ResourceUsage new_value;
do {
new_value = current;
new_value += rhs;
} while (!atomic_lhs.compare_exchange_weak(current, new_value));
}
inline void
operator-=(std::atomic<ResourceUsage>& atomic_lhs, const ResourceUsage& rhs) {
ResourceUsage current = atomic_lhs.load();
ResourceUsage new_value;
do {
new_value = current;
new_value -= rhs;
} while (!atomic_lhs.compare_exchange_weak(current, new_value));
}
namespace internal {
inline prometheus::Gauge&
cache_slot_count(StorageType storage_type) {
switch (storage_type) {
case StorageType::MEMORY:
return monitor::internal_cache_slot_count_memory;
case StorageType::DISK:
return monitor::internal_cache_slot_count_disk;
case StorageType::MIXED:
return monitor::internal_cache_slot_count_mixed;
default:
PanicInfo(ErrorCode::UnexpectedError, "Unknown StorageType");
}
}
inline prometheus::Gauge&
cache_cell_count(StorageType storage_type) {
switch (storage_type) {
case StorageType::MEMORY:
return monitor::internal_cache_cell_count_memory;
case StorageType::DISK:
return monitor::internal_cache_cell_count_disk;
case StorageType::MIXED:
return monitor::internal_cache_cell_count_mixed;
default:
PanicInfo(ErrorCode::UnexpectedError, "Unknown StorageType");
}
}
inline prometheus::Gauge&
cache_cell_loaded_count(StorageType storage_type) {
switch (storage_type) {
case StorageType::MEMORY:
return monitor::internal_cache_cell_loaded_count_memory;
case StorageType::DISK:
return monitor::internal_cache_cell_loaded_count_disk;
case StorageType::MIXED:
return monitor::internal_cache_cell_loaded_count_mixed;
default:
PanicInfo(ErrorCode::UnexpectedError, "Unknown StorageType");
}
}
inline prometheus::Histogram&
cache_load_latency(StorageType storage_type) {
switch (storage_type) {
case StorageType::MEMORY:
return monitor::internal_cache_load_latency_memory;
case StorageType::DISK:
return monitor::internal_cache_load_latency_disk;
case StorageType::MIXED:
return monitor::internal_cache_load_latency_mixed;
default:
PanicInfo(ErrorCode::UnexpectedError, "Unknown StorageType");
}
}
inline prometheus::Counter&
cache_op_result_count_hit(StorageType storage_type) {
switch (storage_type) {
case StorageType::MEMORY:
return monitor::internal_cache_op_result_count_hit_memory;
case StorageType::DISK:
return monitor::internal_cache_op_result_count_hit_disk;
case StorageType::MIXED:
return monitor::internal_cache_op_result_count_hit_mixed;
default:
PanicInfo(ErrorCode::UnexpectedError, "Unknown StorageType");
}
}
inline prometheus::Counter&
cache_op_result_count_miss(StorageType storage_type) {
switch (storage_type) {
case StorageType::MEMORY:
return monitor::internal_cache_op_result_count_miss_memory;
case StorageType::DISK:
return monitor::internal_cache_op_result_count_miss_disk;
case StorageType::MIXED:
return monitor::internal_cache_op_result_count_miss_mixed;
default:
PanicInfo(ErrorCode::UnexpectedError, "Unknown StorageType");
}
}
inline prometheus::Counter&
cache_eviction_count(StorageType storage_type) {
switch (storage_type) {
case StorageType::MEMORY:
return monitor::internal_cache_eviction_count_memory;
case StorageType::DISK:
return monitor::internal_cache_eviction_count_disk;
case StorageType::MIXED:
return monitor::internal_cache_eviction_count_mixed;
default:
PanicInfo(ErrorCode::UnexpectedError, "Unknown StorageType");
}
}
inline prometheus::Histogram&
cache_item_lifetime_seconds(StorageType storage_type) {
switch (storage_type) {
case StorageType::MEMORY:
return monitor::internal_cache_item_lifetime_seconds_memory;
case StorageType::DISK:
return monitor::internal_cache_item_lifetime_seconds_disk;
case StorageType::MIXED:
return monitor::internal_cache_item_lifetime_seconds_mixed;
default:
PanicInfo(ErrorCode::UnexpectedError, "Unknown StorageType");
}
}
inline prometheus::Counter&
cache_load_count_success(StorageType storage_type) {
switch (storage_type) {
case StorageType::MEMORY:
return monitor::internal_cache_load_count_success_memory;
case StorageType::DISK:
return monitor::internal_cache_load_count_success_disk;
case StorageType::MIXED:
return monitor::internal_cache_load_count_success_mixed;
default:
PanicInfo(ErrorCode::UnexpectedError, "Unknown StorageType");
}
}
inline prometheus::Counter&
cache_load_count_fail(StorageType storage_type) {
switch (storage_type) {
case StorageType::MEMORY:
return monitor::internal_cache_load_count_fail_memory;
case StorageType::DISK:
return monitor::internal_cache_load_count_fail_disk;
case StorageType::MIXED:
return monitor::internal_cache_load_count_fail_mixed;
default:
PanicInfo(ErrorCode::UnexpectedError, "Unknown StorageType");
}
}
inline prometheus::Gauge&
cache_memory_overhead_bytes(StorageType storage_type) {
switch (storage_type) {
case StorageType::MEMORY:
return monitor::internal_cache_memory_overhead_bytes_memory;
case StorageType::DISK:
return monitor::internal_cache_memory_overhead_bytes_disk;
case StorageType::MIXED:
return monitor::internal_cache_memory_overhead_bytes_mixed;
default:
PanicInfo(ErrorCode::UnexpectedError, "Unknown StorageType");
}
}
} // namespace internal
} // namespace milvus::cachinglayer

View File

@ -0,0 +1,201 @@
// Copyright (C) 2019-2025 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#include "cachinglayer/lrucache/DList.h"
#include <mutex>
#include <vector>
#include <folly/futures/Future.h>
#include <folly/futures/SharedPromise.h>
#include "cachinglayer/Utils.h"
#include "cachinglayer/lrucache/ListNode.h"
#include "monitor/prometheus_client.h"
namespace milvus::cachinglayer::internal {
bool
DList::reserveMemory(const ResourceUsage& size) {
std::unique_lock<std::mutex> list_lock(list_mtx_);
auto used = used_memory_.load();
if (max_memory_.CanHold(used + size)) {
used_memory_ += size;
return true;
}
if (tryEvict(used + size - max_memory_)) {
used_memory_ += size;
return true;
}
return false;
}
bool
DList::tryEvict(const ResourceUsage& expected_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.
std::vector<std::unique_lock<std::shared_mutex>> item_locks;
ResourceUsage size_to_evict;
auto would_help = [&](const ResourceUsage& size) -> bool {
auto need_memory =
size_to_evict.memory_bytes < expected_eviction.memory_bytes;
auto need_disk =
size_to_evict.file_bytes < expected_eviction.file_bytes;
return (need_memory && size.memory_bytes > 0) ||
(need_disk && size.file_bytes > 0);
};
for (auto it = tail_; it != nullptr; it = it->next_) {
if (!would_help(it->size())) {
continue;
}
// use try_to_lock to avoid dead lock by failing immediately if the ListNode lock is already held.
auto& lock = item_locks.emplace_back(it->mtx_, std::try_to_lock);
// if lock failed, it means this ListNode will be used again, so we don't evict it anymore.
if (lock.owns_lock() && it->pin_count_ == 0) {
to_evict.push_back(it);
size_to_evict += it->size();
if (size_to_evict.CanHold(expected_eviction)) {
break;
}
} else {
// 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();
}
}
if (!size_to_evict.CanHold(expected_eviction)) {
return false;
}
for (auto* list_node : to_evict) {
auto size = list_node->size();
internal::cache_eviction_count(size.storage_type()).Increment();
popItem(list_node);
list_node->clear_data();
used_memory_ -= size;
}
switch (size_to_evict.storage_type()) {
case StorageType::MEMORY:
milvus::monitor::internal_cache_evicted_bytes_memory.Increment(
size_to_evict.memory_bytes);
break;
case StorageType::DISK:
milvus::monitor::internal_cache_evicted_bytes_disk.Increment(
size_to_evict.file_bytes);
break;
case StorageType::MIXED:
milvus::monitor::internal_cache_evicted_bytes_memory.Increment(
size_to_evict.memory_bytes);
milvus::monitor::internal_cache_evicted_bytes_disk.Increment(
size_to_evict.file_bytes);
break;
default:
PanicInfo(ErrorCode::UnexpectedError, "Unknown StorageType");
}
return true;
}
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");
}
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)) {
return false;
}
}
max_memory_ = new_limit;
milvus::monitor::internal_cache_capacity_bytes_memory.Set(
max_memory_.memory_bytes);
milvus::monitor::internal_cache_capacity_bytes_disk.Set(
max_memory_.file_bytes);
return true;
}
void
DList::releaseMemory(const ResourceUsage& size) {
// safe to substract on atomic without lock
used_memory_ -= size;
}
void
DList::touchItem(ListNode* list_node, std::optional<ResourceUsage> size) {
std::lock_guard<std::mutex> list_lock(list_mtx_);
popItem(list_node);
pushHead(list_node);
if (size.has_value()) {
used_memory_ += size.value();
}
}
void
DList::removeItem(ListNode* list_node, ResourceUsage size) {
std::lock_guard<std::mutex> list_lock(list_mtx_);
if (popItem(list_node)) {
used_memory_ -= size;
}
}
void
DList::pushHead(ListNode* list_node) {
if (head_ == nullptr) {
head_ = list_node;
tail_ = list_node;
} else {
list_node->prev_ = head_;
head_->next_ = list_node;
head_ = list_node;
}
}
bool
DList::popItem(ListNode* list_node) {
if (list_node->prev_ == nullptr && list_node->next_ == nullptr &&
list_node != head_) {
// list_node is not in the list
return false;
}
if (head_ == tail_) {
head_ = tail_ = nullptr;
list_node->prev_ = list_node->next_ = nullptr;
} else if (head_ == list_node) {
head_ = list_node->prev_;
head_->next_ = nullptr;
list_node->prev_ = nullptr;
} else if (tail_ == list_node) {
tail_ = list_node->next_;
tail_->prev_ = nullptr;
list_node->next_ = nullptr;
} else {
list_node->prev_->next_ = list_node->next_;
list_node->next_->prev_ = list_node->prev_;
list_node->prev_ = list_node->next_ = nullptr;
}
return true;
}
bool
DList::IsEmpty() const {
std::lock_guard<std::mutex> list_lock(list_mtx_);
return head_ == nullptr;
}
} // namespace milvus::cachinglayer::internal

View File

@ -0,0 +1,108 @@
// Copyright (C) 2019-2025 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#pragma once
#include <atomic>
#include <mutex>
#include <folly/futures/Future.h>
#include <folly/futures/SharedPromise.h>
#include "cachinglayer/lrucache/ListNode.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, TouchConfig touch_config)
: max_memory_(max_memory), touch_config_(touch_config) {
}
// If after evicting all unpinned items, the used_memory_ is still larger than new_limit, false will be returned
// and no eviction will be done.
// Will throw if new_limit is negative.
bool
UpdateLimit(const ResourceUsage& new_limit);
// True if no nodes in the list.
bool
IsEmpty() const;
// This method uses a global lock.
bool
reserveMemory(const ResourceUsage& size);
// Used only when load failed. This will only cause used_memory_ to decrease, which will not affect the correctness
// of concurrent reserveMemory() even without lock.
void
releaseMemory(const ResourceUsage& size);
// Caller must guarantee that the current thread holds the lock of list_node->mtx_.
// touchItem is used in 2 places:
// 1. when a loaded cell is pinned/unpinned, we need to touch it to refresh the LRU order.
// we don't update used_memory_ here.
// 2. when a cell is loaded as a bonus, we need to touch it to insert into the LRU and update
// used_memory_ to track the memory usage(usage of such cell is not counted during reservation).
void
touchItem(ListNode* list_node,
std::optional<ResourceUsage> size = std::nullopt);
// Caller must guarantee that the current thread holds the lock of list_node->mtx_.
// Removes the node from the list and updates used_memory_.
void
removeItem(ListNode* list_node, ResourceUsage size);
const TouchConfig&
touch_config() const {
return touch_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.
// Must be called under the lock of list_mtx_.
bool
tryEvict(const ResourceUsage& expected_eviction);
// Must be called under the lock of list_mtx_ and list_node->mtx_.
// ListNode is guaranteed to be not in the list.
void
pushHead(ListNode* list_node);
// Must be called under the lock of list_mtx_ and list_node->mtx_.
// If ListNode is not in the list, this function does nothing.
// Returns true if ListNode is in the list and popped, false otherwise.
bool
popItem(ListNode* list_node);
// head_ is the most recently used item, tail_ is the least recently used item.
// tail_ -> next -> ... -> head_
// tail_ <- prev <- ... <- head_
ListNode* head_ = nullptr;
ListNode* tail_ = nullptr;
// TODO(tiered storage 3): benchmark folly::DistributedMutex for this usecase.
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 max_memory_;
const TouchConfig touch_config_;
};
} // namespace milvus::cachinglayer::internal

View File

@ -0,0 +1,211 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#include "cachinglayer/lrucache/ListNode.h"
#include <atomic>
#include <chrono>
#include <memory>
#include <mutex>
#include <fmt/core.h>
#include <folly/ExceptionWrapper.h>
#include <folly/futures/Future.h>
#include <folly/futures/SharedPromise.h>
#include "cachinglayer/lrucache/DList.h"
#include "cachinglayer/Utils.h"
#include "common/EasyAssert.h"
namespace milvus::cachinglayer::internal {
ListNode::NodePin::NodePin(ListNode* node) : node_(node) {
if (node_) {
node_->pin_count_++;
}
}
ListNode::NodePin::~NodePin() {
if (node_) {
node_->unpin();
}
}
ListNode::NodePin::NodePin(NodePin&& other) : NodePin(nullptr) {
std::swap(node_, other.node_);
}
ListNode::NodePin&
ListNode::NodePin::operator=(NodePin&& other) {
std::swap(node_, other.node_);
return *this;
}
ListNode::ListNode(DList* dlist, ResourceUsage size)
: last_touch_(dlist ? (std::chrono::high_resolution_clock::now() -
2 * dlist->touch_config().refresh_window)
: std::chrono::high_resolution_clock::now()),
dlist_(dlist),
size_(size),
state_(State::NOT_LOADED) {
}
ListNode::~ListNode() {
if (dlist_) {
std::unique_lock<std::shared_mutex> lock(mtx_);
dlist_->removeItem(this, size_);
}
}
ResourceUsage&
ListNode::size() {
return size_;
}
std::pair<bool, folly::SemiFuture<ListNode::NodePin>>
ListNode::pin() {
// must be called with lock acquired, and state must not be NOT_LOADED.
auto read_op = [this]() -> std::pair<bool, folly::SemiFuture<NodePin>> {
AssertInfo(state_ != State::NOT_LOADED,
"Programming error: read_op called on a {} cell",
state_to_string(state_));
if (state_ == State::ERROR) {
return std::make_pair(false,
folly::makeSemiFuture<NodePin>(error_));
}
// pin the cell now so that we can avoid taking the lock again in deferValue.
auto p = NodePin(this);
if (state_ == State::LOADED) {
internal::cache_op_result_count_hit(size_.storage_type())
.Increment();
return std::make_pair(false, std::move(p));
}
internal::cache_op_result_count_miss(size_.storage_type()).Increment();
return std::make_pair(false,
load_promise_->getSemiFuture().deferValue(
[this, p = std::move(p)](auto&&) mutable {
return std::move(p);
}));
};
{
std::shared_lock<std::shared_mutex> lock(mtx_);
if (state_ != State::NOT_LOADED) {
return read_op();
}
}
std::unique_lock<std::shared_mutex> lock(mtx_);
if (state_ != State::NOT_LOADED) {
return read_op();
}
// need to load.
internal::cache_op_result_count_miss(size_.storage_type()).Increment();
load_promise_ = std::make_unique<folly::SharedPromise<folly::Unit>>();
state_ = State::LOADING;
if (dlist_ && !dlist_->reserveMemory(size())) {
// if another thread sees LOADING status, the memory reservation has succeeded.
state_ = State::ERROR;
error_ = folly::make_exception_wrapper<std::runtime_error>(fmt::format(
"Failed to load {} due to insufficient resource", key()));
load_promise_->setException(error_);
load_promise_ = nullptr;
return std::make_pair(false, folly::makeSemiFuture<NodePin>(error_));
}
// pin the cell now so that we can avoid taking the lock again in deferValue.
auto p = NodePin(this);
return std::make_pair(
true,
load_promise_->getSemiFuture().deferValue(
[this, p = std::move(p)](auto&&) mutable { return std::move(p); }));
}
void
ListNode::set_error(folly::exception_wrapper error) {
std::unique_lock<std::shared_mutex> lock(mtx_);
AssertInfo(state_ != State::NOT_LOADED && state_ != State::ERROR,
"Programming error: set_error() called on a {} cell",
state_to_string(state_));
// load failed, release the memory reservation.
if (dlist_) {
dlist_->releaseMemory(size());
}
// may be successfully loaded by another thread as a bonus, they will update used memory.
if (state_ == State::LOADED) {
return;
}
// else: state_ is LOADING
state_ = State::ERROR;
load_promise_->setException(error);
load_promise_ = nullptr;
error_ = std::move(error);
}
std::string
ListNode::state_to_string(State state) {
switch (state) {
case State::NOT_LOADED:
return "NOT_LOADED";
case State::LOADING:
return "LOADING";
case State::LOADED:
return "LOADED";
case State::ERROR:
return "ERROR";
}
throw std::invalid_argument("Invalid state");
}
void
ListNode::unpin() {
std::unique_lock<std::shared_mutex> lock(mtx_);
AssertInfo(
state_ == State::LOADED || state_ == State::ERROR,
"Programming error: unpin() called on a {} cell, current pin_count {}",
state_to_string(state_),
pin_count_.load());
if (state_ == State::ERROR) {
return;
}
if (pin_count_.fetch_sub(1) == 1) {
touch(false);
}
}
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) {
std::optional<ResourceUsage> size = std::nullopt;
if (update_used_memory) {
size = size_;
}
dlist_->touchItem(this, size);
last_touch_ = now;
}
}
void
ListNode::clear_data() {
// if the cell is evicted, loaded, pinned and unpinned within a single refresh window,
// 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;
}
unload();
state_ = State::NOT_LOADED;
}
void
ListNode::unload() {
// Default implementation does nothing
}
} // namespace milvus::cachinglayer::internal

View File

@ -0,0 +1,183 @@
// Copyright (C) 2019-2025 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#pragma once
#include <atomic>
#include <chrono>
#include <memory>
#include <folly/ExceptionWrapper.h>
#include <folly/futures/Future.h>
#include <folly/futures/SharedPromise.h>
#include "cachinglayer/Utils.h"
#include "common/EasyAssert.h"
namespace milvus::cachinglayer::internal {
class DList;
// ListNode is not movable/copyable.
class ListNode {
public:
// RAII class to unpin the node.
class NodePin {
public:
// NodePin is movable but not copyable.
NodePin(NodePin&&);
NodePin&
operator=(NodePin&&);
NodePin(const NodePin&) = delete;
NodePin&
operator=(const NodePin&) = delete;
~NodePin();
private:
NodePin(ListNode* node);
friend class ListNode;
ListNode* node_;
};
ListNode() = default;
ListNode(DList* dlist, ResourceUsage size);
virtual ~ListNode();
// ListNode is not movable/copyable because it contains a shared_mutex.
// ListNode also should not be movable/copyable because that would make
// all NodePin::node_ dangling pointers.
ListNode(const ListNode&) = delete;
ListNode&
operator=(const ListNode&) = delete;
ListNode(ListNode&&) = delete;
ListNode&
operator=(ListNode&&) = delete;
// bool in return value: whether the caller needs to load this cell.
// - If the cell is already loaded, return false and an immediately ready future with a NodePin, the node is pinned
// upon return.
// - If the cell is in error state, return false and an immediately ready future with an exception.
// - If the cell is already being loaded by another thread, return false and a future that will be ready when the
// cell is loaded. The node will not be pinned until the future is ready.
// - Otherwise, the cell is not loaded and not being loaded, return true and a future that will be ready when the
// cell is loaded. The caller needs to load this cell and call mark_loaded() to set the cell as loaded.
// The node will not be pinned until the future is ready.
std::pair<bool, folly::SemiFuture<NodePin>>
pin();
ResourceUsage&
size();
// TODO(tiered storage 1): pin on ERROR should re-trigger loading.
// NOT_LOADED ---> LOADING ---> ERROR
// ^ |
// | v
// |------- LOADED
enum class State { NOT_LOADED, LOADING, LOADED, ERROR };
protected:
// will be called during eviction, implementation should release all resources.
virtual void
unload();
virtual std::string
key() const = 0;
template <typename Fn>
void
mark_loaded(Fn&& cb, bool requesting_thread) {
std::unique_lock<std::shared_mutex> lock(mtx_);
if (requesting_thread) {
// requesting thread will promote NOT_LOADED to LOADING and only requesting
// thread will set state to ERROR, thus it is not possible for the requesting
// thread to see NOT_LOADED or ERROR.
AssertInfo(state_ != State::NOT_LOADED && state_ != State::ERROR,
"Programming error: mark_loaded(requesting_thread=true) "
"called on a {} cell",
state_to_string(state_));
// no need to touch() here: node is pinned thus not eligible for eviction.
// we can delay touch() to when unpin() is called.
if (state_ == State::LOADING) {
cb();
state_ = State::LOADED;
load_promise_->setValue(folly::Unit());
load_promise_ = nullptr;
} else {
// LOADED: cell has been loaded by another thread, do nothing.
return;
}
} else {
// Even though this thread did not request loading this cell, translator still
// decided to download it because the adjacent cells are requested.
if (state_ == State::NOT_LOADED || state_ == State::ERROR) {
state_ = State::LOADED;
cb();
// memory of this cell is not reserved, touch() to track it.
touch(true);
} else if (state_ == State::LOADING) {
// another thread has explicitly requested loading this cell, we did it first
// thus we set up the state first.
state_ = State::LOADED;
load_promise_->setValue(folly::Unit());
load_promise_ = nullptr;
cb();
// the node that marked LOADING has already reserved memory, do not double count.
touch(false);
} else {
// LOADED: cell has been loaded by another thread, do nothing.
return;
}
}
}
void
set_error(folly::exception_wrapper error);
State state_{State::NOT_LOADED};
static std::string
state_to_string(State state);
ResourceUsage size_{};
private:
friend class DList;
friend class NodePin;
friend class MockListNode;
friend class DListTest;
friend class DListTestFriend;
friend class ListNodeTestFriend;
friend class ListNodeTest;
// called by DList during eviction. must be called under the lock of mtx_.
// Made virtual for mock testing.
virtual void
clear_data();
void
unpin();
// must be called under the lock of mtx_.
void
touch(bool update_used_memory = true);
mutable std::shared_mutex mtx_;
std::chrono::high_resolution_clock::time_point last_touch_;
// a nullptr dlist_ means this node is not in any DList, and is not prone to cache management.
DList* dlist_;
ListNode* prev_ = nullptr;
ListNode* next_ = nullptr;
std::atomic<int> pin_count_{0};
std::unique_ptr<folly::SharedPromise<folly::Unit>> load_promise_{nullptr};
folly::exception_wrapper error_;
};
} // namespace milvus::cachinglayer::internal

View File

@ -0,0 +1,39 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#pragma once
#include <memory>
#include "common/Channel.h"
#include "parquet/arrow/reader.h"
namespace milvus {
struct ArrowDataWrapper {
ArrowDataWrapper() = default;
ArrowDataWrapper(std::shared_ptr<arrow::RecordBatchReader> reader,
std::shared_ptr<parquet::arrow::FileReader> arrow_reader,
std::shared_ptr<uint8_t[]> file_data)
: reader(std::move(reader)),
arrow_reader(std::move(arrow_reader)),
file_data(std::move(file_data)) {
}
std::shared_ptr<arrow::RecordBatchReader> reader;
// file reader must outlive the record batch reader
std::shared_ptr<parquet::arrow::FileReader> arrow_reader;
// underlying file data memory, must outlive the arrow reader
std::shared_ptr<uint8_t[]> file_data;
std::vector<std::shared_ptr<arrow::Table>> arrow_tables;
};
using ArrowReaderChannel = Channel<std::shared_ptr<milvus::ArrowDataWrapper>>;
} // namespace milvus

View File

@ -8,13 +8,12 @@
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#pragma once
#include <oneapi/tbb/concurrent_queue.h>
#include <atomic>
#include <exception>
#include <optional>
#include "Exception.h"
namespace milvus {
template <typename T>

View File

@ -11,6 +11,7 @@
#pragma once
#include <sys/types.h>
#include <cstddef>
#include <cstdint>
#include <memory>
@ -54,6 +55,11 @@ class Chunk {
return size_;
}
size_t
CellByteSize() const {
return size_;
}
int64_t
RowNums() const {
return row_nums_;

View File

@ -76,14 +76,14 @@ StringChunkWriter::write(const arrow::ArrayVector& array_vec) {
}
}
std::shared_ptr<Chunk>
std::unique_ptr<Chunk>
StringChunkWriter::finish() {
// write padding, maybe not needed anymore
// FIXME
char padding[MMAP_STRING_PADDING];
target_->write(padding, MMAP_STRING_PADDING);
auto [data, size] = target_->get();
return std::make_shared<StringChunk>(row_nums_, data, size, nullable_);
return std::make_unique<StringChunk>(row_nums_, data, size, nullable_);
}
void
@ -135,13 +135,13 @@ JSONChunkWriter::write(const arrow::ArrayVector& array_vec) {
}
}
std::shared_ptr<Chunk>
std::unique_ptr<Chunk>
JSONChunkWriter::finish() {
char padding[simdjson::SIMDJSON_PADDING];
target_->write(padding, simdjson::SIMDJSON_PADDING);
auto [data, size] = target_->get();
return std::make_shared<JSONChunk>(row_nums_, data, size, nullable_);
return std::make_unique<JSONChunk>(row_nums_, data, size, nullable_);
}
void
@ -219,13 +219,13 @@ ArrayChunkWriter::write(const arrow::ArrayVector& array_vec) {
}
}
std::shared_ptr<Chunk>
std::unique_ptr<Chunk>
ArrayChunkWriter::finish() {
char padding[MMAP_ARRAY_PADDING];
target_->write(padding, MMAP_ARRAY_PADDING);
auto [data, size] = target_->get();
return std::make_shared<ArrayChunk>(
return std::make_unique<ArrayChunk>(
row_nums_, data, size, element_type_, nullable_);
}
@ -283,14 +283,14 @@ SparseFloatVectorChunkWriter::write(const arrow::ArrayVector& array_vec) {
}
}
std::shared_ptr<Chunk>
std::unique_ptr<Chunk>
SparseFloatVectorChunkWriter::finish() {
auto [data, size] = target_->get();
return std::make_shared<SparseFloatVectorChunk>(
return std::make_unique<SparseFloatVectorChunk>(
row_nums_, data, size, nullable_);
}
std::shared_ptr<Chunk>
std::unique_ptr<Chunk>
create_chunk(const FieldMeta& field_meta,
int dim,
const arrow::ArrayVector& array_vec) {
@ -390,7 +390,7 @@ create_chunk(const FieldMeta& field_meta,
return w->finish();
}
std::shared_ptr<Chunk>
std::unique_ptr<Chunk>
create_chunk(const FieldMeta& field_meta,
int dim,
File& file,

View File

@ -35,7 +35,7 @@ class ChunkWriterBase {
virtual void
write(const arrow::ArrayVector& data) = 0;
virtual std::shared_ptr<Chunk>
virtual std::unique_ptr<Chunk>
finish() = 0;
std::pair<char*, size_t>
@ -121,10 +121,10 @@ class ChunkWriter final : public ChunkWriterBase {
}
}
std::shared_ptr<Chunk>
std::unique_ptr<Chunk>
finish() override {
auto [data, size] = target_->get();
return std::make_shared<FixedWidthChunk>(
return std::make_unique<FixedWidthChunk>(
row_nums_, dim_, data, size, sizeof(T), nullable_);
}
@ -182,7 +182,7 @@ class StringChunkWriter : public ChunkWriterBase {
void
write(const arrow::ArrayVector& array_vec) override;
std::shared_ptr<Chunk>
std::unique_ptr<Chunk>
finish() override;
};
@ -193,7 +193,7 @@ class JSONChunkWriter : public ChunkWriterBase {
void
write(const arrow::ArrayVector& array_vec) override;
std::shared_ptr<Chunk>
std::unique_ptr<Chunk>
finish() override;
};
@ -212,7 +212,7 @@ class ArrayChunkWriter : public ChunkWriterBase {
void
write(const arrow::ArrayVector& array_vec) override;
std::shared_ptr<Chunk>
std::unique_ptr<Chunk>
finish() override;
private:
@ -226,16 +226,16 @@ class SparseFloatVectorChunkWriter : public ChunkWriterBase {
void
write(const arrow::ArrayVector& array_vec) override;
std::shared_ptr<Chunk>
std::unique_ptr<Chunk>
finish() override;
};
std::shared_ptr<Chunk>
std::unique_ptr<Chunk>
create_chunk(const FieldMeta& field_meta,
int dim,
const arrow::ArrayVector& array_vec);
std::shared_ptr<Chunk>
std::unique_ptr<Chunk>
create_chunk(const FieldMeta& field_meta,
int dim,
File& file,
@ -245,4 +245,4 @@ create_chunk(const FieldMeta& field_meta,
arrow::ArrayVector
read_single_column_batches(std::shared_ptr<arrow::RecordBatchReader> reader);
} // namespace milvus
} // namespace milvus

View File

@ -24,7 +24,7 @@
#include "common/FieldDataInterface.h"
#include "common/Channel.h"
#include "parquet/arrow/reader.h"
#include "common/ArrowDataWrapper.h"
namespace milvus {
@ -156,25 +156,7 @@ using FieldDataPtr = std::shared_ptr<FieldDataBase>;
using FieldDataChannel = Channel<FieldDataPtr>;
using FieldDataChannelPtr = std::shared_ptr<FieldDataChannel>;
struct ArrowDataWrapper {
ArrowDataWrapper() = default;
ArrowDataWrapper(std::shared_ptr<arrow::RecordBatchReader> reader,
std::shared_ptr<parquet::arrow::FileReader> arrow_reader,
std::shared_ptr<uint8_t[]> file_data)
: reader(std::move(reader)),
arrow_reader(std::move(arrow_reader)),
file_data(std::move(file_data)) {
}
std::shared_ptr<arrow::RecordBatchReader> reader;
// file reader must outlive the record batch reader
std::shared_ptr<parquet::arrow::FileReader> arrow_reader;
// underlying file data memory, must outlive the arrow reader
std::shared_ptr<uint8_t[]> file_data;
std::vector<std::shared_ptr<arrow::Table>> arrow_tables;
};
using ArrowReaderChannel = Channel<std::shared_ptr<milvus::ArrowDataWrapper>>;
FieldDataPtr
InitScalarFieldData(const DataType& type, bool nullable, int64_t cap_rows);
} // namespace milvus
} // namespace milvus

View File

@ -95,4 +95,8 @@ class CollectionIndexMeta {
using IndexMetaPtr = std::shared_ptr<CollectionIndexMeta>;
const static IndexMetaPtr empty_index_meta =
std::make_shared<CollectionIndexMeta>(1024,
std::map<FieldId, FieldIndexMeta>());
} //namespace milvus

View File

@ -21,7 +21,6 @@
#include <vector>
#include "Types.h"
#include "common/CDataType.h"
// NOTE: field_id can be system field
// NOTE: Refer to common/SystemProperty.cpp for details
@ -38,7 +37,6 @@ struct LoadFieldDataInfo {
// Set empty to disable mmap,
// mmap file path will be {mmap_dir_path}/{segment_id}/{field_id}
std::string mmap_dir_path = "";
std::string url;
int64_t storage_version = 0;
};

View File

@ -127,7 +127,7 @@ class Span<T,
private:
const T* data_;
const bool* valid_data_;
const int64_t row_count_;
int64_t row_count_;
};
template <typename VectorType>
@ -168,7 +168,7 @@ class Span<
private:
const embedded_type* data_;
const int64_t row_count_;
const int64_t element_sizeof_;
int64_t row_count_;
int64_t element_sizeof_;
};
} // namespace milvus

View File

@ -1175,4 +1175,4 @@ struct fmt::formatter<milvus::ErrorCode> : fmt::formatter<std::string> {
}
return fmt::formatter<std::string>::format(name, ctx);
}
};
};

View File

@ -720,8 +720,7 @@ PhyBinaryRangeFilterExpr::ExecRangeVisitorImplForJsonForIndex() {
if (!json_pair.second) {
return false;
}
auto json = milvus::Json(json_pair.first.data(),
json_pair.first.size());
auto& json = json_pair.first;
if (lower_inclusive && upper_inclusive) {
if (type == uint8_t(milvus::index::JSONType::STRING) ||
type == uint8_t(milvus::index::JSONType::DOUBLE) ||

View File

@ -295,12 +295,12 @@ class PhyCompareFilterExpr : public Expr {
auto [right_chunk_id, right_chunk_offset] =
get_chunk_id_and_offset(right_field_);
auto left_chunk = segment_chunk_reader_.segment_->chunk_data<T>(
auto pw_left = segment_chunk_reader_.segment_->chunk_data<T>(
left_field_, left_chunk_id);
auto right_chunk =
segment_chunk_reader_.segment_->chunk_data<U>(
right_field_, right_chunk_id);
auto left_chunk = pw_left.get();
auto pw_right = segment_chunk_reader_.segment_->chunk_data<U>(
right_field_, right_chunk_id);
auto right_chunk = pw_right.get();
const T* left_data = left_chunk.data() + left_chunk_offset;
const U* right_data = right_chunk.data() + right_chunk_offset;
func.template operator()<FilterType::random>(
@ -326,10 +326,12 @@ class PhyCompareFilterExpr : public Expr {
}
return processed_size;
} else {
auto left_chunk =
auto pw_left =
segment_chunk_reader_.segment_->chunk_data<T>(left_field_, 0);
auto right_chunk =
auto left_chunk = pw_left.get();
auto pw_right =
segment_chunk_reader_.segment_->chunk_data<U>(right_field_, 0);
auto right_chunk = pw_right.get();
const T* left_data = left_chunk.data();
const U* right_data = right_chunk.data();
func.template operator()<FilterType::random>(
@ -363,10 +365,12 @@ class PhyCompareFilterExpr : public Expr {
const auto active_count = segment_chunk_reader_.active_count_;
for (size_t i = current_chunk_id_; i < num_chunk_; i++) {
auto left_chunk =
auto pw_left =
segment_chunk_reader_.segment_->chunk_data<T>(left_field_, i);
auto right_chunk =
auto left_chunk = pw_left.get();
auto pw_right =
segment_chunk_reader_.segment_->chunk_data<U>(right_field_, i);
auto right_chunk = pw_right.get();
auto data_pos = (i == current_chunk_id_) ? current_chunk_pos_ : 0;
auto size =
(i == (num_chunk_ - 1))
@ -431,10 +435,12 @@ class PhyCompareFilterExpr : public Expr {
// only call this function when left and right are not indexed, so they have the same number of chunks
for (size_t i = left_current_chunk_id_; i < left_num_chunk_; i++) {
auto left_chunk =
auto pw_left =
segment_chunk_reader_.segment_->chunk_data<T>(left_field_, i);
auto right_chunk =
auto left_chunk = pw_left.get();
auto pw_right =
segment_chunk_reader_.segment_->chunk_data<U>(right_field_, i);
auto right_chunk = pw_right.get();
auto data_pos =
(i == left_current_chunk_id_) ? left_current_chunk_pos_ : 0;
auto size = 0;

View File

@ -33,6 +33,7 @@
#include "exec/expression/UnaryExpr.h"
#include "exec/expression/ValueExpr.h"
#include "expr/ITypeExpr.h"
#include "monitor/prometheus_client.h"
#include <memory>

View File

@ -344,8 +344,9 @@ class SegmentExpr : public Expr {
std::min(active_count_ - current_data_chunk_pos_, batch_size_);
auto& skip_index = segment_->GetSkipIndex();
auto views_info = segment_->get_batch_views<T>(
auto pw = segment_->get_batch_views<T>(
field_id_, 0, current_data_chunk_pos_, need_size);
auto views_info = pw.get();
if (!skip_func || !skip_func(skip_index, field_id_, 0)) {
// first is the raw data, second is valid_data
// use valid_data to see if raw data is null
@ -381,8 +382,8 @@ class SegmentExpr : public Expr {
Assert(num_data_chunk_ == 1);
auto& skip_index = segment_->GetSkipIndex();
auto [data_vec, valid_data] =
segment_->get_views_by_offsets<T>(field_id_, 0, *input);
auto pw = segment_->get_views_by_offsets<T>(field_id_, 0, *input);
auto [data_vec, valid_data] = pw.get();
if (!skip_func || !skip_func(skip_index, field_id_, 0)) {
func(data_vec.data(),
valid_data.data(),
@ -504,9 +505,9 @@ class SegmentExpr : public Expr {
int64_t offset = (*input)[i];
auto [chunk_id, chunk_offset] =
segment_->get_chunk_by_offset(field_id_, offset);
auto [data_vec, valid_data] =
segment_->get_views_by_offsets<T>(
field_id_, chunk_id, {int32_t(chunk_offset)});
auto pw = segment_->get_views_by_offsets<T>(
field_id_, chunk_id, {int32_t(chunk_offset)});
auto [data_vec, valid_data] = pw.get();
if (!skip_func ||
!skip_func(skip_index, field_id_, chunk_id)) {
func.template operator()<FilterType::random>(
@ -529,7 +530,8 @@ class SegmentExpr : public Expr {
int64_t offset = (*input)[i];
auto [chunk_id, chunk_offset] =
segment_->get_chunk_by_offset(field_id_, offset);
auto chunk = segment_->chunk_data<T>(field_id_, chunk_id);
auto pw = segment_->chunk_data<T>(field_id_, chunk_id);
auto chunk = pw.get();
const T* data = chunk.data() + chunk_offset;
const bool* valid_data = chunk.valid_data();
if (valid_data != nullptr) {
@ -560,7 +562,8 @@ class SegmentExpr : public Expr {
return ProcessDataByOffsetsForSealedSeg<T>(
func, skip_func, input, res, valid_res, values...);
}
auto chunk = segment_->chunk_data<T>(field_id_, 0);
auto pw = segment_->chunk_data<T>(field_id_, 0);
auto chunk = pw.get();
const T* data = chunk.data();
const bool* valid_data = chunk.valid_data();
if (!skip_func || !skip_func(skip_index, field_id_, 0)) {
@ -582,7 +585,8 @@ class SegmentExpr : public Expr {
int64_t offset = (*input)[i];
auto chunk_id = offset / size_per_chunk_;
auto chunk_offset = offset % size_per_chunk_;
auto chunk = segment_->chunk_data<T>(field_id_, chunk_id);
auto pw = segment_->chunk_data<T>(field_id_, chunk_id);
auto chunk = pw.get();
const T* data = chunk.data() + chunk_offset;
const bool* valid_data = chunk.valid_data();
if (valid_data != nullptr) {
@ -642,7 +646,8 @@ class SegmentExpr : public Expr {
size = std::min(size, batch_size_ - processed_size);
auto& skip_index = segment_->GetSkipIndex();
auto chunk = segment_->chunk_data<T>(field_id_, i);
auto pw = segment_->chunk_data<T>(field_id_, i);
auto chunk = pw.get();
const bool* valid_data = chunk.valid_data();
if (valid_data != nullptr) {
valid_data += data_pos;
@ -683,14 +688,6 @@ class SegmentExpr : public Expr {
ValTypes... values) {
int64_t processed_size = 0;
// if constexpr (std::is_same_v<T, std::string_view> ||
// std::is_same_v<T, Json>) {
// if (segment_->type() == SegmentType::Sealed) {
// return ProcessChunkForSealedSeg<T>(
// func, skip_func, res, values...);
// }
// }
for (size_t i = current_data_chunk_; i < num_data_chunk_; i++) {
auto data_pos =
(i == current_data_chunk_) ? current_data_chunk_pos_ : 0;
@ -710,9 +707,10 @@ class SegmentExpr : public Expr {
if (segment_->type() == SegmentType::Sealed) {
// first is the raw data, second is valid_data
// use valid_data to see if raw data is null
auto [data_vec, valid_data] =
segment_->get_batch_views<T>(
field_id_, i, data_pos, size);
auto pw = segment_->get_batch_views<T>(
field_id_, i, data_pos, size);
auto [data_vec, valid_data] = pw.get();
func(data_vec.data(),
valid_data.data(),
nullptr,
@ -724,7 +722,8 @@ class SegmentExpr : public Expr {
}
}
if (!is_seal) {
auto chunk = segment_->chunk_data<T>(field_id_, i);
auto pw = segment_->chunk_data<T>(field_id_, i);
auto chunk = pw.get();
const T* data = chunk.data() + data_pos;
const bool* valid_data = chunk.valid_data();
if (valid_data != nullptr) {
@ -742,15 +741,16 @@ class SegmentExpr : public Expr {
const bool* valid_data;
if constexpr (std::is_same_v<T, std::string_view> ||
std::is_same_v<T, Json>) {
auto batch_views = segment_->get_batch_views<T>(
auto pw = segment_->get_batch_views<T>(
field_id_, i, data_pos, size);
valid_data = batch_views.second.data();
valid_data = pw.get().second.data();
ApplyValidData(valid_data,
res + processed_size,
valid_res + processed_size,
size);
} else {
auto chunk = segment_->chunk_data<T>(field_id_, i);
auto pw = segment_->chunk_data<T>(field_id_, i);
auto chunk = pw.get();
valid_data = chunk.valid_data();
if (valid_data != nullptr) {
valid_data += data_pos;
@ -994,7 +994,8 @@ class SegmentExpr : public Expr {
return {0, offset};
}
}();
auto chunk = segment_->chunk_data<T>(field_id_, chunk_id);
auto pw = segment_->chunk_data<T>(field_id_, chunk_id);
auto chunk = pw.get();
const bool* valid_data = chunk.valid_data();
if (valid_data != nullptr) {
valid_result[i] = valid_data[chunk_offset];
@ -1037,8 +1038,9 @@ class SegmentExpr : public Expr {
std::is_same_v<T, Json> ||
std::is_same_v<T, ArrayView>) {
if (segment_->type() == SegmentType::Sealed) {
auto [data_vec, valid_data] = segment_->get_batch_views<T>(
auto pw = segment_->get_batch_views<T>(
field_id_, i, data_pos, size);
auto [data_vec, valid_data] = pw.get();
ApplyValidData(valid_data.data(),
valid_result + processed_size,
valid_result + processed_size,
@ -1048,7 +1050,8 @@ class SegmentExpr : public Expr {
}
if (!access_sealed_variable_column) {
auto chunk = segment_->chunk_data<T>(field_id_, i);
auto pw = segment_->chunk_data<T>(field_id_, i);
auto chunk = pw.get();
const bool* valid_data = chunk.valid_data();
if (valid_data == nullptr) {
return valid_result;

View File

@ -401,8 +401,7 @@ PhyJsonContainsFilterExpr::ExecJsonContainsByKeyIndex() {
if (!json_pair.second) {
return false;
}
auto json = milvus::Json(json_pair.first.data(),
json_pair.first.size());
auto& json = json_pair.first;
auto array = json.array_at(offset, size);
if (array.error()) {
@ -586,8 +585,7 @@ PhyJsonContainsFilterExpr::ExecJsonContainsArrayByKeyIndex() {
if (!json_pair.second) {
return false;
}
auto json = milvus::Json(json_pair.first.data(),
json_pair.first.size());
auto& json = json_pair.first;
auto array = json.array_at(offset, size);
if (array.error()) {
return false;
@ -865,8 +863,7 @@ PhyJsonContainsFilterExpr::ExecJsonContainsAllByKeyIndex() {
if (!json_pair.second) {
return false;
}
auto json = milvus::Json(json_pair.first.data(),
json_pair.first.size());
auto& json = json_pair.first;
auto array = json.array_at(offset, size);
if (array.error()) {
return false;
@ -1114,8 +1111,7 @@ PhyJsonContainsFilterExpr::ExecJsonContainsAllWithDiffTypeByKeyIndex() {
if (!json_pair.second) {
return false;
}
auto json = milvus::Json(json_pair.first.data(),
json_pair.first.size());
auto& json = json_pair.first;
std::set<int> tmp_elements_index(elements_index);
auto array = json.array_at(offset, size);
if (array.error()) {
@ -1366,8 +1362,7 @@ PhyJsonContainsFilterExpr::ExecJsonContainsAllArrayByKeyIndex() {
if (!json_pair.second) {
return false;
}
auto json = milvus::Json(json_pair.first.data(),
json_pair.first.size());
auto& json = json_pair.first;
auto array = json.array_at(offset, size);
if (array.error()) {
return false;
@ -1602,8 +1597,7 @@ PhyJsonContainsFilterExpr::ExecJsonContainsWithDiffTypeByKeyIndex() {
if (!json_pair.second) {
return false;
}
auto json = milvus::Json(json_pair.first.data(),
json_pair.first.size());
auto& json = json_pair.first;
auto array = json.array_at(offset, size);
if (array.error()) {
return false;

View File

@ -625,8 +625,7 @@ PhyTermFilterExpr::ExecJsonInVariableByKeyIndex() {
if (!json_pair.second) {
return false;
}
auto json = milvus::Json(json_pair.first.data(),
json_pair.first.size());
auto& json = json_pair.first;
if (type == uint8_t(milvus::index::JSONType::STRING) ||
type == uint8_t(milvus::index::JSONType::DOUBLE) ||
type == uint8_t(milvus::index::JSONType::INT64)) {
@ -880,7 +879,6 @@ VectorPtr
PhyTermFilterExpr::ExecVisitorImplForData(EvalCtx& context) {
auto* input = context.get_offset_input();
const auto& bitmap_input = context.get_bitmap_input();
auto real_batch_size =
has_offset_input_ ? input->size() : GetNextBatchSize();
if (real_batch_size == 0) {

View File

@ -517,91 +517,86 @@ PhyUnaryRangeFilterExpr::ExecArrayEqualForIndex(EvalCtx& context,
}
// cache the result to suit the framework.
auto batch_res =
ProcessIndexChunks<IndexInnerType>([this, &val, reverse](Index* _) {
boost::container::vector<IndexInnerType> elems;
for (auto const& element : val.array()) {
auto e = GetValueFromProto<IndexInnerType>(element);
if (std::find(elems.begin(), elems.end(), e) == elems.end()) {
elems.push_back(e);
}
auto batch_res = ProcessIndexChunks<IndexInnerType>([this, &val, reverse](
Index* _) {
boost::container::vector<IndexInnerType> elems;
for (auto const& element : val.array()) {
auto e = GetValueFromProto<IndexInnerType>(element);
if (std::find(elems.begin(), elems.end(), e) == elems.end()) {
elems.push_back(e);
}
}
// filtering by index, get candidates.
std::function<const milvus::ArrayView*(int64_t)> retrieve;
// filtering by index, get candidates.
std::function<bool(milvus::proto::plan::Array& /*val*/,
int64_t /*offset*/)>
is_same;
// avoid use-after-free
milvus::ArrayView array_view_tmp;
if (segment_->is_chunked()) {
retrieve = [this, &array_view_tmp](
int64_t offset) -> const milvus::ArrayView* {
auto [chunk_idx, chunk_offset] =
segment_->get_chunk_by_offset(field_id_, offset);
const auto& chunk =
segment_->template chunk_view<milvus::ArrayView>(
field_id_, chunk_idx);
array_view_tmp = std::move(chunk.first[chunk_offset]);
return &array_view_tmp;
};
if (segment_->is_chunked()) {
is_same = [this, reverse](milvus::proto::plan::Array& val,
int64_t offset) -> bool {
auto [chunk_idx, chunk_offset] =
segment_->get_chunk_by_offset(field_id_, offset);
auto pw = segment_->template chunk_view<milvus::ArrayView>(
field_id_, chunk_idx);
auto chunk = pw.get();
return chunk.first[chunk_offset].is_same_array(val) ^ reverse;
};
} else {
auto size_per_chunk = segment_->size_per_chunk();
is_same = [this, size_per_chunk, reverse](
milvus::proto::plan::Array& val,
int64_t offset) -> bool {
auto chunk_idx = offset / size_per_chunk;
auto chunk_offset = offset % size_per_chunk;
auto pw = segment_->template chunk_data<milvus::ArrayView>(
field_id_, chunk_idx);
auto chunk = pw.get();
auto array_view = chunk.data() + chunk_offset;
return array_view->is_same_array(val) ^ reverse;
};
}
// collect all candidates.
std::unordered_set<size_t> candidates;
std::unordered_set<size_t> tmp_candidates;
auto first_callback = [&candidates](size_t offset) -> void {
candidates.insert(offset);
};
auto callback = [&candidates, &tmp_candidates](size_t offset) -> void {
if (candidates.find(offset) != candidates.end()) {
tmp_candidates.insert(offset);
}
};
auto execute_sub_batch =
[](Index* index_ptr,
const IndexInnerType& val,
const std::function<void(size_t /* offset */)>& callback) {
index_ptr->InApplyCallback(1, &val, callback);
};
// run in-filter.
for (size_t idx = 0; idx < elems.size(); idx++) {
if (idx == 0) {
ProcessIndexChunksV2<IndexInnerType>(
execute_sub_batch, elems[idx], first_callback);
} else {
auto size_per_chunk = segment_->size_per_chunk();
retrieve = [ size_per_chunk, this ](int64_t offset) -> auto {
auto chunk_idx = offset / size_per_chunk;
auto chunk_offset = offset % size_per_chunk;
const auto& chunk =
segment_->template chunk_data<milvus::ArrayView>(
field_id_, chunk_idx);
return chunk.data() + chunk_offset;
};
ProcessIndexChunksV2<IndexInnerType>(
execute_sub_batch, elems[idx], callback);
candidates = std::move(tmp_candidates);
}
// compare the array via the raw data.
auto filter = [&retrieve, &val, reverse](size_t offset) -> bool {
auto data_ptr = retrieve(offset);
return data_ptr->is_same_array(val) ^ reverse;
};
// collect all candidates.
std::unordered_set<size_t> candidates;
std::unordered_set<size_t> tmp_candidates;
auto first_callback = [&candidates](size_t offset) -> void {
candidates.insert(offset);
};
auto callback = [&candidates,
&tmp_candidates](size_t offset) -> void {
if (candidates.find(offset) != candidates.end()) {
tmp_candidates.insert(offset);
}
};
auto execute_sub_batch =
[](Index* index_ptr,
const IndexInnerType& val,
const std::function<void(size_t /* offset */)>& callback) {
index_ptr->InApplyCallback(1, &val, callback);
};
// run in-filter.
for (size_t idx = 0; idx < elems.size(); idx++) {
if (idx == 0) {
ProcessIndexChunksV2<IndexInnerType>(
execute_sub_batch, elems[idx], first_callback);
} else {
ProcessIndexChunksV2<IndexInnerType>(
execute_sub_batch, elems[idx], callback);
candidates = std::move(tmp_candidates);
}
// the size of candidates is small enough.
if (candidates.size() * 100 < active_count_) {
break;
}
// the size of candidates is small enough.
if (candidates.size() * 100 < active_count_) {
break;
}
TargetBitmap res(active_count_);
// run post-filter. The filter will only be executed once in the framework.
for (const auto& candidate : candidates) {
res[candidate] = filter(candidate);
}
return res;
});
}
TargetBitmap res(active_count_);
// run post-filter. The filter will only be executed once in the framework.
for (const auto& candidate : candidates) {
res[candidate] = is_same(val, candidate);
}
return res;
});
AssertInfo(batch_res->size() == real_batch_size,
"internal error: expr processed rows {} not equal "
"expect batch size {}",
@ -1224,8 +1219,7 @@ PhyUnaryRangeFilterExpr::ExecRangeVisitorImplJsonForIndex() {
if (!json_pair.second) {
return false;
}
auto json = milvus::Json(json_pair.first.data(),
json_pair.first.size());
auto& json = json_pair.first;
switch (op_type) {
case proto::plan::GreaterThan:
if constexpr (std::is_same_v<GetType,

View File

@ -16,6 +16,8 @@
#include "FilterBitsNode.h"
#include "monitor/prometheus_client.h"
namespace milvus {
namespace exec {
PhyFilterBitsNode::PhyFilterBitsNode(

View File

@ -16,6 +16,8 @@
#include "IterativeFilterNode.h"
#include "monitor/prometheus_client.h"
namespace milvus {
namespace exec {
PhyIterativeFilterNode::PhyIterativeFilterNode(

View File

@ -16,7 +16,8 @@
#include "RandomSampleNode.h"
#include <exec/expression/Utils.h>
#include "exec/expression/Utils.h"
#include "monitor/prometheus_client.h"
namespace milvus {
namespace exec {

View File

@ -16,6 +16,8 @@
#include "VectorSearchNode.h"
#include "monitor/prometheus_client.h"
namespace milvus {
namespace exec {

View File

@ -17,6 +17,8 @@
#pragma once
#include <optional>
#include "cachinglayer/CacheSlot.h"
#include "common/QueryInfo.h"
#include "common/Types.h"
#include "knowhere/index/index_node.h"
@ -78,9 +80,10 @@ class SealedDataGetter : public DataGetter<T> {
const FieldId field_id_;
bool from_data_;
mutable std::unordered_map<int64_t, std::vector<std::string_view>>
str_view_map_;
mutable std::unordered_map<int64_t, FixedVector<bool>> valid_map_;
mutable std::unordered_map<
int64_t,
PinWrapper<std::pair<std::vector<std::string_view>, FixedVector<bool>>>>
pw_map_;
// Getting str_view from segment is cpu-costly, this map is to cache this view for performance
public:
SealedDataGetter(const segcore::SegmentSealed& segment, FieldId& field_id)
@ -103,23 +106,22 @@ class SealedDataGetter : public DataGetter<T> {
auto chunk_id = id_offset_pair.first;
auto inner_offset = id_offset_pair.second;
if constexpr (std::is_same_v<T, std::string>) {
if (str_view_map_.find(chunk_id) == str_view_map_.end()) {
auto [str_chunk_view, valid_data] =
segment_.chunk_view<std::string_view>(field_id_,
chunk_id);
valid_map_[chunk_id] = std::move(valid_data);
str_view_map_[chunk_id] = std::move(str_chunk_view);
if (pw_map_.find(chunk_id) == pw_map_.end()) {
// for now, search_group_by does not handle null values
auto pw = segment_.chunk_view<std::string_view>(field_id_,
chunk_id);
pw_map_[chunk_id] = std::move(pw);
}
auto valid_data = valid_map_[chunk_id];
if (!valid_data.empty()) {
if (!valid_map_[chunk_id][inner_offset]) {
return std::nullopt;
}
auto& pw = pw_map_[chunk_id];
auto& [str_chunk_view, valid_data] = pw.get();
if (!valid_data.empty() && !valid_data[inner_offset]) {
return std::nullopt;
}
auto str_val_view = str_view_map_[chunk_id][inner_offset];
std::string_view str_val_view = str_chunk_view[inner_offset];
return std::string(str_val_view.data(), str_val_view.length());
} else {
Span<T> span = segment_.chunk_data<T>(field_id_, chunk_id);
auto pw = segment_.chunk_data<T>(field_id_, chunk_id);
auto& span = pw.get();
if (span.valid_data() && !span.valid_data()[inner_offset]) {
return std::nullopt;
}

View File

@ -24,7 +24,7 @@ getGlobalCPUExecutor() {
static folly::CPUThreadPoolExecutor executor(
thread_num,
folly::CPUThreadPoolExecutor::makeDefaultPriorityQueue(kNumPriority),
std::make_shared<folly::NamedThreadFactory>("MILVUS_FUTURE_CPU_"));
std::make_shared<folly::NamedThreadFactory>("MILVUS_CPU_"));
return &executor;
}

View File

@ -103,19 +103,13 @@ class JsonKeyStatsInvertedIndex : public InvertedIndexTantivy<std::string> {
return bitset;
};
if (is_growing) {
if (shouldTriggerCommit() || is_strong_consistency) {
if (is_data_uncommitted_) {
Commit();
}
Reload();
return processArray();
} else {
return processArray();
if (is_growing && (shouldTriggerCommit() || is_strong_consistency)) {
if (is_data_uncommitted_) {
Commit();
}
} else {
return processArray();
Reload();
}
return processArray();
}
void

View File

@ -10,13 +10,10 @@
// or implied. See the License for the specific language governing permissions and limitations under the License
#pragma once
#include <cstddef>
#include <unordered_map>
#include "common/Types.h"
#include "log/Log.h"
#include "mmap/Column.h"
#include "mmap/ChunkedColumn.h"
namespace milvus {
@ -308,15 +305,15 @@ class SkipIndex {
if (start > num_rows - 1) {
return {std::string(), std::string(), num_rows};
}
std::string_view min_string = var_column.RawAt(start);
std::string_view max_string = var_column.RawAt(start);
std::string min_string = var_column.RawAt(start);
std::string max_string = min_string;
int64_t null_count = start;
for (int64_t i = start; i < num_rows; i++) {
const auto& val = var_column.RawAt(i);
if (!var_column.IsValid(i)) {
null_count++;
continue;
}
const auto& val = var_column.RawAt(i);
if (val < min_string) {
min_string = val;
}
@ -325,7 +322,7 @@ class SkipIndex {
}
}
// The field data may 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};
return {min_string, max_string, null_count};
}
private:

View File

@ -14,8 +14,10 @@
// See the License for the specific language governing permissions and
// limitations under the License.
#pragma once
#include "common/Array.h"
#include "storage/MmapManager.h"
namespace milvus {
/**
* @brief FixedLengthChunk
@ -207,7 +209,7 @@ VariableLengthChunk<Array>::set(
auto mcm = storage::MmapManager::GetInstance().GetMmapChunkManager();
AssertInfo(
begin + length <= size_,
"failed to set a chunk with length: {} from beign {}, map_size={}",
"failed to set a chunk with length: {} from begin {}, map_size={}",
length,
begin,
size_);

View File

@ -14,8 +14,10 @@
// See the License for the specific language governing permissions and
// limitations under the License.
#pragma once
#include "mmap/ChunkData.h"
#include "storage/MmapManager.h"
namespace milvus {
template <typename Type>
class ChunkVectorBase {

View File

@ -21,82 +21,76 @@
#include <cstddef>
#include <cstdint>
#include <cstring>
#include <filesystem>
#include <memory>
#include <queue>
#include <string>
#include <vector>
#include <math.h>
#include "cachinglayer/CacheSlot.h"
#include "cachinglayer/Manager.h"
#include "cachinglayer/Translator.h"
#include "cachinglayer/Utils.h"
#include "common/Array.h"
#include "common/Chunk.h"
#include "common/Common.h"
#include "common/EasyAssert.h"
#include "common/File.h"
#include "common/FieldMeta.h"
#include "common/FieldData.h"
#include "common/Span.h"
#include "fmt/format.h"
#include "log/Log.h"
#include "mmap/Utils.h"
#include "common/FieldData.h"
#include "common/FieldDataInterface.h"
#include "common/Array.h"
#include "knowhere/dataset.h"
#include "monitor/prometheus_client.h"
#include "storage/MmapChunkManager.h"
#include "segcore/storagev1translator/ChunkTranslator.h"
#include "mmap/Column.h"
namespace milvus {
class ChunkedColumnBase : public ColumnBase {
using namespace milvus::cachinglayer;
std::pair<size_t, size_t> inline GetChunkIDByOffset(
int64_t offset, std::vector<int64_t>& num_rows_until_chunk) {
AssertInfo(offset >= 0 && offset < num_rows_until_chunk.back(),
"offset is out of range, offset: {}, num rows: {}",
offset,
num_rows_until_chunk.back());
auto iter = std::lower_bound(
num_rows_until_chunk.begin(), num_rows_until_chunk.end(), offset + 1);
size_t chunk_idx = std::distance(num_rows_until_chunk.begin(), iter) - 1;
size_t offset_in_chunk = offset - num_rows_until_chunk[chunk_idx];
return {chunk_idx, offset_in_chunk};
}
class ChunkedColumnBase {
public:
ChunkedColumnBase() = default;
// memory mode ctor
explicit ChunkedColumnBase(const FieldMeta& field_meta) {
nullable_ = field_meta.is_nullable();
explicit ChunkedColumnBase(std::unique_ptr<Translator<Chunk>> translator,
const FieldMeta& field_meta)
: nullable_(field_meta.is_nullable()),
num_chunks_(translator->num_cells()),
slot_(Manager::GetInstance().CreateCacheSlot(std::move(translator))) {
num_rows_ = GetNumRowsUntilChunk().back();
}
virtual ~ChunkedColumnBase() = default;
void
AppendBatch(const FieldDataPtr data) override {
PanicInfo(ErrorCode::Unsupported, "AppendBatch not supported");
}
const char*
Data(int chunk_id) const override {
return chunks_[chunk_id]->Data();
}
virtual const char*
ValueAt(int64_t offset) const {
auto [chunk_id, offset_in_chunk] = GetChunkIDByOffset(offset);
return chunks_[chunk_id]->ValueAt(offset_in_chunk);
};
// MmappedData() returns the mmaped address
const char*
MmappedData() const override {
AssertInfo(chunks_.size() == 1,
"only support one chunk, but got {} chunk(s)",
chunks_.size());
return chunks_[0]->RawData();
PinWrapper<const char*>
DataOfChunk(int chunk_id) const {
auto ca = SemiInlineGet(slot_->PinCells({chunk_id}));
auto chunk = ca->get_cell_of(chunk_id);
return PinWrapper<const char*>(ca, chunk->Data());
}
bool
IsValid(size_t offset) const {
if (nullable_) {
auto [chunk_id, offset_in_chunk] = GetChunkIDByOffset(offset);
return chunks_[chunk_id]->isValid(offset_in_chunk);
if (!nullable_) {
return true;
}
return true;
auto [chunk_id, offset_in_chunk] = GetChunkIDByOffset(offset);
return IsValid(chunk_id, offset_in_chunk);
}
bool
IsValid(int64_t chunk_id, int64_t offset) const {
if (nullable_) {
return chunks_[chunk_id]->isValid(offset);
auto ca =
SemiInlineGet(slot_->PinCells({static_cast<cid_t>(chunk_id)}));
auto chunk = ca->get_cell_of(chunk_id);
return chunk->isValid(offset);
}
return true;
}
@ -113,65 +107,53 @@ class ChunkedColumnBase : public ColumnBase {
int64_t
num_chunks() const {
return chunks_.size();
}
virtual void
AddChunk(std::shared_ptr<Chunk> chunk) {
num_rows_until_chunk_.push_back(num_rows_);
num_rows_ += chunk->RowNums();
chunks_.push_back(chunk);
return num_chunks_;
}
// This returns only memory byte size.
size_t
DataByteSize() const override {
DataByteSize() const {
auto size = 0;
for (auto& chunk : chunks_) {
size += chunk->Size();
for (auto i = 0; i < num_chunks_; i++) {
size += slot_->size_of_cell(i).memory_bytes;
}
return size;
}
int64_t
chunk_row_nums(int64_t chunk_id) const {
return chunks_[chunk_id]->RowNums();
return GetNumRowsUntilChunk(chunk_id + 1) -
GetNumRowsUntilChunk(chunk_id);
}
virtual SpanBase
Span(int64_t chunk_id) const = 0;
// used for sequential access for search
virtual BufferView
GetBatchBuffer(int64_t chunk_id, int64_t start_offset, int64_t length) {
virtual PinWrapper<SpanBase>
Span(int64_t chunk_id) const {
PanicInfo(ErrorCode::Unsupported,
"GetBatchBuffer only supported for VariableColumn");
"Span only supported for ChunkedColumn");
}
virtual std::string_view
RawAt(const size_t i) const {
PanicInfo(ErrorCode::Unsupported,
"RawAt only supported for VariableColumn");
}
virtual std::pair<std::vector<std::string_view>, FixedVector<bool>>
virtual 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) const {
std::optional<std::pair<int64_t, int64_t>> offset_len =
std::nullopt) const {
PanicInfo(ErrorCode::Unsupported,
"StringViews only supported for VariableColumn");
}
virtual std::pair<std::vector<ArrayView>, FixedVector<bool>>
virtual PinWrapper<std::pair<std::vector<ArrayView>, FixedVector<bool>>>
ArrayViews(int64_t chunk_id,
std::optional<std::pair<int64_t, int64_t>> offset_len) const {
PanicInfo(ErrorCode::Unsupported,
"ArrayViews only supported for ArrayChunkedColumn");
}
virtual std::pair<std::vector<std::string_view>, FixedVector<bool>>
virtual PinWrapper<
std::pair<std::vector<std::string_view>, FixedVector<bool>>>
ViewsByOffsets(int64_t chunk_id,
const FixedVector<int32_t>& offsets) const {
PanicInfo(ErrorCode::Unsupported,
"viewsbyoffsets only supported for VariableColumn");
"ViewsByOffsets only supported for VariableColumn");
}
std::pair<size_t, size_t>
@ -180,268 +162,148 @@ class ChunkedColumnBase : public ColumnBase {
"offset {} is out of range, num_rows: {}",
offset,
num_rows_);
auto iter = std::lower_bound(num_rows_until_chunk_.begin(),
num_rows_until_chunk_.end(),
offset + 1);
size_t chunk_idx =
std::distance(num_rows_until_chunk_.begin(), iter) - 1;
size_t offset_in_chunk = offset - num_rows_until_chunk_[chunk_idx];
return {chunk_idx, offset_in_chunk};
auto num_rows_until_chunk = GetNumRowsUntilChunk();
return ::milvus::GetChunkIDByOffset(offset, num_rows_until_chunk);
}
std::shared_ptr<Chunk>
PinWrapper<Chunk*>
GetChunk(int64_t chunk_id) const {
return chunks_[chunk_id];
auto ca = SemiInlineGet(slot_->PinCells({chunk_id}));
auto chunk = ca->get_cell_of(chunk_id);
return PinWrapper<Chunk*>(ca, chunk);
}
int64_t
GetNumRowsUntilChunk(int64_t chunk_id) const {
return num_rows_until_chunk_[chunk_id];
return GetNumRowsUntilChunk()[chunk_id];
}
const std::vector<int64_t>&
GetNumRowsUntilChunk() const {
return num_rows_until_chunk_;
auto meta = static_cast<milvus::segcore::storagev1translator::CTMeta*>(
slot_->meta());
return meta->num_rows_until_chunk_;
}
protected:
bool nullable_{false};
size_t num_rows_{0};
std::vector<int64_t> num_rows_until_chunk_;
private:
// void
// UpdateMetricWhenMmap(size_t mmaped_size) {
// UpdateMetricWhenMmap(mapping_type_, mmaped_size);
// }
// void
// UpdateMetricWhenMmap(bool is_map_anonymous, size_t mapped_size) {
// if (mapping_type_ == MappingType::MAP_WITH_ANONYMOUS) {
// milvus::monitor::internal_mmap_allocated_space_bytes_anon.Observe(
// mapped_size);
// milvus::monitor::internal_mmap_in_used_space_bytes_anon.Increment(
// mapped_size);
// } else {
// milvus::monitor::internal_mmap_allocated_space_bytes_file.Observe(
// mapped_size);
// milvus::monitor::internal_mmap_in_used_space_bytes_file.Increment(
// mapped_size);
// }
// }
// void
// UpdateMetricWhenMunmap(size_t mapped_size) {
// if (mapping_type_ == MappingType::MAP_WITH_ANONYMOUS) {
// milvus::monitor::internal_mmap_in_used_space_bytes_anon.Decrement(
// mapped_size);
// } else {
// milvus::monitor::internal_mmap_in_used_space_bytes_file.Decrement(
// mapped_size);
// }
// }
private:
storage::MmapChunkManagerPtr mcm_ = nullptr;
protected:
std::vector<std::shared_ptr<Chunk>> chunks_;
size_t num_chunks_{0};
mutable std::shared_ptr<CacheSlot<Chunk>> slot_;
};
class ChunkedColumn : public ChunkedColumnBase {
public:
ChunkedColumn() = default;
// memory mode ctor
explicit ChunkedColumn(const FieldMeta& field_meta)
: ChunkedColumnBase(field_meta) {
explicit ChunkedColumn(std::unique_ptr<Translator<Chunk>> translator,
const FieldMeta& field_meta)
: ChunkedColumnBase(std::move(translator), field_meta) {
}
explicit ChunkedColumn(const FieldMeta& field_meta,
const std::vector<std::shared_ptr<Chunk>>& chunks)
: ChunkedColumnBase(field_meta) {
for (auto& chunk : chunks) {
AddChunk(chunk);
}
// TODO(tiered storage 1): this method should be replaced with a bulk access method.
const char*
ValueAt(int64_t offset) {
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);
}
~ChunkedColumn() override = default;
SpanBase
PinWrapper<SpanBase>
Span(int64_t chunk_id) const override {
return std::static_pointer_cast<FixedWidthChunk>(chunks_[chunk_id])
->Span();
auto ca = SemiInlineGet(slot_->PinCells({chunk_id}));
auto chunk = ca->get_cell_of(chunk_id);
return PinWrapper<SpanBase>(
ca, static_cast<FixedWidthChunk*>(chunk)->Span());
}
};
// when mmap is used, size_, data_ and num_rows_ of ColumnBase are used.
class ChunkedSparseFloatColumn : public ChunkedColumnBase {
public:
// memory mode ctor
explicit ChunkedSparseFloatColumn(const FieldMeta& field_meta)
: ChunkedColumnBase(field_meta) {
}
explicit ChunkedSparseFloatColumn(
const FieldMeta& field_meta,
const std::vector<std::shared_ptr<Chunk>>& chunks)
: ChunkedColumnBase(field_meta) {
for (auto& chunk : chunks) {
AddChunk(chunk);
}
}
~ChunkedSparseFloatColumn() override = default;
void
AddChunk(std::shared_ptr<Chunk> chunk) override {
num_rows_until_chunk_.push_back(num_rows_);
num_rows_ += chunk->RowNums();
chunks_.push_back(chunk);
dim_ = std::max(
dim_,
std::static_pointer_cast<SparseFloatVectorChunk>(chunk)->Dim());
}
SpanBase
Span(int64_t chunk_id) const override {
PanicInfo(ErrorCode::Unsupported,
"Span not supported for sparse float column");
}
int64_t
Dim() const {
return dim_;
}
private:
int64_t dim_ = 0;
};
template <typename T>
class ChunkedVariableColumn : public ChunkedColumnBase {
public:
using ViewType =
std::conditional_t<std::is_same_v<T, std::string>, std::string_view, T>;
static_assert(
std::is_same_v<T, std::string> || std::is_same_v<T, Json>,
"ChunkedVariableColumn only supports std::string or Json types");
// memory mode ctor
explicit ChunkedVariableColumn(const FieldMeta& field_meta)
: ChunkedColumnBase(field_meta) {
}
explicit ChunkedVariableColumn(
const FieldMeta& field_meta,
const std::vector<std::shared_ptr<Chunk>>& chunks)
: ChunkedColumnBase(field_meta) {
for (auto& chunk : chunks) {
AddChunk(chunk);
}
std::unique_ptr<Translator<Chunk>> translator,
const FieldMeta& field_meta)
: ChunkedColumnBase(std::move(translator), field_meta) {
}
~ChunkedVariableColumn() override = default;
SpanBase
Span(int64_t chunk_id) const override {
PanicInfo(ErrorCode::NotImplemented,
"span() interface is not implemented for variable column");
}
std::pair<std::vector<std::string_view>, FixedVector<bool>>
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 {
return std::static_pointer_cast<StringChunk>(chunks_[chunk_id])
->StringViews(offset_len);
auto ca = SemiInlineGet(slot_->PinCells({chunk_id}));
auto chunk = ca->get_cell_of(chunk_id);
return PinWrapper<
std::pair<std::vector<std::string_view>, FixedVector<bool>>>(
ca, static_cast<StringChunk*>(chunk)->StringViews(offset_len));
}
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 {
return std::static_pointer_cast<StringChunk>(chunks_[chunk_id])
->ViewsByOffsets(offsets);
auto ca = SemiInlineGet(slot_->PinCells({chunk_id}));
auto chunk = ca->get_cell_of(chunk_id);
return PinWrapper<
std::pair<std::vector<std::string_view>, FixedVector<bool>>>(
ca, static_cast<StringChunk*>(chunk)->ViewsByOffsets(offsets));
}
BufferView
GetBatchBuffer(int64_t chunk_id,
int64_t start_offset,
int64_t length) override {
BufferView buffer_view;
std::vector<BufferView::Element> elements;
elements.push_back(
{chunks_[chunk_id]->Data(),
std::static_pointer_cast<StringChunk>(chunks_[chunk_id])
->Offsets(),
static_cast<int>(start_offset),
static_cast<int>(start_offset + length)});
buffer_view.data_ = elements;
return buffer_view;
}
ViewType
operator[](const int i) const {
// 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_) {
PanicInfo(ErrorCode::OutOfRange, "index out of range");
}
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);
std::string_view str_view =
std::static_pointer_cast<StringChunk>(chunks_[chunk_id])
->
operator[](offset_in_chunk);
return ViewType(str_view.data(), str_view.size());
}
std::string_view
RawAt(const size_t i) const {
return std::string_view((*this)[i]);
static_cast<StringChunk*>(chunk)->operator[](offset_in_chunk);
return T(str_view.data(), str_view.size());
}
};
class ChunkedArrayColumn : public ChunkedColumnBase {
public:
// memory mode ctor
explicit ChunkedArrayColumn(const FieldMeta& field_meta)
: ChunkedColumnBase(field_meta) {
}
explicit ChunkedArrayColumn(
const FieldMeta& field_meta,
const std::vector<std::shared_ptr<Chunk>>& chunks)
: ChunkedColumnBase(field_meta) {
for (auto& chunk : chunks) {
AddChunk(chunk);
}
}
~ChunkedArrayColumn() override = default;
SpanBase
Span(int64_t chunk_id) const override {
PanicInfo(ErrorCode::NotImplemented,
"span() interface is not implemented for arr chunk column");
}
ArrayView
operator[](const int i) const {
auto [chunk_id, offset_in_chunk] = GetChunkIDByOffset(i);
return std::static_pointer_cast<ArrayChunk>(chunks_[chunk_id])
->View(offset_in_chunk);
explicit ChunkedArrayColumn(std::unique_ptr<Translator<Chunk>> translator,
const FieldMeta& field_meta)
: ChunkedColumnBase(std::move(translator), field_meta) {
}
// TODO(tiered storage 1): this method should be replaced with a bulk access method.
ScalarArray
RawAt(const int i) const {
auto [chunk_id, offset_in_chunk] = GetChunkIDByOffset(i);
return std::static_pointer_cast<ArrayChunk>(chunks_[chunk_id])
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();
}
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 =
std::nullopt) const override {
return std::static_pointer_cast<ArrayChunk>(chunks_[chunk_id])
->Views(offset_len);
auto ca =
SemiInlineGet(slot_->PinCells({static_cast<cid_t>(chunk_id)}));
auto chunk = ca->get_cell_of(chunk_id);
return PinWrapper<std::pair<std::vector<ArrayView>, FixedVector<bool>>>(
ca, static_cast<ArrayChunk*>(chunk)->Views(offset_len));
}
};
} // namespace milvus
} // namespace milvus

View File

@ -1,137 +0,0 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#pragma once
#include <folly/io/IOBuf.h>
#include <sys/mman.h>
#include <algorithm>
#include <cstddef>
#include <cstring>
#include <filesystem>
#include <memory>
#include <queue>
#include <string>
#include <utility>
#include <vector>
#include "common/Array.h"
#include "common/Common.h"
#include "common/EasyAssert.h"
#include "common/File.h"
#include "common/FieldMeta.h"
#include "common/FieldData.h"
#include "common/Span.h"
#include "fmt/format.h"
#include "log/Log.h"
#include "mmap/Utils.h"
#include "common/FieldData.h"
#include "common/FieldDataInterface.h"
#include "common/Array.h"
#include "knowhere/dataset.h"
#include "monitor/prometheus_client.h"
#include "storage/MmapChunkManager.h"
namespace milvus {
constexpr size_t DEFAULT_PK_VRCOL_BLOCK_SIZE = 1;
constexpr size_t DEFAULT_MEM_VRCOL_BLOCK_SIZE = 32;
constexpr size_t DEFAULT_MMAP_VRCOL_BLOCK_SIZE = 256;
/**
* ColumnBase and its subclasses are designed to store and retrieve the raw data
* of a field.
*
* It has 3 types of constructors corresponding to 3 MappingTypes:
*
* 1. MAP_WITH_ANONYMOUS: ColumnBase(size_t reserve_size, const FieldMeta& field_meta)
*
* This is used when we store the entire data in memory. Upon return, a piece
* of unwritten memory is allocated and the caller can fill the memory with data by
* calling AppendBatch/Append.
*
* 2. MAP_WITH_FILE: ColumnBase(const File& file, size_t size, const FieldMeta& field_meta)
*
* This is used when the raw data has already been written into a file, and we
* simply mmap the file to memory and interpret the memory as a column. In this
* mode, since the data is already in the file/mmapped memory, calling AppendBatch
* and Append is not allowed.
*
* 3. MAP_WITH_MANAGER: ColumnBase(size_t reserve,
* const DataType& data_type,
* storage::MmapChunkManagerPtr mcm,
* storage::MmapChunkDescriptorPtr descriptor,
* bool nullable)
*
* This is used when we want to mmap but don't want to download all the data at once.
* Instead, we download the data in chunks, cache and mmap each chunk as a single
* ColumnBase. Upon return, a piece of unwritten mmaped memory is allocated by the chunk
* manager, and the caller should fill the memory with data by calling AppendBatch
* and Append.
*
* - Types with fixed length can use the Column subclass directly.
* - Types with variable lengths:
* - SparseFloatColumn:
* - To store sparse float vectors.
* - All 3 modes are supported.
* - VariableColumn:
* - To store string like types such as VARCHAR and JSON.
* - MAP_WITH_MANAGER is not yet supported(as of 2024.09.11).
* - ArrayColumn:
* - To store ARRAY types.
* - MAP_WITH_MANAGER is not yet supported(as of 2024.09.11).
*
*/
class ColumnBase {
/**
* - data_ points at a piece of memory of size data_cap_size_ + padding_.
* Based on mapping_type_, such memory can be:
* - an anonymous memory region, allocated by mmap(MAP_ANON)
* - a file-backed memory region, mapped by mmap(MAP_FILE)
* - a memory region managed by MmapChunkManager, allocated by
* MmapChunkManager::Allocate()
*
* Memory Layout of `data_`:
*
* |<-- data_cap_size_ -->|<-- padding_ -->|
* |<-- data_size_ -->|<-- free space -->|
*
* AppendBatch/Append should first check if there's enough space for new data.
* If not, call ExpandData() to expand the space.
*
* - only the first data_cap_size_ bytes can be used to store actual data.
* - padding at the end is to ensure when all values are empty, we don't try
* to allocate/mmap 0 bytes memory, which will cause mmap() to fail.
* - data_size_ is the number of bytes currently used to store actual data.
* - num_rows_ is the number of rows currently stored.
* - valid_data_ is a FixedVector<bool> indicating whether each element is
* not null. it is only used when nullable is true.
* - nullable_ is true if null(0 byte) is a valid value for the column.
*
*/
public:
virtual size_t
DataByteSize() const = 0;
virtual const char*
MmappedData() const = 0;
virtual void
AppendBatch(const FieldDataPtr data) = 0;
virtual const char*
Data(int chunk_id) const = 0;
};
} // namespace milvus

View File

@ -19,9 +19,8 @@
#include <memory>
#include <string>
#include <vector>
#include "arrow/record_batch.h"
#include "common/FieldData.h"
#include "storage/DataCodec.h"
namespace milvus {
@ -39,18 +38,6 @@ struct FieldDataInfo {
arrow_reader_channel = std::make_shared<ArrowReaderChannel>();
}
FieldDataInfo(
int64_t field_id,
size_t row_count,
const std::vector<std::shared_ptr<milvus::ArrowDataWrapper>>& batch)
: field_id(field_id), row_count(row_count) {
arrow_reader_channel = std::make_shared<ArrowReaderChannel>();
for (auto& data : batch) {
arrow_reader_channel->push(data);
}
arrow_reader_channel->close();
}
int64_t field_id;
size_t row_count;
std::string mmap_dir_path;

View File

@ -326,4 +326,218 @@ DEFINE_PROMETHEUS_GAUGE(internal_cgo_executing_task_total_all,
internal_cgo_executing_task_total,
{});
// --- caching layer metrics ---
// TODO(tiered storage 1): choose better buckets.
std::map<std::string, std::string> cacheMemoryLabel = {{"location", "memory"}};
std::map<std::string, std::string> cacheDiskLabel = {{"location", "disk"}};
std::map<std::string, std::string> cacheMixedLabel = {{"location", "mixed"}};
// Cache slot count
DEFINE_PROMETHEUS_GAUGE_FAMILY(internal_cache_slot_count,
"[cpp]cache slot count");
DEFINE_PROMETHEUS_GAUGE(internal_cache_slot_count_memory,
internal_cache_slot_count,
cacheMemoryLabel);
DEFINE_PROMETHEUS_GAUGE(internal_cache_slot_count_disk,
internal_cache_slot_count,
cacheDiskLabel);
DEFINE_PROMETHEUS_GAUGE(internal_cache_slot_count_mixed,
internal_cache_slot_count,
cacheMixedLabel);
// Cache cell count
DEFINE_PROMETHEUS_GAUGE_FAMILY(internal_cache_cell_count,
"[cpp]cache cell count");
DEFINE_PROMETHEUS_GAUGE(internal_cache_cell_count_memory,
internal_cache_cell_count,
cacheMemoryLabel);
DEFINE_PROMETHEUS_GAUGE(internal_cache_cell_count_disk,
internal_cache_cell_count,
cacheDiskLabel);
DEFINE_PROMETHEUS_GAUGE(internal_cache_cell_count_mixed,
internal_cache_cell_count,
cacheMixedLabel);
// Cache cell loaded count
DEFINE_PROMETHEUS_GAUGE_FAMILY(internal_cache_cell_loaded_count,
"[cpp]cache cell loaded count");
DEFINE_PROMETHEUS_GAUGE(internal_cache_cell_loaded_count_memory,
internal_cache_cell_loaded_count,
cacheMemoryLabel);
DEFINE_PROMETHEUS_GAUGE(internal_cache_cell_loaded_count_disk,
internal_cache_cell_loaded_count,
cacheDiskLabel);
DEFINE_PROMETHEUS_GAUGE(internal_cache_cell_loaded_count_mixed,
internal_cache_cell_loaded_count,
cacheMixedLabel);
// Cache load latency histogram
DEFINE_PROMETHEUS_HISTOGRAM_FAMILY(internal_cache_load_latency,
"[cpp]cache load latency histogram");
DEFINE_PROMETHEUS_HISTOGRAM_WITH_BUCKETS(internal_cache_load_latency_memory,
internal_cache_load_latency,
cacheMemoryLabel,
secondsBuckets);
DEFINE_PROMETHEUS_HISTOGRAM_WITH_BUCKETS(internal_cache_load_latency_disk,
internal_cache_load_latency,
cacheDiskLabel,
secondsBuckets);
DEFINE_PROMETHEUS_HISTOGRAM_WITH_BUCKETS(internal_cache_load_latency_mixed,
internal_cache_load_latency,
cacheMixedLabel,
secondsBuckets);
// Cache hit rate (represented by hit/miss counters)
std::map<std::string, std::string> cacheHitMemoryLabels = {
{"result", "hit"}, {"location", "memory"}};
std::map<std::string, std::string> cacheHitDiskLabels = {{"result", "hit"},
{"location", "disk"}};
std::map<std::string, std::string> cacheHitMixedLabels = {
{"result", "hit"}, {"location", "mixed"}};
std::map<std::string, std::string> cacheMissMemoryLabels = {
{"result", "miss"}, {"location", "memory"}};
std::map<std::string, std::string> cacheMissDiskLabels = {{"result", "miss"},
{"location", "disk"}};
std::map<std::string, std::string> cacheMissMixedLabels = {
{"result", "miss"}, {"location", "mixed"}};
DEFINE_PROMETHEUS_COUNTER_FAMILY(internal_cache_op_result_count,
"[cpp]cache operation result count");
DEFINE_PROMETHEUS_COUNTER(internal_cache_op_result_count_hit_memory,
internal_cache_op_result_count,
cacheHitMemoryLabels);
DEFINE_PROMETHEUS_COUNTER(internal_cache_op_result_count_hit_disk,
internal_cache_op_result_count,
cacheHitDiskLabels);
DEFINE_PROMETHEUS_COUNTER(internal_cache_op_result_count_hit_mixed,
internal_cache_op_result_count,
cacheHitMixedLabels);
DEFINE_PROMETHEUS_COUNTER(internal_cache_op_result_count_miss_memory,
internal_cache_op_result_count,
cacheMissMemoryLabels);
DEFINE_PROMETHEUS_COUNTER(internal_cache_op_result_count_miss_disk,
internal_cache_op_result_count,
cacheMissDiskLabels);
DEFINE_PROMETHEUS_COUNTER(internal_cache_op_result_count_miss_mixed,
internal_cache_op_result_count,
cacheMissMixedLabels);
// Cache usage (bytes)
DEFINE_PROMETHEUS_GAUGE_FAMILY(internal_cache_used_bytes,
"[cpp]currently used bytes in cache");
DEFINE_PROMETHEUS_GAUGE(internal_cache_used_bytes_memory,
internal_cache_used_bytes,
cacheMemoryLabel);
DEFINE_PROMETHEUS_GAUGE(internal_cache_used_bytes_disk,
internal_cache_used_bytes,
cacheDiskLabel);
DEFINE_PROMETHEUS_GAUGE(internal_cache_used_bytes_mixed,
internal_cache_used_bytes,
cacheMixedLabel);
DEFINE_PROMETHEUS_GAUGE_FAMILY(internal_cache_capacity_bytes,
"[cpp]total capacity bytes of cache");
DEFINE_PROMETHEUS_GAUGE(internal_cache_capacity_bytes_memory,
internal_cache_capacity_bytes,
cacheMemoryLabel);
DEFINE_PROMETHEUS_GAUGE(internal_cache_capacity_bytes_disk,
internal_cache_capacity_bytes,
cacheDiskLabel);
DEFINE_PROMETHEUS_GAUGE(internal_cache_capacity_bytes_mixed,
internal_cache_capacity_bytes,
cacheMixedLabel);
// Eviction count and resource size
DEFINE_PROMETHEUS_COUNTER_FAMILY(internal_cache_eviction_count,
"[cpp]cache eviction count");
DEFINE_PROMETHEUS_COUNTER(internal_cache_eviction_count_memory,
internal_cache_eviction_count,
cacheMemoryLabel);
DEFINE_PROMETHEUS_COUNTER(internal_cache_eviction_count_disk,
internal_cache_eviction_count,
cacheDiskLabel);
DEFINE_PROMETHEUS_COUNTER(internal_cache_eviction_count_mixed,
internal_cache_eviction_count,
cacheMixedLabel);
DEFINE_PROMETHEUS_COUNTER_FAMILY(internal_cache_evicted_bytes,
"[cpp]total bytes evicted from cache");
DEFINE_PROMETHEUS_COUNTER(internal_cache_evicted_bytes_memory,
internal_cache_evicted_bytes,
cacheMemoryLabel);
DEFINE_PROMETHEUS_COUNTER(internal_cache_evicted_bytes_disk,
internal_cache_evicted_bytes,
cacheDiskLabel);
DEFINE_PROMETHEUS_COUNTER(internal_cache_evicted_bytes_mixed,
internal_cache_evicted_bytes,
cacheMixedLabel);
// Cache item lifetime histogram
DEFINE_PROMETHEUS_HISTOGRAM_FAMILY(internal_cache_item_lifetime_seconds,
"[cpp]cache item lifetime histogram");
DEFINE_PROMETHEUS_HISTOGRAM_WITH_BUCKETS(
internal_cache_item_lifetime_seconds_memory,
internal_cache_item_lifetime_seconds,
cacheMemoryLabel,
secondsBuckets);
DEFINE_PROMETHEUS_HISTOGRAM_WITH_BUCKETS(
internal_cache_item_lifetime_seconds_disk,
internal_cache_item_lifetime_seconds,
cacheDiskLabel,
secondsBuckets);
DEFINE_PROMETHEUS_HISTOGRAM_WITH_BUCKETS(
internal_cache_item_lifetime_seconds_mixed,
internal_cache_item_lifetime_seconds,
cacheMixedLabel,
secondsBuckets);
// Load error rate (represented by success/fail counters)
std::map<std::string, std::string> cacheLoadSuccessMemoryLabels = {
{"status", "success"}, {"location", "memory"}};
std::map<std::string, std::string> cacheLoadSuccessDiskLabels = {
{"status", "success"}, {"location", "disk"}};
std::map<std::string, std::string> cacheLoadSuccessMixedLabels = {
{"status", "success"}, {"location", "mixed"}};
std::map<std::string, std::string> cacheLoadFailMemoryLabels = {
{"status", "fail"}, {"location", "memory"}};
std::map<std::string, std::string> cacheLoadFailDiskLabels = {
{"status", "fail"}, {"location", "disk"}};
std::map<std::string, std::string> cacheLoadFailMixedLabels = {
{"status", "fail"}, {"location", "mixed"}};
DEFINE_PROMETHEUS_COUNTER_FAMILY(internal_cache_load_count,
"[cpp]cache load operation count");
DEFINE_PROMETHEUS_COUNTER(internal_cache_load_count_success_memory,
internal_cache_load_count,
cacheLoadSuccessMemoryLabels);
DEFINE_PROMETHEUS_COUNTER(internal_cache_load_count_success_disk,
internal_cache_load_count,
cacheLoadSuccessDiskLabels);
DEFINE_PROMETHEUS_COUNTER(internal_cache_load_count_success_mixed,
internal_cache_load_count,
cacheLoadSuccessMixedLabels);
DEFINE_PROMETHEUS_COUNTER(internal_cache_load_count_fail_memory,
internal_cache_load_count,
cacheLoadFailMemoryLabels);
DEFINE_PROMETHEUS_COUNTER(internal_cache_load_count_fail_disk,
internal_cache_load_count,
cacheLoadFailDiskLabels);
DEFINE_PROMETHEUS_COUNTER(internal_cache_load_count_fail_mixed,
internal_cache_load_count,
cacheLoadFailMixedLabels);
// Cache system memory overhead (bytes)
DEFINE_PROMETHEUS_GAUGE_FAMILY(internal_cache_memory_overhead_bytes,
"[cpp]cache system memory overhead in bytes");
DEFINE_PROMETHEUS_GAUGE(internal_cache_memory_overhead_bytes_memory,
internal_cache_memory_overhead_bytes,
cacheMemoryLabel);
DEFINE_PROMETHEUS_GAUGE(internal_cache_memory_overhead_bytes_disk,
internal_cache_memory_overhead_bytes,
cacheDiskLabel);
DEFINE_PROMETHEUS_GAUGE(internal_cache_memory_overhead_bytes_mixed,
internal_cache_memory_overhead_bytes,
cacheMixedLabel);
// --- caching layer metrics end ---
} // namespace milvus::monitor

View File

@ -158,4 +158,72 @@ DECLARE_PROMETHEUS_GAUGE(internal_cgo_inflight_task_total_all);
DECLARE_PROMETHEUS_GAUGE_FAMILY(internal_cgo_executing_task_total);
DECLARE_PROMETHEUS_GAUGE(internal_cgo_executing_task_total_all);
// --- caching layer metrics ---
DECLARE_PROMETHEUS_GAUGE_FAMILY(internal_cache_slot_count);
DECLARE_PROMETHEUS_GAUGE(internal_cache_slot_count_memory);
DECLARE_PROMETHEUS_GAUGE(internal_cache_slot_count_disk);
DECLARE_PROMETHEUS_GAUGE(internal_cache_slot_count_mixed);
DECLARE_PROMETHEUS_GAUGE_FAMILY(internal_cache_cell_count);
DECLARE_PROMETHEUS_GAUGE(internal_cache_cell_count_memory);
DECLARE_PROMETHEUS_GAUGE(internal_cache_cell_count_disk);
DECLARE_PROMETHEUS_GAUGE(internal_cache_cell_count_mixed);
DECLARE_PROMETHEUS_GAUGE_FAMILY(internal_cache_cell_loaded_count);
DECLARE_PROMETHEUS_GAUGE(internal_cache_cell_loaded_count_memory);
DECLARE_PROMETHEUS_GAUGE(internal_cache_cell_loaded_count_disk);
DECLARE_PROMETHEUS_GAUGE(internal_cache_cell_loaded_count_mixed);
DECLARE_PROMETHEUS_HISTOGRAM_FAMILY(internal_cache_load_latency);
DECLARE_PROMETHEUS_HISTOGRAM(internal_cache_load_latency_memory);
DECLARE_PROMETHEUS_HISTOGRAM(internal_cache_load_latency_disk);
DECLARE_PROMETHEUS_HISTOGRAM(internal_cache_load_latency_mixed);
DECLARE_PROMETHEUS_COUNTER_FAMILY(internal_cache_op_result_count);
DECLARE_PROMETHEUS_COUNTER(internal_cache_op_result_count_hit_memory);
DECLARE_PROMETHEUS_COUNTER(internal_cache_op_result_count_hit_disk);
DECLARE_PROMETHEUS_COUNTER(internal_cache_op_result_count_hit_mixed);
DECLARE_PROMETHEUS_COUNTER(internal_cache_op_result_count_miss_memory);
DECLARE_PROMETHEUS_COUNTER(internal_cache_op_result_count_miss_disk);
DECLARE_PROMETHEUS_COUNTER(internal_cache_op_result_count_miss_mixed);
DECLARE_PROMETHEUS_GAUGE_FAMILY(internal_cache_used_bytes);
DECLARE_PROMETHEUS_GAUGE(internal_cache_used_bytes_memory);
DECLARE_PROMETHEUS_GAUGE(internal_cache_used_bytes_disk);
DECLARE_PROMETHEUS_GAUGE_FAMILY(internal_cache_capacity_bytes);
DECLARE_PROMETHEUS_GAUGE(internal_cache_capacity_bytes_memory);
DECLARE_PROMETHEUS_GAUGE(internal_cache_capacity_bytes_disk);
DECLARE_PROMETHEUS_COUNTER_FAMILY(internal_cache_eviction_count);
DECLARE_PROMETHEUS_COUNTER(internal_cache_eviction_count_memory);
DECLARE_PROMETHEUS_COUNTER(internal_cache_eviction_count_disk);
DECLARE_PROMETHEUS_COUNTER(internal_cache_eviction_count_mixed);
DECLARE_PROMETHEUS_COUNTER_FAMILY(internal_cache_evicted_bytes);
DECLARE_PROMETHEUS_COUNTER(internal_cache_evicted_bytes_memory);
DECLARE_PROMETHEUS_COUNTER(internal_cache_evicted_bytes_disk);
DECLARE_PROMETHEUS_HISTOGRAM_FAMILY(internal_cache_item_lifetime_seconds);
DECLARE_PROMETHEUS_HISTOGRAM(internal_cache_item_lifetime_seconds_memory);
DECLARE_PROMETHEUS_HISTOGRAM(internal_cache_item_lifetime_seconds_disk);
DECLARE_PROMETHEUS_HISTOGRAM(internal_cache_item_lifetime_seconds_mixed);
DECLARE_PROMETHEUS_COUNTER_FAMILY(internal_cache_load_count);
DECLARE_PROMETHEUS_COUNTER(internal_cache_load_count_success_memory);
DECLARE_PROMETHEUS_COUNTER(internal_cache_load_count_success_disk);
DECLARE_PROMETHEUS_COUNTER(internal_cache_load_count_success_mixed);
DECLARE_PROMETHEUS_COUNTER(internal_cache_load_count_fail_memory);
DECLARE_PROMETHEUS_COUNTER(internal_cache_load_count_fail_disk);
DECLARE_PROMETHEUS_COUNTER(internal_cache_load_count_fail_mixed);
// TODO(tiered storage 1): not added
DECLARE_PROMETHEUS_GAUGE_FAMILY(internal_cache_memory_overhead_bytes);
DECLARE_PROMETHEUS_GAUGE(internal_cache_memory_overhead_bytes_memory);
DECLARE_PROMETHEUS_GAUGE(internal_cache_memory_overhead_bytes_disk);
DECLARE_PROMETHEUS_GAUGE(internal_cache_memory_overhead_bytes_mixed);
// --- caching layer metrics end ---
} // namespace milvus::monitor

View File

@ -9,9 +9,11 @@
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#include <algorithm>
#include "mmap/ChunkedColumn.h"
#include "query/CachedSearchIterator.h"
#include "query/SearchBruteForce.h"
#include <algorithm>
namespace milvus::query {
@ -72,9 +74,10 @@ 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};
offset, query_ds.dim, chunk_size, chunk_data.get()};
auto expected_iterators = GetBruteForceSearchIterators(
query_ds, sub_data, search_info, index_info, bitset, data_type);
@ -121,17 +124,17 @@ CachedSearchIterator::CachedSearchIterator(
index_info,
bitset,
data_type,
[&vec_data, vec_size_per_chunk, row_count](
int64_t chunk_id) -> std::pair<const void*, int64_t> {
const auto chunk_data = vec_data->get_chunk_data(chunk_id);
[&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);
int64_t chunk_size = std::min(
vec_size_per_chunk, row_count - chunk_id * vec_size_per_chunk);
return {chunk_data, chunk_size};
return std::make_pair(pw, chunk_size);
});
}
CachedSearchIterator::CachedSearchIterator(
const std::shared_ptr<ChunkedColumnBase>& column,
ChunkedColumnBase* column,
const dataset::SearchDataset& query_ds,
const SearchInfo& search_info,
const std::map<std::string, std::string>& index_info,
@ -153,11 +156,11 @@ CachedSearchIterator::CachedSearchIterator(
index_info,
bitset,
data_type,
[&column](int64_t chunk_id) {
const char* chunk_data = column->Data(chunk_id);
[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(static_cast<const void*>(chunk_data),
chunk_size);
return std::make_pair(pw, chunk_size);
});
}

View File

@ -12,12 +12,15 @@
#pragma once
#include <utility>
#include "cachinglayer/CacheSlot.h"
#include "common/BitsetView.h"
#include "common/QueryInfo.h"
#include "common/QueryResult.h"
#include "query/helper.h"
#include "segcore/ConcurrentVector.h"
#include "index/VectorIndex.h"
#include "mmap/ChunkedColumn.h"
namespace milvus::query {
@ -58,7 +61,7 @@ class CachedSearchIterator {
const milvus::DataType& data_type);
// For sealed segment with chunked data, BF
CachedSearchIterator(const std::shared_ptr<ChunkedColumnBase>& column,
CachedSearchIterator(ChunkedColumnBase* column,
const dataset::SearchDataset& dataset,
const SearchInfo& search_info,
const std::map<std::string, std::string>& index_info,
@ -83,7 +86,8 @@ class CachedSearchIterator {
using IterIdx = size_t;
using IterIdDisIdPair = std::pair<IterIdx, DisIdPair>;
using GetChunkDataFunc =
std::function<std::pair<const void*, int64_t>(int64_t)>;
std::function<std::pair<milvus::cachinglayer::PinWrapper<const void*>,
int64_t>(int64_t)>;
int64_t batch_size_ = 0;
std::vector<knowhere::IndexNode::IteratorPtr> iterators_;

View File

@ -17,7 +17,6 @@
#include "common/BitsetView.h"
#include "common/QueryInfo.h"
#include "common/Types.h"
#include "mmap/Column.h"
#include "query/CachedSearchIterator.h"
#include "query/SearchBruteForce.h"
#include "query/SearchOnSealed.h"
@ -86,15 +85,15 @@ SearchOnSealedIndex(const Schema& schema,
}
void
SearchOnSealed(const Schema& schema,
std::shared_ptr<ChunkedColumnBase> column,
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& bitview,
SearchResult& result) {
SearchOnSealedColumn(const Schema& schema,
ChunkedColumnBase* column,
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& bitview,
SearchResult& result) {
auto field_id = search_info.field_id_;
auto& field = schema[field_id];
@ -129,7 +128,8 @@ SearchOnSealed(const Schema& schema,
auto offset = 0;
for (int i = 0; i < num_chunk; ++i) {
auto vec_data = column->Data(i);
auto pw = column->DataOfChunk(i);
auto vec_data = pw.get();
auto chunk_size = column->chunk_row_nums(i);
auto raw_dataset =
query::dataset::RawDataset{offset, dim, chunk_size, vec_data};
@ -167,15 +167,15 @@ SearchOnSealed(const Schema& schema,
}
void
SearchOnSealed(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) {
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];

View File

@ -28,25 +28,25 @@ SearchOnSealedIndex(const Schema& schema,
SearchResult& search_result);
void
SearchOnSealed(const Schema& schema,
std::shared_ptr<ChunkedColumnBase> column,
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);
SearchOnSealedColumn(const Schema& schema,
ChunkedColumnBase* column,
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);
void
SearchOnSealed(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);
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

File diff suppressed because it is too large Load Diff

View File

@ -14,30 +14,36 @@
#include <tbb/concurrent_priority_queue.h>
#include <tbb/concurrent_vector.h>
#include <deque>
#include <map>
#include <memory>
#include <string>
#include <unordered_map>
#include <utility>
#include <vector>
#include "folly/executors/InlineExecutor.h"
#include "ConcurrentVector.h"
#include "DeletedRecord.h"
#include "SealedIndexingRecord.h"
#include "SegmentSealed.h"
#include "TimestampIndex.h"
#include "common/EasyAssert.h"
#include "common/Schema.h"
#include "google/protobuf/message_lite.h"
#include "mmap/ChunkedColumn.h"
#include "index/ScalarIndex.h"
#include "sys/mman.h"
#include "mmap/Types.h"
#include "common/Types.h"
#include "common/IndexMeta.h"
#include "cachinglayer/CacheSlot.h"
#include "cachinglayer/CacheSlot.h"
namespace milvus::segcore {
namespace storagev1translator {
class InsertRecordTranslator;
}
using namespace milvus::cachinglayer;
class ChunkedSegmentSealedImpl : public SegmentSealed {
public:
explicit ChunkedSegmentSealedImpl(SchemaPtr schema,
@ -67,13 +73,9 @@ class ChunkedSegmentSealedImpl : public SegmentSealed {
bool
Contain(const PkType& pk) const override {
return insert_record_.contain(pk);
return pin_insert_record()->get_cell_of(0)->contain(pk);
}
void
LoadFieldData(FieldId field_id, FieldDataInfo& data) override;
void
MapFieldData(const FieldId field_id, FieldDataInfo& data) override;
void
AddFieldDataInfoForSealed(
const LoadFieldDataInfo& field_data_info) override;
@ -117,11 +119,17 @@ class ChunkedSegmentSealedImpl : public SegmentSealed {
return iter->second.get();
}
std::pair<std::string_view, bool>
// TODO(tiered storage 1): should return a PinWrapper
std::pair<milvus::Json, bool>
GetJsonData(FieldId field_id, size_t offset) const override {
auto column = fields_.at(field_id);
auto column =
std::dynamic_pointer_cast<ChunkedVariableColumn<milvus::Json>>(
fields_.at(field_id));
bool is_valid = column->IsValid(offset);
return std::make_pair(std::move(column->RawAt(offset)), is_valid);
if (!is_valid) {
return std::make_pair(milvus::Json(), false);
}
return std::make_pair(milvus::Json(column->RawAt(offset)), is_valid);
}
void
@ -136,12 +144,7 @@ class ChunkedSegmentSealedImpl : public SegmentSealed {
public:
size_t
GetMemoryUsageInBytes() const override {
return stats_.mem_size.load() + deleted_record_.mem_size();
}
InsertRecord<true>&
get_insert_record() override {
return insert_record_;
return stats_.mem_size.load() + deleted_record_->mem_size();
}
int64_t
@ -156,9 +159,6 @@ class ChunkedSegmentSealedImpl : public SegmentSealed {
std::vector<SegOffset>
search_pk(const PkType& pk, Timestamp timestamp) const override;
std::vector<SegOffset>
search_pk(const PkType& pk, int64_t insert_barrier) const override;
template <typename Condition>
std::vector<SegOffset>
search_sorted_pk(const PkType& pk, Condition condition) const;
@ -170,11 +170,8 @@ class ChunkedSegmentSealedImpl : public SegmentSealed {
bool
is_nullable(FieldId field_id) const override {
auto it = fields_.find(field_id);
AssertInfo(it != fields_.end(),
"Cannot find field with field_id: " +
std::to_string(field_id.get()));
return it->second->IsNullable();
auto& field_meta = schema_->operator[](field_id);
return field_meta.is_nullable();
};
bool
@ -210,8 +207,7 @@ class ChunkedSegmentSealedImpl : public SegmentSealed {
debug() const override;
SegcoreError
Delete(int64_t reserved_offset,
int64_t size,
Delete(int64_t size,
const IdArray* pks,
const Timestamp* timestamps) override;
@ -246,32 +242,26 @@ class ChunkedSegmentSealedImpl : public SegmentSealed {
protected:
// blob and row_count
SpanBase
PinWrapper<SpanBase>
chunk_data_impl(FieldId field_id, int64_t chunk_id) const override;
std::pair<std::vector<std::string_view>, FixedVector<bool>>
PinWrapper<std::pair<std::vector<std::string_view>, FixedVector<bool>>>
chunk_string_view_impl(
FieldId field_id,
int64_t chunk_id,
std::optional<std::pair<int64_t, int64_t>> offset_len) const override;
std::pair<std::vector<ArrayView>, FixedVector<bool>>
PinWrapper<std::pair<std::vector<ArrayView>, FixedVector<bool>>>
chunk_array_view_impl(
FieldId field_id,
int64_t chunk_id,
std::optional<std::pair<int64_t, int64_t>> offset_len) const override;
std::pair<std::vector<std::string_view>, FixedVector<bool>>
PinWrapper<std::pair<std::vector<std::string_view>, FixedVector<bool>>>
chunk_view_by_offsets(FieldId field_id,
int64_t chunk_id,
const FixedVector<int32_t>& offsets) const override;
std::pair<BufferView, FixedVector<bool>>
get_chunk_buffer(FieldId field_id,
int64_t chunk_id,
int64_t start_offset,
int64_t length) const override;
const index::IndexBase*
chunk_index_impl(FieldId field_id, int64_t chunk_id) const override;
@ -291,10 +281,13 @@ class ChunkedSegmentSealedImpl : public SegmentSealed {
const ConcurrentVector<Timestamp>&
get_timestamps() const override {
return insert_record_.timestamps_;
return pin_insert_record()->get_cell_of(0)->timestamps_;
}
private:
void
LoadSystemFieldInternal(FieldId field_id, FieldDataInfo& data);
template <typename S, typename T = S>
static void
bulk_subscript_impl(const void* src_raw,
@ -304,35 +297,28 @@ class ChunkedSegmentSealedImpl : public SegmentSealed {
template <typename S, typename T = S>
static void
bulk_subscript_impl(const ChunkedColumnBase* field,
bulk_subscript_impl(ChunkedColumnBase* field,
const int64_t* seg_offsets,
int64_t count,
T* dst_raw);
template <typename S, typename T = S>
static void
bulk_subscript_impl(const ChunkedColumnBase* field,
const int64_t* seg_offsets,
int64_t count,
void* dst_raw);
template <typename S, typename T = S>
static void
bulk_subscript_ptr_impl(const ChunkedColumnBase* field,
bulk_subscript_ptr_impl(ChunkedColumnBase* field,
const int64_t* seg_offsets,
int64_t count,
google::protobuf::RepeatedPtrField<T>* dst_raw);
template <typename T>
static void
bulk_subscript_array_impl(const ChunkedColumnBase* column,
bulk_subscript_array_impl(ChunkedColumnBase* column,
const int64_t* seg_offsets,
int64_t count,
google::protobuf::RepeatedPtrField<T>* dst);
static void
bulk_subscript_impl(int64_t element_sizeof,
const ChunkedColumnBase* field,
ChunkedColumnBase* field,
const int64_t* seg_offsets,
int64_t count,
void* dst_raw);
@ -348,12 +334,8 @@ class ChunkedSegmentSealedImpl : public SegmentSealed {
void
update_row_count(int64_t row_count) {
// if (row_count_opt_.has_value()) {
// AssertInfo(row_count_opt_.value() == row_count, "load data has different row count from other columns");
// } else {
num_rows_ = row_count;
// }
deleted_record_.set_sealed_row_count(row_count);
deleted_record_->set_sealed_row_count(row_count);
}
void
@ -375,31 +357,34 @@ class ChunkedSegmentSealedImpl : public SegmentSealed {
bool
is_system_field_ready() const {
return system_ready_count_ == 2;
return system_ready_count_ == 1;
}
std::pair<std::unique_ptr<IdArray>, std::vector<SegOffset>>
search_ids(const IdArray& id_array, Timestamp timestamp) const override;
std::tuple<std::string, int64_t>
GetFieldDataPath(FieldId field_id, int64_t offset) const;
void
LoadVecIndex(const LoadIndexInfo& info);
void
LoadScalarIndex(const LoadIndexInfo& info);
void
WarmupChunkCache(const FieldId field_id, bool mmap_enabled) override;
bool
generate_interim_index(const FieldId field_id);
void
fill_empty_field(const FieldMeta& field_meta);
// used only by unit test
std::shared_ptr<CacheSlot<InsertRecord<true>>>
get_insert_record_slot() const override {
return insert_record_slot_;
}
private:
// InsertRecord needs to pin pk column.
friend class storagev1translator::InsertRecordTranslator;
// mmap descriptor, used in chunk cache
storage::MmapChunkDescriptorPtr mmap_descriptor_ = nullptr;
// segment loading state
@ -417,17 +402,25 @@ class ChunkedSegmentSealedImpl : public SegmentSealed {
// vector field index
SealedIndexingRecord vector_indexings_;
std::shared_ptr<CellAccessor<InsertRecord<true>>>
pin_insert_record() const {
return insert_record_slot_->PinCells({0})
.via(&folly::InlineExecutor::instance())
.get();
}
// inserted fields data and row_ids, timestamps
InsertRecord<true> insert_record_;
mutable std::shared_ptr<CacheSlot<InsertRecord<true>>> insert_record_slot_;
// deleted pks
mutable DeletedRecord<true> deleted_record_;
mutable std::unique_ptr<DeletedRecord<true>> deleted_record_;
LoadFieldDataInfo field_data_info_;
SchemaPtr schema_;
int64_t id_;
std::unordered_map<FieldId, std::shared_ptr<ChunkedColumnBase>> fields_;
mutable std::unordered_map<FieldId, std::shared_ptr<ChunkedColumnBase>>
fields_;
std::unordered_set<FieldId> mmap_fields_;
// only useful in binlog
@ -443,6 +436,7 @@ class ChunkedSegmentSealedImpl : public SegmentSealed {
bool TEST_skip_index_for_retrieve_ = false;
// whether the segment is sorted by the pk
// 1. will skip index loading for primary key field
bool is_sorted_by_pk_ = false;
// used for json expr optimization
std::unordered_map<FieldId,
@ -453,7 +447,7 @@ class ChunkedSegmentSealedImpl : public SegmentSealed {
inline SegmentSealedUPtr
CreateSealedSegment(
SchemaPtr schema,
IndexMetaPtr index_meta = nullptr,
IndexMetaPtr index_meta = empty_index_meta,
int64_t segment_id = 0,
const SegcoreConfig& segcore_config = SegcoreConfig::default_config(),
bool TEST_skip_index_for_retrieve = false,

View File

@ -100,58 +100,6 @@ class ThreadSafeValidData {
};
using ThreadSafeValidDataPtr = std::shared_ptr<ThreadSafeValidData>;
template <typename Type>
class ThreadSafeVector {
public:
template <typename... Args>
void
emplace_to_at_least(int64_t size, Args... args) {
std::lock_guard lck(mutex_);
if (size <= size_) {
return;
}
while (vec_.size() < size) {
vec_.emplace_back(std::forward<Args...>(args...));
++size_;
}
}
const Type&
operator[](int64_t index) const {
std::shared_lock lck(mutex_);
AssertInfo(index < size_,
fmt::format(
"index out of range, index={}, size_={}", index, size_));
return vec_[index];
}
Type&
operator[](int64_t index) {
std::shared_lock lck(mutex_);
AssertInfo(index < size_,
fmt::format(
"index out of range, index={}, size_={}", index, size_));
return vec_[index];
}
int64_t
size() const {
std::shared_lock lck(mutex_);
return size_;
}
void
clear() {
std::lock_guard lck(mutex_);
size_ = 0;
vec_.clear();
}
private:
int64_t size_ = 0;
std::deque<Type> vec_;
mutable std::shared_mutex mutex_;
};
class VectorBase {
public:
explicit VectorBase(int64_t size_per_chunk)

View File

@ -19,13 +19,9 @@
#include <vector>
#include <folly/ConcurrentSkipList.h>
#include "AckResponder.h"
#include "common/Schema.h"
#include "cachinglayer/CacheSlot.h"
#include "common/Types.h"
#include "segcore/Record.h"
#include "segcore/InsertRecord.h"
#include "segcore/SegmentInterface.h"
#include "ConcurrentVector.h"
namespace milvus::segcore {
@ -52,7 +48,12 @@ static int32_t DELETE_PAIR_SIZE = sizeof(std::pair<Timestamp, Offset>);
template <bool is_sealed = false>
class DeletedRecord {
public:
DeletedRecord(InsertRecord<is_sealed>* insert_record,
using InsertRecordType = std::conditional_t<
is_sealed,
std::shared_ptr<milvus::cachinglayer::CacheSlot<InsertRecord<true>>>,
InsertRecord<false>*>;
DeletedRecord(InsertRecordType insert_record,
std::function<std::vector<SegOffset>(
const PkType& pk, Timestamp timestamp)> search_pk_func,
int64_t segment_id)
@ -108,6 +109,18 @@ class DeletedRecord {
Timestamp max_timestamp = 0;
SortedDeleteList::Accessor accessor(deleted_lists_);
InsertRecord<is_sealed>* insert_record = nullptr;
std::shared_ptr<
milvus::cachinglayer::CellAccessor<InsertRecord<is_sealed>>>
cell_accessor{nullptr};
if constexpr (is_sealed) {
cell_accessor = insert_record_->PinCells({0})
.via(&folly::InlineExecutor::instance())
.get();
insert_record = cell_accessor->get_cell_of(0);
} else {
insert_record = insert_record_;
}
for (size_t i = 0; i < pks.size(); ++i) {
auto deleted_pk = pks[i];
auto deleted_ts = timestamps[i];
@ -124,7 +137,7 @@ class DeletedRecord {
}
// if insert record and delete record is same timestamp,
// delete not take effect on this record.
if (deleted_ts == insert_record_->timestamps_[row_id]) {
if (deleted_ts == insert_record->timestamps_[row_id]) {
continue;
}
accessor.insert(std::make_pair(deleted_ts, row_id));
@ -133,7 +146,7 @@ class DeletedRecord {
deleted_mask_.set(row_id);
} else {
// need to add mask size firstly for growing segment
deleted_mask_.resize(insert_record_->size());
deleted_mask_.resize(insert_record->size());
deleted_mask_.set(row_id);
}
removed_num++;
@ -322,7 +335,7 @@ class DeletedRecord {
public:
std::atomic<int64_t> n_ = 0;
std::atomic<int64_t> mem_size_ = 0;
InsertRecord<is_sealed>* insert_record_;
InsertRecordType insert_record_;
std::function<std::vector<SegOffset>(const PkType& pk, Timestamp timestamp)>
search_pk_func_;
int64_t segment_id_{0};

View File

@ -20,7 +20,6 @@
#include <index/Index.h>
#include <index/ScalarIndex.h>
#include "AckResponder.h"
#include "InsertRecord.h"
#include "common/FieldMeta.h"
#include "common/Schema.h"
@ -29,6 +28,7 @@
#include "knowhere/config.h"
#include "log/Log.h"
#include "segcore/SegcoreConfig.h"
#include "segcore/InsertRecord.h"
#include "index/VectorIndex.h"
namespace milvus::segcore {
@ -306,13 +306,12 @@ class IndexingRecord {
}
// concurrent, reentrant
template <bool is_sealed>
void
AppendingIndex(int64_t reserved_offset,
int64_t size,
FieldId fieldId,
const DataArray* stream_data,
const InsertRecord<is_sealed>& record) {
const InsertRecord<false>& record) {
if (!is_in(fieldId)) {
return;
}
@ -339,13 +338,12 @@ class IndexingRecord {
}
// concurrent, reentrant
template <bool is_sealed>
void
AppendingIndex(int64_t reserved_offset,
int64_t size,
FieldId fieldId,
const FieldDataPtr data,
const InsertRecord<is_sealed>& record) {
const InsertRecord<false>& record) {
if (!is_in(fieldId)) {
return;
}

View File

@ -20,19 +20,14 @@
#include <unordered_map>
#include <utility>
#include <vector>
#include <queue>
#include "TimestampIndex.h"
#include "common/EasyAssert.h"
#include "common/Schema.h"
#include "common/Types.h"
#include "fmt/format.h"
#include "mmap/ChunkedColumn.h"
#include "mmap/Column.h"
#include "segcore/AckResponder.h"
#include "segcore/ConcurrentVector.h"
#include "segcore/Record.h"
#include "storage/MmapManager.h"
namespace milvus::segcore {
@ -279,39 +274,35 @@ class OffsetOrderedArray : public OffsetMap {
std::vector<std::pair<T, int32_t>> array_;
};
template <bool is_sealed = false>
template <bool is_sealed>
struct InsertRecord {
public:
InsertRecord(
const Schema& schema,
const int64_t size_per_chunk,
const storage::MmapChunkDescriptorPtr mmap_descriptor = nullptr)
const storage::MmapChunkDescriptorPtr mmap_descriptor = nullptr,
bool called_from_subclass = false)
: timestamps_(size_per_chunk), mmap_descriptor_(mmap_descriptor) {
if (called_from_subclass) {
return;
}
std::optional<FieldId> pk_field_id = schema.get_primary_field_id();
// for sealed segment, only pk field is added.
for (auto& field : schema) {
auto field_id = field.first;
auto& field_meta = field.second;
if (pk2offset_ == nullptr && pk_field_id.has_value() &&
pk_field_id.value() == field_id) {
if (pk_field_id.has_value() && pk_field_id.value() == field_id) {
AssertInfo(!field_meta.is_nullable(),
"Primary key should not be nullable");
switch (field_meta.get_data_type()) {
case DataType::INT64: {
if constexpr (is_sealed) {
pk2offset_ =
std::make_unique<OffsetOrderedArray<int64_t>>();
} else {
pk2offset_ =
std::make_unique<OffsetOrderedMap<int64_t>>();
}
pk2offset_ =
std::make_unique<OffsetOrderedArray<int64_t>>();
break;
}
case DataType::VARCHAR: {
if constexpr (is_sealed) {
pk2offset_ = std::make_unique<
OffsetOrderedArray<std::string>>();
} else {
pk2offset_ = std::make_unique<
OffsetOrderedMap<std::string>>();
}
pk2offset_ =
std::make_unique<OffsetOrderedArray<std::string>>();
break;
}
default: {
@ -321,7 +312,6 @@ struct InsertRecord {
}
}
}
append_field_meta(field_id, field_meta, size_per_chunk);
}
}
@ -344,18 +334,16 @@ struct InsertRecord {
}
void
insert_pks(milvus::DataType data_type,
const std::shared_ptr<ChunkedColumnBase>& data) {
insert_pks(milvus::DataType data_type, ChunkedColumnBase* data) {
std::lock_guard lck(shared_mutex_);
int64_t offset = 0;
switch (data_type) {
case DataType::INT64: {
auto column = std::dynamic_pointer_cast<ChunkedColumn>(data);
auto num_chunk = column->num_chunks();
auto num_chunk = data->num_chunks();
for (int i = 0; i < num_chunk; ++i) {
auto pks =
reinterpret_cast<const int64_t*>(column->Data(i));
auto chunk_num_rows = column->chunk_row_nums(i);
auto pw = data->DataOfChunk(i);
auto pks = reinterpret_cast<const int64_t*>(pw.get());
auto chunk_num_rows = data->chunk_row_nums(i);
for (int j = 0; j < chunk_num_rows; ++j) {
pk2offset_->insert(pks[j], offset++);
}
@ -363,12 +351,10 @@ struct InsertRecord {
break;
}
case DataType::VARCHAR: {
auto column = std::dynamic_pointer_cast<
ChunkedVariableColumn<std::string>>(data);
auto num_chunk = column->num_chunks();
auto num_chunk = data->num_chunks();
for (int i = 0; i < num_chunk; ++i) {
auto pks = column->StringViews(i).first;
auto pw = data->StringViews(i);
auto pks = pw.get().first;
for (auto& pk : pks) {
pk2offset_->insert(std::string(pk), offset++);
}
@ -383,6 +369,90 @@ struct InsertRecord {
}
}
bool
empty_pks() const {
std::shared_lock lck(shared_mutex_);
return pk2offset_->empty();
}
void
seal_pks() {
std::lock_guard lck(shared_mutex_);
pk2offset_->seal();
}
const ConcurrentVector<Timestamp>&
timestamps() const {
return timestamps_;
}
virtual void
clear() {
timestamps_.clear();
timestamp_index_ = TimestampIndex();
pk2offset_->clear();
reserved = 0;
}
size_t
CellByteSize() const {
return 0;
}
public:
ConcurrentVector<Timestamp> timestamps_;
std::atomic<int64_t> reserved = 0;
// used for timestamps index of sealed segment
TimestampIndex timestamp_index_;
// pks to row offset
std::unique_ptr<OffsetMap> pk2offset_;
protected:
storage::MmapChunkDescriptorPtr mmap_descriptor_;
std::unordered_map<FieldId, std::unique_ptr<VectorBase>> data_{};
mutable std::shared_mutex shared_mutex_{};
};
template <>
struct InsertRecord<false> : public InsertRecord<true> {
public:
InsertRecord(
const Schema& schema,
const int64_t size_per_chunk,
const storage::MmapChunkDescriptorPtr mmap_descriptor = nullptr)
: InsertRecord<true>(schema, size_per_chunk, mmap_descriptor, true) {
std::optional<FieldId> pk_field_id = schema.get_primary_field_id();
for (auto& field : schema) {
auto field_id = field.first;
auto& field_meta = field.second;
if (pk_field_id.has_value() && pk_field_id.value() == field_id) {
AssertInfo(!field_meta.is_nullable(),
"Primary key should not be nullable");
switch (field_meta.get_data_type()) {
case DataType::INT64: {
pk2offset_ =
std::make_unique<OffsetOrderedMap<int64_t>>();
break;
}
case DataType::VARCHAR: {
pk2offset_ =
std::make_unique<OffsetOrderedMap<std::string>>();
break;
}
default: {
PanicInfo(DataTypeInvalid,
fmt::format("unsupported pk type",
field_meta.get_data_type()));
}
}
}
append_field_meta(field_id, field_meta, size_per_chunk);
}
}
void
insert_pks(const std::vector<FieldDataPtr>& field_datas) {
std::lock_guard lck(shared_mutex_);
@ -416,103 +486,6 @@ struct InsertRecord {
}
}
std::vector<SegOffset>
search_pk(const PkType& pk, int64_t insert_barrier) const {
std::shared_lock lck(shared_mutex_);
std::vector<SegOffset> res_offsets;
auto offset_iter = pk2offset_->find(pk);
for (auto offset : offset_iter) {
if (offset < insert_barrier) {
res_offsets.emplace_back(offset);
}
}
return res_offsets;
}
void
insert_pk(const PkType& pk, int64_t offset) {
std::lock_guard lck(shared_mutex_);
pk2offset_->insert(pk, offset);
}
bool
empty_pks() const {
std::shared_lock lck(shared_mutex_);
return pk2offset_->empty();
}
void
seal_pks() {
std::lock_guard lck(shared_mutex_);
pk2offset_->seal();
}
// get data without knowing the type
VectorBase*
get_data_base(FieldId field_id) const {
AssertInfo(data_.find(field_id) != data_.end(),
"Cannot find field_data with field_id: " +
std::to_string(field_id.get()));
AssertInfo(data_.at(field_id) != nullptr,
"data_ at i is null" + std::to_string(field_id.get()));
return data_.at(field_id).get();
}
// get field data in given type, const version
template <typename Type>
const ConcurrentVector<Type>*
get_data(FieldId field_id) const {
auto base_ptr = get_data_base(field_id);
auto ptr = dynamic_cast<const ConcurrentVector<Type>*>(base_ptr);
Assert(ptr);
return ptr;
}
// get field data in given type, non-const version
template <typename Type>
ConcurrentVector<Type>*
get_data(FieldId field_id) {
auto base_ptr = get_data_base(field_id);
auto ptr = dynamic_cast<ConcurrentVector<Type>*>(base_ptr);
Assert(ptr);
return ptr;
}
ThreadSafeValidDataPtr
get_valid_data(FieldId field_id) const {
AssertInfo(valid_data_.find(field_id) != valid_data_.end(),
"Cannot find valid_data with field_id: " +
std::to_string(field_id.get()));
AssertInfo(valid_data_.at(field_id) != nullptr,
"valid_data_ at i is null" + std::to_string(field_id.get()));
return valid_data_.at(field_id);
}
bool
is_data_exist(FieldId field_id) const {
return data_.find(field_id) != data_.end();
}
bool
is_valid_data_exist(FieldId field_id) const {
return valid_data_.find(field_id) != valid_data_.end();
}
SpanBase
get_span_base(FieldId field_id, int64_t chunk_id) const {
auto data = get_data_base(field_id);
if (is_valid_data_exist(field_id)) {
auto size = data->get_chunk_size(chunk_id);
auto element_offset = data->get_element_offset(chunk_id);
return SpanBase(
data->get_chunk_data(chunk_id),
get_valid_data(field_id)->get_chunk_data(element_offset),
size,
data->get_element_size());
}
return data->get_span_base(chunk_id);
}
void
append_field_meta(FieldId field_id,
const FieldMeta& field_meta,
@ -602,18 +575,76 @@ struct InsertRecord {
}
}
// append a column of scalar or sparse float vector type
template <typename Type>
void
append_data(FieldId field_id, int64_t size_per_chunk) {
static_assert(IsScalar<Type> || IsSparse<Type>);
data_.emplace(
field_id,
std::make_unique<ConcurrentVector<Type>>(
size_per_chunk,
mmap_descriptor_,
is_valid_data_exist(field_id) ? get_valid_data(field_id)
: nullptr));
insert_pk(const PkType& pk, int64_t offset) {
std::lock_guard lck(shared_mutex_);
pk2offset_->insert(pk, offset);
}
// get data without knowing the type
VectorBase*
get_data_base(FieldId field_id) const {
AssertInfo(data_.find(field_id) != data_.end(),
"Cannot find field_data with field_id: " +
std::to_string(field_id.get()));
AssertInfo(data_.at(field_id) != nullptr,
"data_ at i is null" + std::to_string(field_id.get()));
return data_.at(field_id).get();
}
// get field data in given type, const version
template <typename Type>
const ConcurrentVector<Type>*
get_data(FieldId field_id) const {
auto base_ptr = get_data_base(field_id);
auto ptr = dynamic_cast<const ConcurrentVector<Type>*>(base_ptr);
Assert(ptr);
return ptr;
}
// get field data in given type, non-const version
template <typename Type>
ConcurrentVector<Type>*
get_data(FieldId field_id) {
auto base_ptr = get_data_base(field_id);
auto ptr = dynamic_cast<ConcurrentVector<Type>*>(base_ptr);
Assert(ptr);
return ptr;
}
ThreadSafeValidDataPtr
get_valid_data(FieldId field_id) const {
AssertInfo(valid_data_.find(field_id) != valid_data_.end(),
"Cannot find valid_data with field_id: " +
std::to_string(field_id.get()));
AssertInfo(valid_data_.at(field_id) != nullptr,
"valid_data_ at i is null" + std::to_string(field_id.get()));
return valid_data_.at(field_id);
}
bool
is_data_exist(FieldId field_id) const {
return data_.find(field_id) != data_.end();
}
bool
is_valid_data_exist(FieldId field_id) const {
return valid_data_.find(field_id) != valid_data_.end();
}
SpanBase
get_span_base(FieldId field_id, int64_t chunk_id) const {
auto data = get_data_base(field_id);
if (is_valid_data_exist(field_id)) {
auto size = data->get_chunk_size(chunk_id);
auto element_offset = data->get_element_offset(chunk_id);
return SpanBase(
data->get_chunk_data(chunk_id),
get_valid_data(field_id)->get_chunk_data(element_offset),
size,
data->get_element_size());
}
return data->get_span_base(chunk_id);
}
// append a column of scalar type
@ -633,55 +664,48 @@ struct InsertRecord {
dim, size_per_chunk, mmap_descriptor_));
}
// append a column of scalar or sparse float vector type
template <typename Type>
void
append_data(FieldId field_id, int64_t size_per_chunk) {
static_assert(IsScalar<Type> || IsSparse<Type>);
data_.emplace(
field_id,
std::make_unique<ConcurrentVector<Type>>(
size_per_chunk,
mmap_descriptor_,
is_valid_data_exist(field_id) ? get_valid_data(field_id)
: nullptr));
}
void
drop_field_data(FieldId field_id) {
data_.erase(field_id);
valid_data_.erase(field_id);
}
const ConcurrentVector<Timestamp>&
timestamps() const {
return timestamps_;
}
int64_t
size() const {
return ack_responder_.GetAck();
}
void
clear() {
timestamps_.clear();
reserved = 0;
ack_responder_.clear();
timestamp_index_ = TimestampIndex();
pk2offset_->clear();
data_.clear();
}
bool
empty() const {
return pk2offset_->empty();
}
void
clear() override {
InsertRecord<true>::clear();
data_.clear();
ack_responder_.clear();
}
public:
ConcurrentVector<Timestamp> timestamps_;
// used for preInsert of growing segment
std::atomic<int64_t> reserved = 0;
AckResponder ack_responder_;
// used for timestamps index of sealed segment
TimestampIndex timestamp_index_;
// pks to row offset
std::unique_ptr<OffsetMap> pk2offset_;
private:
std::unordered_map<FieldId, std::unique_ptr<VectorBase>> data_{};
std::unordered_map<FieldId, ThreadSafeValidDataPtr> valid_data_{};
mutable std::shared_mutex shared_mutex_{};
storage::MmapChunkDescriptorPtr mmap_descriptor_;
};
} // namespace milvus::segcore

View File

@ -11,6 +11,8 @@
#pragma once
#include "common/Types.h"
namespace milvus::segcore {
template <typename RecordType>

View File

@ -39,20 +39,24 @@ SegmentChunkReader::GetChunkDataAccessor(FieldId field_id,
};
}
}
auto chunk_info = segment_->chunk_data<T>(field_id, current_chunk_id);
// pw is captured by value, each time we need to access a new chunk, we need to
// pin a new Chunk.
auto pw = segment_->chunk_data<T>(field_id, current_chunk_id);
auto chunk_info = pw.get();
auto chunk_data = chunk_info.data();
auto chunk_valid_data = chunk_info.valid_data();
auto current_chunk_size = segment_->chunk_size(field_id, current_chunk_id);
return [=,
pw = std::move(pw),
&current_chunk_id,
&current_chunk_pos]() mutable -> const data_access_type {
if (current_chunk_pos >= current_chunk_size) {
current_chunk_id++;
current_chunk_pos = 0;
auto chunk_info =
segment_->chunk_data<T>(field_id, current_chunk_id);
chunk_data = chunk_info.data();
chunk_valid_data = chunk_info.valid_data();
// the old chunk will be unpinned, pw will now pin the new chunk.
pw = segment_->chunk_data<T>(field_id, current_chunk_id);
chunk_data = pw.get().data();
chunk_valid_data = pw.get().valid_data();
current_chunk_size =
segment_->chunk_size(field_id, current_chunk_id);
}
@ -92,26 +96,28 @@ SegmentChunkReader::GetChunkDataAccessor<std::string>(
!storage::MmapManager::GetInstance()
.GetMmapConfig()
.growing_enable_mmap) {
auto chunk_info =
segment_->chunk_data<std::string>(field_id, current_chunk_id);
auto pw = segment_->chunk_data<std::string>(field_id, current_chunk_id);
auto chunk_info = pw.get();
auto chunk_data = chunk_info.data();
auto chunk_valid_data = chunk_info.valid_data();
auto current_chunk_size =
segment_->chunk_size(field_id, current_chunk_id);
return [=,
return [pw = std::move(pw),
this,
field_id,
chunk_data,
chunk_valid_data,
current_chunk_size,
// pw = std::move(pw),
&current_chunk_id,
&current_chunk_pos]() mutable -> const data_access_type {
if (current_chunk_pos >= current_chunk_size) {
current_chunk_id++;
current_chunk_pos = 0;
chunk_data =
segment_
->chunk_data<std::string>(field_id, current_chunk_id)
.data();
chunk_valid_data =
segment_
->chunk_data<std::string>(field_id, current_chunk_id)
.valid_data();
pw = segment_->chunk_data<std::string>(field_id,
current_chunk_id);
chunk_data = pw.get().data();
chunk_valid_data = pw.get().valid_data();
current_chunk_size =
segment_->chunk_size(field_id, current_chunk_id);
}
@ -122,24 +128,24 @@ SegmentChunkReader::GetChunkDataAccessor<std::string>(
return chunk_data[current_chunk_pos++];
};
} else {
auto chunk_info =
auto pw =
segment_->chunk_view<std::string_view>(field_id, current_chunk_id);
auto current_chunk_size =
segment_->chunk_size(field_id, current_chunk_id);
return [=,
pw = std::move(pw),
&current_chunk_id,
&current_chunk_pos]() mutable -> const data_access_type {
if (current_chunk_pos >= current_chunk_size) {
current_chunk_id++;
current_chunk_pos = 0;
chunk_info = segment_->chunk_view<std::string_view>(
field_id, current_chunk_id);
pw = segment_->chunk_view<std::string_view>(field_id,
current_chunk_id);
current_chunk_size =
segment_->chunk_size(field_id, current_chunk_id);
}
auto& chunk_data = chunk_info.first;
auto& chunk_valid_data = chunk_info.second;
auto& chunk_data = pw.get().first;
auto& chunk_valid_data = pw.get().second;
if (current_chunk_pos < chunk_valid_data.size() &&
!chunk_valid_data[current_chunk_pos]) {
current_chunk_pos++;
@ -204,10 +210,11 @@ SegmentChunkReader::GetChunkDataAccessor(FieldId field_id,
};
}
}
auto chunk_data = segment_->chunk_data<T>(field_id, chunk_id).data();
auto chunk_valid_data =
segment_->chunk_data<T>(field_id, chunk_id).valid_data();
return [chunk_data, chunk_valid_data](int i) -> const data_access_type {
auto pw = segment_->chunk_data<T>(field_id, chunk_id);
return [pw = std::move(pw)](int i) mutable -> const data_access_type {
auto chunk_info = pw.get();
auto chunk_data = chunk_info.data();
auto chunk_valid_data = chunk_info.valid_data();
if (chunk_valid_data && !chunk_valid_data[i]) {
return std::nullopt;
}
@ -237,22 +244,20 @@ SegmentChunkReader::GetChunkDataAccessor<std::string>(FieldId field_id,
!storage::MmapManager::GetInstance()
.GetMmapConfig()
.growing_enable_mmap) {
auto chunk_data =
segment_->chunk_data<std::string>(field_id, chunk_id).data();
auto chunk_valid_data =
segment_->chunk_data<std::string>(field_id, chunk_id).valid_data();
return [chunk_data, chunk_valid_data](int i) -> const data_access_type {
auto pw = segment_->chunk_data<std::string>(field_id, chunk_id);
return [pw = std::move(pw)](int i) mutable -> const data_access_type {
auto chunk_data = pw.get().data();
auto chunk_valid_data = pw.get().valid_data();
if (chunk_valid_data && !chunk_valid_data[i]) {
return std::nullopt;
}
return chunk_data[i];
};
} else {
auto chunk_info =
segment_->chunk_view<std::string_view>(field_id, chunk_id);
return [chunk_data = std::move(chunk_info.first),
chunk_valid_data = std::move(chunk_info.second)](
int i) -> const data_access_type {
auto pw = segment_->chunk_view<std::string_view>(field_id, chunk_id);
return [pw = std::move(pw)](int i) mutable -> const data_access_type {
auto chunk_data = pw.get().first;
auto chunk_valid_data = pw.get().second;
if (i < chunk_valid_data.size() && !chunk_valid_data[i]) {
return std::nullopt;
}

View File

@ -22,6 +22,7 @@
#include <unordered_map>
#include <variant>
#include "cachinglayer/CacheSlot.h"
#include "common/Consts.h"
#include "common/EasyAssert.h"
#include "common/FieldData.h"
@ -46,6 +47,8 @@
namespace milvus::segcore {
using namespace milvus::cachinglayer;
int64_t
SegmentGrowingImpl::PreInsert(int64_t size) {
auto reserved_begin = insert_record_.reserved.fetch_add(size);
@ -509,8 +512,7 @@ SegmentGrowingImpl::load_column_group_data_internal(
}
SegcoreError
SegmentGrowingImpl::Delete(int64_t reserved_begin,
int64_t size,
SegmentGrowingImpl::Delete(int64_t size,
const IdArray* ids,
const Timestamp* timestamps_raw) {
auto field_id = schema_->get_primary_field_id().value_or(FieldId(-1));
@ -573,12 +575,13 @@ SegmentGrowingImpl::LoadDeletedRecord(const LoadDeletedRecordInfo& info) {
deleted_record_.LoadPush(pks, timestamps);
}
SpanBase
PinWrapper<SpanBase>
SegmentGrowingImpl::chunk_data_impl(FieldId field_id, int64_t chunk_id) const {
return get_insert_record().get_span_base(field_id, chunk_id);
return PinWrapper<SpanBase>(
get_insert_record().get_span_base(field_id, chunk_id));
}
std::pair<std::vector<std::string_view>, FixedVector<bool>>
PinWrapper<std::pair<std::vector<std::string_view>, FixedVector<bool>>>
SegmentGrowingImpl::chunk_string_view_impl(
FieldId field_id,
int64_t chunk_id,
@ -588,7 +591,7 @@ SegmentGrowingImpl::chunk_string_view_impl(
"chunk string view impl not implement for growing segment");
}
std::pair<std::vector<ArrayView>, FixedVector<bool>>
PinWrapper<std::pair<std::vector<ArrayView>, FixedVector<bool>>>
SegmentGrowingImpl::chunk_array_view_impl(
FieldId field_id,
int64_t chunk_id,
@ -598,7 +601,7 @@ SegmentGrowingImpl::chunk_array_view_impl(
"chunk array view impl not implement for growing segment");
}
std::pair<std::vector<std::string_view>, FixedVector<bool>>
PinWrapper<std::pair<std::vector<std::string_view>, FixedVector<bool>>>
SegmentGrowingImpl::chunk_view_by_offsets(
FieldId field_id,
int64_t chunk_id,
@ -1154,7 +1157,7 @@ SegmentGrowingImpl::CreateJSONIndex(FieldId field_id) {
json_indexes_[field_id] = std::move(index);
}
std::pair<std::string_view, bool>
std::pair<milvus::Json, bool>
SegmentGrowingImpl::GetJsonData(FieldId field_id, size_t offset) const {
auto vec_ptr = dynamic_cast<const ConcurrentVector<Json>*>(
insert_record_.get_data_base(field_id));
@ -1162,10 +1165,9 @@ SegmentGrowingImpl::GetJsonData(FieldId field_id, size_t offset) const {
auto& field_meta = schema_->operator[](field_id);
if (field_meta.is_nullable()) {
auto valid_data_ptr = insert_record_.get_valid_data(field_id);
return std::make_pair(std::string_view(src[offset]),
valid_data_ptr->is_valid(offset));
return std::make_pair(src[offset], valid_data_ptr->is_valid(offset));
}
return std::make_pair(std::string_view(src[offset]), true);
return std::make_pair(src[offset], true);
}
void

View File

@ -21,6 +21,7 @@
#include <vector>
#include <utility>
#include "cachinglayer/CacheSlot.h"
#include "AckResponder.h"
#include "ConcurrentVector.h"
#include "DeletedRecord.h"
@ -35,6 +36,8 @@
namespace milvus::segcore {
using namespace milvus::cachinglayer;
class SegmentGrowingImpl : public SegmentGrowing {
public:
int64_t
@ -54,8 +57,7 @@ class SegmentGrowingImpl : public SegmentGrowing {
// TODO: add id into delete log, possibly bitmap
SegcoreError
Delete(int64_t reserved_offset,
int64_t size,
Delete(int64_t size,
const IdArray* pks,
const Timestamp* timestamps) override;
@ -107,7 +109,7 @@ class SegmentGrowingImpl : public SegmentGrowing {
finish_load() override;
public:
const InsertRecord<>&
const InsertRecord<false>&
get_insert_record() const {
return insert_record_;
}
@ -248,7 +250,7 @@ class SegmentGrowingImpl : public SegmentGrowing {
int64_t count,
const std::vector<std::string>& dynamic_field_names) const override;
virtual std::pair<std::string_view, bool>
virtual std::pair<milvus::Json, bool>
GetJsonData(FieldId field_id, size_t offset) const override;
public:
@ -374,11 +376,6 @@ class SegmentGrowingImpl : public SegmentGrowing {
return insert_record_.search_pk(pk, timestamp);
}
std::vector<SegOffset>
search_pk(const PkType& pk, int64_t insert_barrier) const override {
return insert_record_.search_pk(pk, insert_barrier);
}
bool
is_field_exist(FieldId field_id) const override {
return schema_->get_fields().find(field_id) !=
@ -389,36 +386,26 @@ class SegmentGrowingImpl : public SegmentGrowing {
int64_t
num_chunk(FieldId field_id) const override;
SpanBase
PinWrapper<SpanBase>
chunk_data_impl(FieldId field_id, int64_t chunk_id) const override;
std::pair<std::vector<std::string_view>, FixedVector<bool>>
PinWrapper<std::pair<std::vector<std::string_view>, FixedVector<bool>>>
chunk_string_view_impl(
FieldId field_id,
int64_t chunk_id,
std::optional<std::pair<int64_t, int64_t>> offset_len) const override;
std::pair<std::vector<ArrayView>, FixedVector<bool>>
PinWrapper<std::pair<std::vector<ArrayView>, FixedVector<bool>>>
chunk_array_view_impl(
FieldId field_id,
int64_t chunk_id,
std::optional<std::pair<int64_t, int64_t>> offset_len) const override;
std::pair<std::vector<std::string_view>, FixedVector<bool>>
PinWrapper<std::pair<std::vector<std::string_view>, FixedVector<bool>>>
chunk_view_by_offsets(FieldId field_id,
int64_t chunk_id,
const FixedVector<int32_t>& offsets) const override;
std::pair<BufferView, FixedVector<bool>>
get_chunk_buffer(FieldId field_id,
int64_t chunk_id,
int64_t start_offset,
int64_t length) const override {
PanicInfo(
ErrorCode::Unsupported,
"get_chunk_buffer interface not supported for growing segment");
}
void
check_search(const query::Plan* plan) const override {
Assert(plan);
@ -478,10 +465,6 @@ class SegmentGrowingImpl : public SegmentGrowing {
SegmentStats stats_{};
};
const static IndexMetaPtr empty_index_meta =
std::make_shared<CollectionIndexMeta>(1024,
std::map<FieldId, FieldIndexMeta>());
inline SegmentGrowingPtr
CreateGrowingSegment(
SchemaPtr schema,

View File

@ -18,6 +18,7 @@
#include "common/SystemProperty.h"
#include "common/Tracer.h"
#include "common/Types.h"
#include "monitor/prometheus_client.h"
#include "query/ExecPlanNodeVisitor.h"
namespace milvus::segcore {

View File

@ -20,8 +20,10 @@
#include <index/ScalarIndex.h>
#include "FieldIndexing.h"
#include "cachinglayer/CacheSlot.h"
#include "common/Common.h"
#include "common/EasyAssert.h"
#include "common/Json.h"
#include "common/Schema.h"
#include "common/Span.h"
#include "common/SystemProperty.h"
@ -36,12 +38,13 @@
#include "pb/segcore.pb.h"
#include "index/IndexInfo.h"
#include "index/SkipIndex.h"
#include "mmap/Column.h"
#include "index/TextMatchIndex.h"
#include "index/JsonKeyStatsInvertedIndex.h"
namespace milvus::segcore {
using namespace milvus::cachinglayer;
struct SegmentStats {
// we stat the memory size used by the segment,
// including the insert data and delete data.
@ -105,14 +108,8 @@ class SegmentInterface {
int64_t num_rows,
int64_t field_size) = 0;
// virtual int64_t
// PreDelete(int64_t size) = 0;
virtual SegcoreError
Delete(int64_t reserved_offset,
int64_t size,
const IdArray* pks,
const Timestamp* timestamps) = 0;
Delete(int64_t size, const IdArray* pks, const Timestamp* timestamps) = 0;
virtual void
LoadDeletedRecord(const LoadDeletedRecordInfo& info) = 0;
@ -145,7 +142,7 @@ class SegmentInterface {
virtual index::JsonKeyStatsInvertedIndex*
GetJsonKeyIndex(FieldId field_id) const = 0;
virtual std::pair<std::string_view, bool>
virtual std::pair<milvus::Json, bool>
GetJsonData(FieldId field_id, size_t offset) const = 0;
virtual void
@ -166,40 +163,40 @@ class SegmentInterface {
class SegmentInternalInterface : public SegmentInterface {
public:
template <typename T>
Span<T>
PinWrapper<Span<T>>
chunk_data(FieldId field_id, int64_t chunk_id) const {
return static_cast<Span<T>>(chunk_data_impl(field_id, chunk_id));
return chunk_data_impl(field_id, chunk_id)
.transform<Span<T>>([](SpanBase&& span_base) {
return static_cast<Span<T>>(span_base);
});
}
template <typename ViewType>
std::pair<std::vector<ViewType>, FixedVector<bool>>
PinWrapper<std::pair<std::vector<ViewType>, FixedVector<bool>>>
chunk_view(FieldId field_id,
int64_t chunk_id,
std::optional<std::pair<int64_t, int64_t>> offset_len =
std::nullopt) const {
if constexpr (std::is_same_v<ViewType, std::string_view>) {
auto [string_views, valid_data] =
chunk_string_view_impl(field_id, chunk_id, offset_len);
return std::make_pair(std::move(string_views),
std::move(valid_data));
return chunk_string_view_impl(field_id, chunk_id, offset_len);
} else if constexpr (std::is_same_v<ViewType, ArrayView>) {
auto [array_views, valid_data] =
chunk_array_view_impl(field_id, chunk_id, offset_len);
return std::make_pair(array_views, valid_data);
return chunk_array_view_impl(field_id, chunk_id, offset_len);
} else if constexpr (std::is_same_v<ViewType, Json>) {
auto [string_views, valid_data] =
chunk_string_view_impl(field_id, chunk_id, offset_len);
auto pw = chunk_string_view_impl(field_id, chunk_id, offset_len);
auto [string_views, valid_data] = pw.get();
std::vector<Json> res;
res.reserve(string_views.size());
for (const auto& str_view : string_views) {
res.emplace_back(str_view);
}
return {std::move(res), std::move(valid_data)};
return PinWrapper<
std::pair<std::vector<ViewType>, FixedVector<bool>>>(
{std::move(res), std::move(valid_data)});
}
}
template <typename ViewType>
std::pair<std::vector<ViewType>, FixedVector<bool>>
PinWrapper<std::pair<std::vector<ViewType>, FixedVector<bool>>>
get_batch_views(FieldId field_id,
int64_t chunk_id,
int64_t start_offset,
@ -213,7 +210,7 @@ class SegmentInternalInterface : public SegmentInterface {
}
template <typename ViewType>
std::pair<std::vector<ViewType>, FixedVector<bool>>
PinWrapper<std::pair<std::vector<ViewType>, FixedVector<bool>>>
get_views_by_offsets(FieldId field_id,
int64_t chunk_id,
const FixedVector<int32_t>& offsets) const {
@ -221,16 +218,20 @@ class SegmentInternalInterface : public SegmentInterface {
PanicInfo(ErrorCode::Unsupported,
"get chunk views not supported for growing segment");
}
auto chunk_view = chunk_view_by_offsets(field_id, chunk_id, offsets);
auto pw = chunk_view_by_offsets(field_id, chunk_id, offsets);
if constexpr (std::is_same_v<ViewType, std::string_view>) {
return chunk_view;
return pw;
} else {
static_assert(std::is_same_v<ViewType, milvus::Json>,
"only Json is supported for get_views_by_offsets");
std::vector<ViewType> res;
res.reserve(chunk_view.first.size());
for (const auto& view : chunk_view.first) {
res.reserve(pw.get().first.size());
for (const auto& view : pw.get().first) {
res.emplace_back(view);
}
return {res, chunk_view.second};
return PinWrapper<
std::pair<std::vector<ViewType>, FixedVector<bool>>>(
{std::move(res), pw.get().second});
}
}
@ -461,30 +462,25 @@ class SegmentInternalInterface : public SegmentInterface {
protected:
// todo: use an Unified struct for all type in growing/seal segment to store data and valid_data.
// internal API: return chunk_data in span
virtual SpanBase
virtual PinWrapper<SpanBase>
chunk_data_impl(FieldId field_id, int64_t chunk_id) const = 0;
// internal API: return chunk string views in vector
virtual std::pair<std::vector<std::string_view>, FixedVector<bool>>
virtual PinWrapper<
std::pair<std::vector<std::string_view>, FixedVector<bool>>>
chunk_string_view_impl(FieldId field_id,
int64_t chunk_id,
std::optional<std::pair<int64_t, int64_t>>
offset_len = std::nullopt) const = 0;
virtual std::pair<std::vector<ArrayView>, FixedVector<bool>>
virtual PinWrapper<std::pair<std::vector<ArrayView>, FixedVector<bool>>>
chunk_array_view_impl(FieldId field_id,
int64_t chunk_id,
std::optional<std::pair<int64_t, int64_t>>
offset_len = std::nullopt) const = 0;
// internal API: return buffer reference to field chunk data located from start_offset
virtual std::pair<BufferView, FixedVector<bool>>
get_chunk_buffer(FieldId field_id,
int64_t chunk_id,
int64_t start_offset,
int64_t length) const = 0;
virtual std::pair<std::vector<std::string_view>, FixedVector<bool>>
virtual PinWrapper<
std::pair<std::vector<std::string_view>, FixedVector<bool>>>
chunk_view_by_offsets(FieldId field_id,
int64_t chunk_id,
const FixedVector<int32_t>& offsets) const = 0;
@ -531,9 +527,6 @@ class SegmentInternalInterface : public SegmentInterface {
virtual std::vector<SegOffset>
search_pk(const PkType& pk, Timestamp timestamp) const = 0;
virtual std::vector<SegOffset>
search_pk(const PkType& pk, int64_t insert_barrier) const = 0;
protected:
mutable std::shared_mutex mutex_;
// fieldID -> std::pair<num_rows, avg_size>

View File

@ -13,13 +13,13 @@
#include <memory>
#include <utility>
#include <tuple>
#include "common/LoadInfo.h"
#include "common/Types.h"
#include "index/JsonInvertedIndex.h"
#include "pb/segcore.pb.h"
#include "segcore/InsertRecord.h"
#include "segcore/SegmentInterface.h"
#include "cachinglayer/CacheSlot.h"
#include "segcore/Types.h"
namespace milvus::segcore {
@ -35,15 +35,9 @@ class SegmentSealed : public SegmentInternalInterface {
virtual void
DropFieldData(const FieldId field_id) = 0;
virtual void
LoadFieldData(FieldId field_id, FieldDataInfo& data) = 0;
virtual void
MapFieldData(const FieldId field_id, FieldDataInfo& data) = 0;
virtual void
AddFieldDataInfoForSealed(const LoadFieldDataInfo& field_data_info) = 0;
virtual void
WarmupChunkCache(const FieldId field_id, bool mmap_enabled) = 0;
virtual void
RemoveFieldFile(const FieldId field_id) = 0;
virtual void
ClearData() = 0;
@ -54,9 +48,6 @@ class SegmentSealed : public SegmentInternalInterface {
LoadTextIndex(FieldId field_id,
std::unique_ptr<index::TextMatchIndex> index) = 0;
virtual InsertRecord<true>&
get_insert_record() = 0;
virtual index::IndexBase*
GetJsonIndex(FieldId field_id, std::string path) const override {
JSONIndexKey key;
@ -74,12 +65,6 @@ class SegmentSealed : public SegmentInternalInterface {
FieldId field_id,
std::unique_ptr<index::JsonKeyStatsInvertedIndex> index) = 0;
virtual index::JsonKeyStatsInvertedIndex*
GetJsonKeyIndex(FieldId field_id) const = 0;
virtual std::pair<std::string_view, bool>
GetJsonData(FieldId field_id, size_t offset) const = 0;
SegmentType
type() const override {
return SegmentType::Sealed;
@ -119,6 +104,9 @@ class SegmentSealed : public SegmentInternalInterface {
index->second->JsonCastType() == DataType::DOUBLE);
}
virtual std::shared_ptr<CacheSlot<InsertRecord<true>>>
get_insert_record_slot() const = 0;
protected:
struct JSONIndexKey {
FieldId field_id;

View File

@ -11,23 +11,15 @@
#pragma once
#include <unordered_map>
#include <exception>
#include <memory>
#include <stdexcept>
#include <cstdlib>
#include <string>
#include <utility>
#include <vector>
#include "common/FieldData.h"
#include "common/QueryResult.h"
// #include "common/Schema.h"
#include "common/Types.h"
#include "index/Index.h"
#include "log/Log.h"
#include "segcore/DeletedRecord.h"
#include "segcore/InsertRecord.h"
#include "segcore/ConcurrentVector.h"
namespace milvus::segcore {

View File

@ -89,12 +89,6 @@ AppendMMapDirPath(CLoadFieldDataInfo c_load_field_data_info,
load_field_data_info->mmap_dir_path = std::string(c_dir_path);
}
void
SetUri(CLoadFieldDataInfo c_load_field_data_info, const char* uri) {
auto load_field_data_info = (LoadFieldDataInfo*)c_load_field_data_info;
load_field_data_info->url = std::string(uri);
}
void
SetStorageVersion(CLoadFieldDataInfo c_load_field_data_info,
int64_t storage_version) {

View File

@ -46,9 +46,6 @@ void
AppendMMapDirPath(CLoadFieldDataInfo c_load_field_data_info,
const char* dir_path);
void
SetUri(CLoadFieldDataInfo c_load_field_data_info, const char* uri);
void
SetStorageVersion(CLoadFieldDataInfo c_load_field_data_info,
int64_t storage_version);

View File

@ -15,9 +15,10 @@
#include <cstdint>
#include <vector>
#include "common/EasyAssert.h"
#include "monitor/prometheus_client.h"
#include "segcore/SegmentInterface.h"
#include "segcore/Utils.h"
#include "common/EasyAssert.h"
#include "segcore/pkVisitor.h"
#include "segcore/ReduceUtils.h"

View File

@ -15,6 +15,7 @@
#include "log/Log.h"
#include "segcore/SegcoreConfig.h"
#include "segcore/segcore_init_c.h"
#include "cachinglayer/Manager.h"
namespace milvus::segcore {
@ -115,4 +116,12 @@ SetThreadName(const char* name) {
#endif
}
extern "C" void
ConfigureTieredStorage(const bool enabled_globally,
const int64_t memory_limit_bytes,
const int64_t disk_limit_bytes) {
milvus::cachinglayer::Manager::ConfigureTieredStorage(
enabled_globally, memory_limit_bytes, disk_limit_bytes);
}
} // namespace milvus::segcore

View File

@ -62,6 +62,11 @@ GetMinimalIndexVersion();
void
SetThreadName(const char*);
void
ConfigureTieredStorage(const bool enabled_globally,
const int64_t memory_limit_bytes,
const int64_t disk_limit_bytes);
#ifdef __cplusplus
}
#endif

View File

@ -36,6 +36,7 @@
#include "segcore/SegmentSealed.h"
#include "segcore/ChunkedSegmentSealedImpl.h"
#include "mmap/Types.h"
#include "storage/RemoteChunkManagerSingleton.h"
////////////////////////////// common interfaces //////////////////////////////
CStatus
@ -319,7 +320,6 @@ PreInsert(CSegmentInterface c_segment, int64_t size, int64_t* offset) {
CStatus
Delete(CSegmentInterface c_segment,
int64_t reserved_offset, // deprecated
int64_t size,
const uint8_t* ids,
const uint64_t ids_size,
@ -329,8 +329,7 @@ Delete(CSegmentInterface c_segment,
auto suc = pks->ParseFromArray(ids, ids_size);
AssertInfo(suc, "failed to parse pks from ids");
try {
auto res =
segment->Delete(reserved_offset, size, pks.get(), timestamps);
auto res = segment->Delete(size, pks.get(), timestamps);
return milvus::SuccessCStatus();
} catch (std::exception& e) {
return milvus::FailureCStatus(&e);
@ -353,49 +352,6 @@ LoadFieldData(CSegmentInterface c_segment,
}
}
// just for test
CStatus
LoadFieldRawData(CSegmentInterface c_segment,
int64_t field_id,
const void* data,
int64_t row_count) {
try {
auto segment_interface =
reinterpret_cast<milvus::segcore::SegmentInterface*>(c_segment);
auto segment =
dynamic_cast<milvus::segcore::SegmentSealed*>(segment_interface);
AssertInfo(segment != nullptr, "segment conversion failed");
milvus::DataType data_type;
int64_t dim = 1;
if (milvus::SystemProperty::Instance().IsSystem(
milvus::FieldId(field_id))) {
data_type = milvus::DataType::INT64;
} else {
auto field_meta = segment->get_schema()[milvus::FieldId(field_id)];
data_type = field_meta.get_data_type();
if (milvus::IsVectorDataType(data_type) &&
!milvus::IsSparseFloatVectorDataType(data_type)) {
dim = field_meta.get_dim();
}
}
auto field_data =
milvus::storage::CreateFieldData(data_type, false, dim);
field_data->FillFieldData(data, row_count);
auto arrow_data_wrapper =
milvus::storage::ConvertFieldDataToArrowDataWrapper(field_data);
auto field_data_info = milvus::FieldDataInfo{
field_id,
static_cast<size_t>(row_count),
std::vector<std::shared_ptr<milvus::ArrowDataWrapper>>{
arrow_data_wrapper}};
segment->LoadFieldData(milvus::FieldId(field_id), field_data_info);
return milvus::SuccessCStatus();
} catch (std::exception& e) {
return milvus::FailureCStatus(&e);
}
}
CStatus
LoadDeletedRecord(CSegmentInterface c_segment,
CLoadDeletedRecordInfo deleted_record_info) {
@ -608,17 +564,7 @@ CStatus
WarmupChunkCache(CSegmentInterface c_segment,
int64_t field_id,
bool mmap_enabled) {
try {
auto segment_interface =
reinterpret_cast<milvus::segcore::SegmentInterface*>(c_segment);
auto segment =
dynamic_cast<milvus::segcore::SegmentSealed*>(segment_interface);
AssertInfo(segment != nullptr, "segment conversion failed");
segment->WarmupChunkCache(milvus::FieldId(field_id), mmap_enabled);
return milvus::SuccessCStatus();
} catch (std::exception& e) {
return milvus::FailureCStatus(milvus::UnexpectedError, e.what());
}
return milvus::SuccessCStatus();
}
void

View File

@ -105,12 +105,6 @@ CStatus
LoadFieldData(CSegmentInterface c_segment,
CLoadFieldDataInfo load_field_data_info);
CStatus
LoadFieldRawData(CSegmentInterface c_segment,
int64_t field_id,
const void* data,
int64_t row_count);
CStatus
LoadDeletedRecord(CSegmentInterface c_segment,
CLoadDeletedRecordInfo deleted_record_info);
@ -136,6 +130,8 @@ UpdateFieldRawDataSize(CSegmentInterface c_segment,
int64_t num_rows,
int64_t field_data_size);
// This function is currently used only in test.
// Current implement supports only dropping of non-system fields.
CStatus
DropFieldData(CSegmentInterface c_segment, int64_t field_id);
@ -160,7 +156,6 @@ ExistPk(CSegmentInterface c_segment,
CStatus
Delete(CSegmentInterface c_segment,
int64_t reserved_offset,
int64_t size,
const uint8_t* ids,
const uint64_t ids_size,

View File

@ -0,0 +1,166 @@
// Copyright (C) 2019-2025 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#include "segcore/storagev1translator/ChunkTranslator.h"
#include <filesystem>
#include <memory>
#include <string>
#include <vector>
#include "cachinglayer/Utils.h"
#include "common/ChunkWriter.h"
#include "common/EasyAssert.h"
#include "common/Types.h"
#include "common/SystemProperty.h"
#include "segcore/Utils.h"
#include "storage/ThreadPools.h"
#include "mmap/Types.h"
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)
: segment_id_(segment_id),
key_(fmt::format("seg_{}_f_{}", segment_id, field_data_info.field_id)),
use_mmap_(use_mmap),
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;
meta_.num_rows_until_chunk_.push_back(0);
if (!use_mmap_) {
std::shared_ptr<milvus::ArrowDataWrapper> r;
while (field_data_info.arrow_reader_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++;
}
} else {
auto filepath = std::filesystem::path(field_data_info.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);
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)) {
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);
}
}
}
size_t
ChunkTranslator::num_cells() const {
return chunks_.size();
}
milvus::cachinglayer::cid_t
ChunkTranslator::cell_id_of(milvus::cachinglayer::uid_t uid) const {
return uid;
}
milvus::cachinglayer::ResourceUsage
ChunkTranslator::estimated_byte_size_of_cell(
milvus::cachinglayer::cid_t cid) const {
return {0, 0};
}
const std::string&
ChunkTranslator::key() const {
return key_;
}
std::vector<
std::pair<milvus::cachinglayer::cid_t, std::unique_ptr<milvus::Chunk>>>
ChunkTranslator::get_cells(
const std::vector<milvus::cachinglayer::cid_t>& cids) {
std::vector<
std::pair<milvus::cachinglayer::cid_t, std::unique_ptr<milvus::Chunk>>>
cells;
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;
}
return cells;
}
} // namespace milvus::segcore::storagev1translator

View File

@ -0,0 +1,71 @@
// Copyright (C) 2019-2025 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#pragma once
#include <string>
#include <vector>
#include "cachinglayer/Translator.h"
#include "cachinglayer/Utils.h"
#include "common/Chunk.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) {
}
};
// 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.
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;
size_t
num_cells() const override;
milvus::cachinglayer::cid_t
cell_id_of(milvus::cachinglayer::uid_t uid) const override;
milvus::cachinglayer::ResourceUsage
estimated_byte_size_of_cell(milvus::cachinglayer::cid_t cid) const override;
const std::string&
key() const override;
std::vector<
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:
int64_t segment_id_;
std::string key_;
bool use_mmap_;
std::vector<milvus::Chunk*> chunks_;
CTMeta meta_;
};
} // namespace milvus::segcore::storagev1translator

View File

@ -0,0 +1,96 @@
// Copyright (C) 2019-2025 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#include "segcore/storagev1translator/DefaultValueChunkTranslator.h"
#include "common/ChunkWriter.h"
#include "storage/Util.h"
namespace milvus::segcore::storagev1translator {
DefaultValueChunkTranslator::DefaultValueChunkTranslator(
int64_t segment_id,
FieldMeta field_meta,
FieldDataInfo field_data_info,
bool use_mmap)
: segment_id_(segment_id),
key_(fmt::format("seg_{}_f_{}", segment_id, field_data_info.field_id)),
use_mmap_(use_mmap),
field_meta_(field_meta),
meta_(use_mmap ? milvus::cachinglayer::StorageType::DISK
: milvus::cachinglayer::StorageType::MEMORY) {
meta_.num_rows_until_chunk_.push_back(0);
meta_.num_rows_until_chunk_.push_back(field_data_info.row_count);
}
DefaultValueChunkTranslator::~DefaultValueChunkTranslator() {
}
size_t
DefaultValueChunkTranslator::num_cells() const {
return 1;
}
milvus::cachinglayer::cid_t
DefaultValueChunkTranslator::cell_id_of(milvus::cachinglayer::uid_t uid) const {
return 0;
}
milvus::cachinglayer::ResourceUsage
DefaultValueChunkTranslator::estimated_byte_size_of_cell(
milvus::cachinglayer::cid_t cid) const {
return milvus::cachinglayer::ResourceUsage{0, 0};
}
const std::string&
DefaultValueChunkTranslator::key() const {
return key_;
}
std::vector<
std::pair<milvus::cachinglayer::cid_t, std::unique_ptr<milvus::Chunk>>>
DefaultValueChunkTranslator::get_cells(
const std::vector<milvus::cachinglayer::cid_t>& cids) {
AssertInfo(cids.size() == 1 && cids[0] == 0,
"DefaultValueChunkTranslator only supports one cell");
auto num_rows = meta_.num_rows_until_chunk_[1];
auto builder =
milvus::storage::CreateArrowBuilder(field_meta_.get_data_type());
arrow::Status ast;
if (field_meta_.default_value().has_value()) {
builder->Reserve(num_rows);
auto scalar = storage::CreateArrowScalarFromDefaultValue(field_meta_);
ast = builder->AppendScalar(*scalar, num_rows);
} else {
ast = builder->AppendNulls(num_rows);
}
AssertInfo(ast.ok(),
"append null/default values to arrow builder failed: {}",
ast.ToString());
arrow::ArrayVector array_vec;
array_vec.emplace_back(builder->Finish().ValueOrDie());
auto chunk = create_chunk(
field_meta_,
IsVectorDataType(field_meta_.get_data_type()) &&
!IsSparseFloatVectorDataType(field_meta_.get_data_type())
? field_meta_.get_dim()
: 1,
array_vec);
std::vector<
std::pair<milvus::cachinglayer::cid_t, std::unique_ptr<milvus::Chunk>>>
res;
res.reserve(1);
res.emplace_back(0, std::move(chunk));
return res;
}
} // namespace milvus::segcore::storagev1translator

View File

@ -0,0 +1,62 @@
// Copyright (C) 2019-2025 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#pragma once
#include <string>
#include <vector>
#include "cachinglayer/Translator.h"
#include "cachinglayer/Utils.h"
#include "common/Chunk.h"
#include "common/FieldMeta.h"
#include "mmap/Types.h"
#include "segcore/storagev1translator/ChunkTranslator.h"
namespace milvus::segcore::storagev1translator {
// This is to support add field.
class DefaultValueChunkTranslator
: public milvus::cachinglayer::Translator<milvus::Chunk> {
public:
DefaultValueChunkTranslator(int64_t segment_id,
FieldMeta field_meta,
FieldDataInfo field_data_info,
bool use_mmap);
~DefaultValueChunkTranslator() override;
size_t
num_cells() const override;
milvus::cachinglayer::cid_t
cell_id_of(milvus::cachinglayer::uid_t uid) const override;
milvus::cachinglayer::ResourceUsage
estimated_byte_size_of_cell(milvus::cachinglayer::cid_t cid) const override;
const std::string&
key() const override;
std::vector<
std::pair<milvus::cachinglayer::cid_t, std::unique_ptr<milvus::Chunk>>>
get_cells(const std::vector<milvus::cachinglayer::cid_t>& cids) override;
milvus::cachinglayer::Meta*
meta() override {
return &meta_;
}
private:
int64_t segment_id_;
std::string key_;
bool use_mmap_;
CTMeta meta_;
milvus::FieldMeta field_meta_;
};
} // namespace milvus::segcore::storagev1translator

View File

@ -0,0 +1,139 @@
#include "segcore/storagev1translator/InsertRecordTranslator.h"
#include <memory>
#include <vector>
#include <string>
#include "fmt/core.h"
#include "cachinglayer/Utils.h"
#include "common/ChunkWriter.h"
#include "common/Types.h"
#include "common/SystemProperty.h"
#include "segcore/Utils.h"
#include "storage/ThreadPools.h"
namespace milvus::segcore::storagev1translator {
InsertRecordTranslator::InsertRecordTranslator(
int64_t segment_id,
DataType data_type,
FieldDataInfo field_data_info,
SchemaPtr schema,
bool is_sorted_by_pk,
std::vector<std::string> insert_files,
ChunkedSegmentSealedImpl* chunked_segment)
: segment_id_(segment_id),
data_type_(data_type),
key_(fmt::format("seg_{}_ir_f_{}", segment_id, field_data_info.field_id)),
field_data_info_(field_data_info),
schema_(schema),
is_sorted_by_pk_(is_sorted_by_pk),
insert_files_(insert_files),
chunked_segment_(chunked_segment),
meta_(milvus::cachinglayer::StorageType::MEMORY) {
}
size_t
InsertRecordTranslator::num_cells() const {
return 1;
}
milvus::cachinglayer::cid_t
InsertRecordTranslator::cell_id_of(milvus::cachinglayer::uid_t uid) const {
return 0;
}
milvus::cachinglayer::ResourceUsage
InsertRecordTranslator::estimated_byte_size_of_cell(
milvus::cachinglayer::cid_t cid) const {
return {0, 0};
}
const std::string&
InsertRecordTranslator::key() const {
return key_;
}
std::vector<std::pair<milvus::cachinglayer::cid_t,
std::unique_ptr<milvus::segcore::InsertRecord<true>>>>
InsertRecordTranslator::get_cells(
const std::vector<milvus::cachinglayer::cid_t>& cids) {
AssertInfo(cids.size() == 1 && cids[0] == 0,
"InsertRecordTranslator only supports single cell");
FieldId fid = FieldId(field_data_info_.field_id);
auto parallel_degree =
static_cast<uint64_t>(DEFAULT_FIELD_MAX_MEMORY_LIMIT / FILE_SLICE_SIZE);
// TODO(tiered storage 4): we should phase out this thread pool and use folly executor.
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 num_rows = field_data_info_.row_count;
AssertInfo(milvus::SystemProperty::Instance().IsSystem(fid),
"system field is not system field");
auto system_field_type =
milvus::SystemProperty::Instance().GetSystemFieldType(fid);
AssertInfo(system_field_type == SystemFieldType::Timestamp,
"system field is not timestamp");
std::vector<Timestamp> timestamps(num_rows);
int64_t offset = 0;
FieldMeta field_meta(
FieldName(""), FieldId(0), DataType::INT64, false, std::nullopt);
std::shared_ptr<milvus::ArrowDataWrapper> r;
while (field_data_info_.arrow_reader_channel->pop(r)) {
arrow::ArrayVector array_vec = read_single_column_batches(r->reader);
auto chunk = create_chunk(field_meta, 1, array_vec);
auto chunk_ptr = static_cast<FixedWidthChunk*>(chunk.get());
std::copy_n(static_cast<const Timestamp*>(chunk_ptr->Span().data()),
chunk_ptr->Span().row_count(),
timestamps.data() + offset);
offset += chunk_ptr->Span().row_count();
}
TimestampIndex index;
auto min_slice_length = num_rows < 4096 ? 1 : 4096;
auto meta =
GenerateFakeSlices(timestamps.data(), num_rows, min_slice_length);
index.set_length_meta(std::move(meta));
// todo ::opt to avoid copy timestamps from field data
index.build_with(timestamps.data(), num_rows);
std::unique_ptr<milvus::segcore::InsertRecord<true>> ir =
std::make_unique<milvus::segcore::InsertRecord<true>>(*schema_,
MAX_ROW_COUNT);
// use special index
AssertInfo(ir->timestamps_.empty(), "already exists");
ir->timestamps_.set_data_raw(0, timestamps.data(), timestamps.size());
ir->timestamp_index_ = std::move(index);
AssertInfo(ir->timestamps_.num_chunk() == 1,
"num chunk not equal to 1 for sealed segment");
chunked_segment_->stats_.mem_size += sizeof(Timestamp) * num_rows;
auto pk_field_id = schema_->get_primary_field_id();
AssertInfo(pk_field_id.has_value(),
"primary key field not found in schema");
auto pk_field_meta = schema_->operator[](pk_field_id.value());
// set pks to offset
if (!is_sorted_by_pk_) {
AssertInfo(ir->empty_pks(), "already exists");
auto it = chunked_segment_->fields_.find(pk_field_id.value());
AssertInfo(it != chunked_segment_->fields_.end(),
"primary key field not found in segment");
ir->insert_pks(pk_field_meta.get_data_type(), it->second.get());
ir->seal_pks();
}
std::vector<std::pair<milvus::cachinglayer::cid_t,
std::unique_ptr<milvus::segcore::InsertRecord<true>>>>
cells;
cells.emplace_back(0, std::move(ir));
return cells;
}
} // namespace milvus::segcore::storagev1translator

View File

@ -0,0 +1,72 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#pragma once
#include <string>
#include <vector>
#include "cachinglayer/Translator.h"
#include "cachinglayer/Utils.h"
#include "common/Schema.h"
#include "segcore/InsertRecord.h"
#include "segcore/ChunkedSegmentSealedImpl.h"
namespace milvus::segcore::storagev1translator {
class InsertRecordTranslator : public milvus::cachinglayer::Translator<
milvus::segcore::InsertRecord<true>> {
public:
InsertRecordTranslator(int64_t segment_id,
DataType data_type,
FieldDataInfo field_data_info,
SchemaPtr schema,
bool is_sorted_by_pk,
std::vector<std::string> insert_files,
ChunkedSegmentSealedImpl* chunked_segment);
size_t
num_cells() const override;
milvus::cachinglayer::cid_t
cell_id_of(milvus::cachinglayer::uid_t uid) const override;
milvus::cachinglayer::ResourceUsage
estimated_byte_size_of_cell(milvus::cachinglayer::cid_t cid) const override;
const std::string&
key() const override;
// each calling of this will trigger a new download.
std::vector<std::pair<milvus::cachinglayer::cid_t,
std::unique_ptr<milvus::segcore::InsertRecord<true>>>>
get_cells(const std::vector<milvus::cachinglayer::cid_t>& cids) override;
// InsertRecord does not have meta.
milvus::cachinglayer::Meta*
meta() override {
return &meta_;
}
private:
std::unique_ptr<milvus::segcore::InsertRecord<true>>
load_column_in_memory() const;
int64_t segment_id_;
std::string key_;
DataType data_type_;
FieldDataInfo field_data_info_;
std::vector<std::string> insert_files_;
mutable size_t estimated_byte_size_of_cell_;
SchemaPtr schema_;
bool is_sorted_by_pk_;
ChunkedSegmentSealedImpl* chunked_segment_;
milvus::cachinglayer::Meta meta_;
};
} // namespace milvus::segcore::storagev1translator

View File

@ -1,202 +0,0 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#include <filesystem>
#include <future>
#include <memory>
#include "ChunkCache.h"
#include "boost/filesystem/operations.hpp"
#include "boost/filesystem/path.hpp"
#include "common/Chunk.h"
#include "common/ChunkWriter.h"
#include "common/FieldMeta.h"
#include "common/Types.h"
#include "log/Log.h"
namespace milvus::storage {
std::shared_ptr<ColumnBase>
ChunkCache::Read(const std::string& filepath,
const FieldMeta& field_meta,
bool mmap_enabled,
bool mmap_rss_not_need) {
// use rlock to get future
{
std::shared_lock lck(mutex_);
auto it = columns_.find(filepath);
if (it != columns_.end()) {
lck.unlock();
auto result = it->second.second.get();
AssertInfo(result, "unexpected null column, file={}", filepath);
return result;
}
}
// lock for mutation
std::unique_lock lck(mutex_);
// double check no-futurn
auto it = columns_.find(filepath);
if (it != columns_.end()) {
lck.unlock();
auto result = it->second.second.get();
AssertInfo(result, "unexpected null column, file={}", filepath);
return result;
}
std::promise<std::shared_ptr<ColumnBase>> p;
std::shared_future<std::shared_ptr<ColumnBase>> f = p.get_future();
columns_.emplace(filepath, std::make_pair(std::move(p), f));
lck.unlock();
// release lock and perform download and decode
// other thread request same path shall get the future.
bool allocate_success = false;
ErrorCode err_code = Success;
std::string err_msg = "";
std::shared_ptr<ChunkedColumnBase> column;
try {
auto field_data =
DownloadAndDecodeRemoteFile(cm_.get(), filepath, false);
std::shared_ptr<Chunk> chunk;
auto dim = IsSparseFloatVectorDataType(field_meta.get_data_type())
? 1
: field_meta.get_dim();
if (mmap_enabled) {
auto path = std::filesystem::path(CachePath(filepath));
auto dir = path.parent_path();
std::filesystem::create_directories(dir);
auto file = File::Open(path.string(), O_CREAT | O_TRUNC | O_RDWR);
arrow::ArrayVector array_vec =
read_single_column_batches(field_data->GetReader()->reader);
chunk = create_chunk(field_meta, dim, file, 0, array_vec);
// unlink
auto ok = unlink(path.c_str());
AssertInfo(ok == 0,
"failed to unlink mmap data file {}, err: {}",
path.c_str(),
strerror(errno));
} else {
arrow::ArrayVector array_vec =
read_single_column_batches(field_data->GetReader()->reader);
chunk = create_chunk(field_meta, dim, array_vec);
}
auto data_type = field_meta.get_data_type();
if (IsSparseFloatVectorDataType(data_type)) {
auto sparse_column =
std::make_shared<ChunkedSparseFloatColumn>(field_meta);
sparse_column->AddChunk(chunk);
column = std::move(sparse_column);
} else if (IsVariableDataType(data_type)) {
AssertInfo(false,
"TODO: unimplemented for variable data type: {}",
data_type);
} else {
std::vector<std::shared_ptr<Chunk>> chunks{chunk};
column = std::make_shared<ChunkedColumn>(field_meta, chunks);
}
if (mmap_enabled && mmap_rss_not_need) {
auto ok = madvise(reinterpret_cast<void*>(
const_cast<char*>(column->MmappedData())),
column->DataByteSize(),
ReadAheadPolicy_Map["dontneed"]);
if (ok != 0) {
LOG_WARN(
"failed to madvise to the data file {}, addr {}, size {}, "
"err: "
"{}",
filepath,
static_cast<const void*>(column->MmappedData()),
column->DataByteSize(),
strerror(errno));
}
}
} catch (const SegcoreError& e) {
err_code = e.get_error_code();
err_msg = fmt::format("failed to read for chunkCache, seg_core_err:{}",
e.what());
}
std::unique_lock mmap_lck(mutex_);
it = columns_.find(filepath);
if (it != columns_.end()) {
// check pair exists then set value
it->second.first.set_value(column);
if (allocate_success) {
AssertInfo(column, "unexpected null column, file={}", filepath);
}
} else {
PanicInfo(UnexpectedError,
"Wrong code, the thread to download for cache should get the "
"target entry");
}
if (err_code != Success) {
columns_.erase(filepath);
throw SegcoreError(err_code, err_msg);
}
return column;
}
void
ChunkCache::Remove(const std::string& filepath) {
std::unique_lock lck(mutex_);
columns_.erase(filepath);
}
void
ChunkCache::Prefetch(const std::string& filepath) {
std::shared_lock lck(mutex_);
auto it = columns_.find(filepath);
if (it == columns_.end()) {
return;
}
auto column = it->second.second.get();
auto ok = madvise(
reinterpret_cast<void*>(const_cast<char*>(column->MmappedData())),
column->DataByteSize(),
read_ahead_policy_);
if (ok != 0) {
LOG_WARN(
"failed to madvise to the data file {}, addr {}, size {}, err: {}",
filepath,
static_cast<const void*>(column->MmappedData()),
column->DataByteSize(),
strerror(errno));
}
}
// TODO(sunby): use mmap chunk manager to create chunk
std::string
ChunkCache::CachePath(const std::string& filepath) {
auto path = std::filesystem::path(filepath);
auto prefix = std::filesystem::path(path_prefix_);
// Cache path shall not use absolute filepath direct, it shall always under path_prefix_
if (path.is_absolute()) {
return (prefix /
filepath.substr(path.root_directory().string().length(),
filepath.length()))
.string();
}
return (prefix / filepath).string();
}
} // namespace milvus::storage

View File

@ -1,83 +0,0 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#pragma once
#include <future>
#include <unordered_map>
#include "common/FieldMeta.h"
#include "storage/MmapChunkManager.h"
#include "mmap/ChunkedColumn.h"
namespace milvus::storage {
extern std::map<std::string, int> ReadAheadPolicy_Map;
class ChunkCache {
public:
explicit ChunkCache(std::string& path_prefix,
const std::string& read_ahead_policy,
ChunkManagerPtr cm,
MmapChunkManagerPtr mcm)
: path_prefix_(path_prefix), cm_(cm), mcm_(mcm) {
auto iter = ReadAheadPolicy_Map.find(read_ahead_policy);
AssertInfo(iter != ReadAheadPolicy_Map.end(),
"unrecognized read ahead policy: {}, "
"should be one of `normal, random, sequential, "
"willneed, dontneed`",
read_ahead_policy);
read_ahead_policy_ = iter->second;
LOG_INFO("Init ChunkCache with read_ahead_policy: {}",
read_ahead_policy);
}
~ChunkCache() = default;
public:
std::shared_ptr<ColumnBase>
Read(const std::string& filepath,
const FieldMeta& field_meta,
bool mmap_enabled,
bool mmap_rss_not_need = false);
void
Remove(const std::string& filepath);
void
Prefetch(const std::string& filepath);
private:
std::string
CachePath(const std::string& filepath);
private:
using ColumnTable = std::unordered_map<
std::string,
std::pair<std::promise<std::shared_ptr<ColumnBase>>,
std::shared_future<std::shared_ptr<ColumnBase>>>>;
private:
mutable std::shared_mutex mutex_;
int read_ahead_policy_;
ChunkManagerPtr cm_;
MmapChunkManagerPtr mcm_;
ColumnTable columns_;
std::string path_prefix_;
};
using ChunkCachePtr = std::shared_ptr<milvus::storage::ChunkCache>;
} // namespace milvus::storage

View File

@ -359,7 +359,7 @@ BaseEvent::Serialize() {
int header_size = header.size();
int len = header_size + data_size;
std::vector<uint8_t> res(len);
std::vector<uint8_t> res(len, 0);
int offset = 0;
memcpy(res.data() + offset, header.data(), header_size);
offset += header_size;
@ -384,7 +384,7 @@ DescriptorEvent::Serialize() {
int header_size = header.size();
int len = header_size + data_size + sizeof(MAGIC_NUM);
std::vector<uint8_t> res(len);
std::vector<uint8_t> res(len, 0);
int offset = 0;
memcpy(res.data(), &MAGIC_NUM, sizeof(MAGIC_NUM));
offset += sizeof(MAGIC_NUM);

View File

@ -16,9 +16,8 @@
#pragma once
#include <shared_mutex>
#include "ChunkCache.h"
#include "RemoteChunkManagerSingleton.h"
#include "storage/MmapChunkManager.h"
#include "storage/Types.h"
namespace milvus::storage {
/**
@ -42,9 +41,6 @@ class MmapManager {
return instance;
}
~MmapManager() {
if (cc_ != nullptr) {
cc_ = nullptr;
}
// delete mmap chunk manager at last
if (mcm_ != nullptr) {
mcm_ = nullptr;
@ -62,15 +58,6 @@ class MmapManager {
mmap_config_.disk_limit,
mmap_config_.fix_file_size);
}
if (cc_ == nullptr) {
auto rcm = RemoteChunkManagerSingleton::GetInstance()
.GetRemoteChunkManager();
cc_ = std::make_shared<ChunkCache>(
mmap_config_.mmap_path,
std::move(mmap_config_.cache_read_ahead_policy),
rcm,
mcm_);
}
LOG_INFO("Init MmapConfig with MmapConfig: {}",
mmap_config_.ToString());
init_flag_ = true;
@ -79,12 +66,6 @@ class MmapManager {
}
}
ChunkCachePtr
GetChunkCache() {
AssertInfo(init_flag_ == true, "Mmap manager has not been init.");
return cc_;
}
MmapChunkManagerPtr
GetMmapChunkManager() {
AssertInfo(init_flag_ == true, "Mmap manager has not been init.");
@ -119,7 +100,6 @@ class MmapManager {
mutable std::mutex init_mutex_;
MmapConfig mmap_config_;
MmapChunkManagerPtr mcm_ = nullptr;
ChunkCachePtr cc_ = nullptr;
std::atomic<bool> init_flag_ = false;
};

View File

@ -85,7 +85,6 @@ class ThreadPool {
template <typename F, typename... Args>
auto
// Submit(F&& f, Args&&... args) -> std::future<decltype(f(args...))>;
Submit(F&& f, Args&&... args) -> std::future<decltype(f(args...))> {
std::function<decltype(f(args...))()> func =
std::bind(std::forward<F>(f), std::forward<Args>(args)...);

View File

@ -18,6 +18,7 @@
#include <memory>
#include "arrow/array/builder_binary.h"
#include "arrow/scalar.h"
#include "arrow/type_fwd.h"
#include "fmt/format.h"
#include "log/Log.h"
@ -326,6 +327,42 @@ CreateArrowBuilder(DataType data_type, int dim) {
}
}
std::shared_ptr<arrow::Scalar>
CreateArrowScalarFromDefaultValue(const FieldMeta& field_meta) {
auto default_var = field_meta.default_value();
AssertInfo(default_var.has_value(),
"cannot create Arrow Scalar from empty default value");
auto default_value = default_var.value();
switch (field_meta.get_data_type()) {
case DataType::BOOL:
return std::make_shared<arrow::BooleanScalar>(
default_value.bool_data());
case DataType::INT8:
case DataType::INT16:
case DataType::INT32:
return std::make_shared<arrow::Int32Scalar>(
default_value.int_data());
case DataType::INT64:
return std::make_shared<arrow::Int64Scalar>(
default_value.long_data());
case DataType::FLOAT:
return std::make_shared<arrow::FloatScalar>(
default_value.float_data());
case DataType::DOUBLE:
return std::make_shared<arrow::DoubleScalar>(
default_value.double_data());
case DataType::VARCHAR:
case DataType::STRING:
case DataType::TEXT:
return std::make_shared<arrow::StringScalar>(
default_value.string_data());
default:
PanicInfo(DataTypeInvalid,
"unsupported default value data type {}",
field_meta.get_data_type());
}
}
std::shared_ptr<arrow::Schema>
CreateArrowSchema(DataType data_type, bool nullable) {
switch (static_cast<DataType>(data_type)) {
@ -649,31 +686,6 @@ EncodeAndUploadIndexSlice(ChunkManager* chunk_manager,
return std::make_pair(std::move(object_key), serialized_index_size);
}
std::pair<std::string, size_t>
EncodeAndUploadFieldSlice(ChunkManager* chunk_manager,
void* buf,
int64_t element_count,
FieldDataMeta field_data_meta,
const FieldMeta& field_meta,
std::string object_key) {
// dim should not be used for sparse float vector field
auto dim = IsSparseFloatVectorDataType(field_meta.get_data_type())
? -1
: field_meta.get_dim();
auto field_data =
CreateFieldData(field_meta.get_data_type(), false, dim, 0);
field_data->FillFieldData(buf, element_count);
auto payload_reader = std::make_shared<PayloadReader>(field_data);
auto insertData = std::make_shared<InsertData>(payload_reader);
insertData->SetFieldDataMeta(field_data_meta);
auto serialized_inserted_data = insertData->serialize_to_remote_file();
auto serialized_inserted_data_size = serialized_inserted_data.size();
chunk_manager->Write(object_key,
serialized_inserted_data.data(),
serialized_inserted_data_size);
return std::make_pair(std::move(object_key), serialized_inserted_data_size);
}
std::vector<std::future<std::unique_ptr<DataCodec>>>
GetObjectData(ChunkManager* remote_chunk_manager,
const std::vector<std::string>& remote_files) {

View File

@ -57,6 +57,19 @@ CreateArrowBuilder(DataType data_type);
std::shared_ptr<arrow::ArrayBuilder>
CreateArrowBuilder(DataType data_type, int dim);
/// \brief Utility function to create arrow:Scalar from FieldMeta.default_value
///
/// Construct a arrow::Scalar based on input field meta
/// The data_type_ is checked to determine which `one_of` member of default value shall be used
/// Note that:
/// 1. default_value shall have value
/// 2. the type check shall be guaranteed(current by go side)
///
/// \param[in] field_meta the field meta object to construct arrow::Scalar from.
/// \return an std::shared_ptr of arrow::Scalar
std::shared_ptr<arrow::Scalar>
CreateArrowScalarFromDefaultValue(const FieldMeta& field_meta);
std::shared_ptr<arrow::Schema>
CreateArrowSchema(DataType data_type, bool nullable);
@ -131,14 +144,6 @@ EncodeAndUploadIndexSlice(ChunkManager* chunk_manager,
FieldDataMeta field_meta,
std::string object_key);
std::pair<std::string, size_t>
EncodeAndUploadFieldSlice(ChunkManager* chunk_manager,
void* buf,
int64_t element_count,
FieldDataMeta field_data_meta,
const FieldMeta& field_meta,
std::string object_key);
std::vector<std::future<std::unique_ptr<DataCodec>>>
GetObjectData(ChunkManager* remote_chunk_manager,
const std::vector<std::string>& remote_files);
@ -160,9 +165,6 @@ GetNumRowsForLoadInfo(const LoadFieldDataInfo& load_info);
void
ReleaseArrowUnused();
// size_t
// getCurrentRSS();
ChunkManagerPtr
CreateChunkManager(const StorageConfig& storage_config);
@ -208,6 +210,7 @@ SortByPath(std::vector<std::string>& paths) {
});
}
// used only for test
inline std::shared_ptr<ArrowDataWrapper>
ConvertFieldDataToArrowDataWrapper(const FieldDataPtr& field_data) {
BaseEventData event_data;

View File

@ -37,7 +37,6 @@ set(MILVUS_TEST_FILES
test_bitmap_index.cpp
test_bool_index.cpp
test_c_api.cpp
test_chunk_cache.cpp
test_chunk.cpp
test_chunk_vector.cpp
test_common.cpp
@ -121,7 +120,8 @@ if (LINUX OR APPLE)
${MILVUS_TEST_FILES}
test_scalar_index_creator.cpp
test_string_index.cpp
test_array.cpp test_array_expr.cpp)
test_array.cpp
test_array_expr.cpp)
endif()
if (DEFINED AZURE_BUILD_DIR)
@ -177,6 +177,7 @@ add_executable(all_tests
target_link_libraries(all_tests
gtest
gmock
milvus_core
knowhere
milvus-storage
@ -185,26 +186,27 @@ target_link_libraries(all_tests
install(TARGETS all_tests DESTINATION unittest)
add_subdirectory(bench)
add_subdirectory(test_cachinglayer)
# if (USE_DYNAMIC_SIMD)
# add_executable(dynamic_simd_test
# test_simd.cpp)
#
#
# target_link_libraries(dynamic_simd_test
# milvus_simd
# milvus_log
# gtest
# ${CONAN_LIBS})
#
#
# install(TARGETS dynamic_simd_test DESTINATION unittest)
# endif()
add_executable(bitset_test
add_executable(bitset_test
test_bitset.cpp
)
target_link_libraries(bitset_test
milvus_bitset
gtest
target_link_libraries(bitset_test
milvus_bitset
gtest
${CONAN_LIBS}
)
install(TARGETS bitset_test DESTINATION unittest)

View File

@ -12,9 +12,12 @@
#include <cstdint>
#include <benchmark/benchmark.h>
#include <string>
#include "common/type_c.h"
#include "segcore/segment_c.h"
#include "segcore/SegmentGrowing.h"
#include "segcore/SegmentSealed.h"
#include "test_utils/DataGen.h"
#include "test_utils/storage_test_utils.h"
using namespace milvus;
using namespace milvus::query;
@ -109,7 +112,12 @@ Search_Sealed(benchmark::State& state) {
auto dataset_ = DataGen(schema, N);
return dataset_;
}();
SealedLoadFieldData(dataset_, *segment);
auto storage_config = get_default_local_storage_config();
auto cm = storage::CreateChunkManager(storage_config);
auto load_info = PrepareInsertBinlog(1, 1, 1, dataset_, cm);
auto segment_ptr = segment.get();
auto status = LoadFieldData(segment_ptr, &load_info);
ASSERT_EQ(status.error_code, Success);
auto choice = state.range(0);
if (choice == 0) {
// Brute Force

View File

@ -28,5 +28,8 @@ main(int argc, char** argv) {
get_default_local_storage_config());
milvus::storage::MmapManager::GetInstance().Init(get_default_mmap_config());
milvus::cachinglayer::Manager::ConfigureTieredStorage(
true, 1024 * 1024 * 1024, 1024 * 1024 * 1024);
return RUN_ALL_TESTS();
}

View File

@ -21,6 +21,8 @@
#include "query/PlanProto.h"
#include "query/ExecPlanNodeVisitor.h"
#include "test_utils/storage_test_utils.h"
using namespace milvus;
using namespace milvus::query;
using namespace milvus::segcore;
@ -61,7 +63,6 @@ class ArrayInvertedIndexTest : public ::testing::Test {
void
SetUp() override {
schema_ = GenTestSchema<T>();
seg_ = CreateSealedSegment(schema_);
N_ = 3000;
uint64_t seed = 19190504;
auto raw_data = DataGen(schema_, N_, seed);
@ -100,7 +101,7 @@ class ArrayInvertedIndexTest : public ::testing::Test {
}
vec_of_array_.push_back(array);
}
SealedLoadFieldData(raw_data, *seg_);
seg_ = CreateSealedWithFieldDataLoaded(schema_, raw_data);
LoadInvertedIndex();
}

View File

@ -12,21 +12,18 @@
#include <gtest/gtest.h>
#include <boost/format.hpp>
#include <optional>
#include <regex>
#include "index/IndexFactory.h"
#include "knowhere/comp/brute_force.h"
#include "pb/plan.pb.h"
#include "pb/schema.pb.h"
#include "query/Plan.h"
#include "segcore/segcore_init_c.h"
#include "segcore/SegmentSealed.h"
#include "test_utils/DataGen.h"
#include "test_utils/storage_test_utils.h"
using namespace milvus;
using namespace milvus::segcore;
namespace pb = milvus::proto;
std::unique_ptr<float[]>
GenRandomFloatVecData(int rows, int dim, int seed = 42) {
@ -121,41 +118,30 @@ class BinlogIndexTest : public ::testing::TestWithParam<Param> {
void
LoadOtherFields() {
auto dataset = DataGen(schema, data_n);
// load id
LoadFieldDataInfo row_id_info;
FieldMeta row_id_field_meta(FieldName("RowID"),
RowFieldID,
DataType::INT64,
false,
std::nullopt);
auto field_data = std::make_shared<milvus::FieldData<int64_t>>(
DataType::INT64, false);
field_data->FillFieldData(dataset.row_ids_.data(), data_n);
auto arrow_data_wrapper =
storage::ConvertFieldDataToArrowDataWrapper(field_data);
auto field_data_info = FieldDataInfo{
RowFieldID.get(),
data_n,
std::vector<std::shared_ptr<ArrowDataWrapper>>{arrow_data_wrapper}};
segment->LoadFieldData(RowFieldID, field_data_info);
// load ts
LoadFieldDataInfo ts_info;
FieldMeta ts_field_meta(FieldName("Timestamp"),
TimestampFieldID,
DataType::INT64,
false,
std::nullopt);
field_data = std::make_shared<milvus::FieldData<int64_t>>(
DataType::INT64, false);
field_data->FillFieldData(dataset.timestamps_.data(), data_n);
auto arrow_data_wrapper2 =
storage::ConvertFieldDataToArrowDataWrapper(field_data);
field_data_info =
FieldDataInfo{TimestampFieldID.get(),
data_n,
std::vector<std::shared_ptr<ArrowDataWrapper>>{
arrow_data_wrapper2}};
segment->LoadFieldData(TimestampFieldID, field_data_info);
auto cm = milvus::storage::RemoteChunkManagerSingleton::GetInstance()
.GetRemoteChunkManager();
auto load_info = PrepareInsertBinlog(kCollectionID,
kPartitionID,
kSegmentID,
dataset,
cm,
"",
{vec_field_id.get()});
segment->LoadFieldData(load_info);
}
void
LoadVectorField(std::string mmap_dir_path = "") {
auto cm = milvus::storage::RemoteChunkManagerSingleton::GetInstance()
.GetRemoteChunkManager();
auto load_info = PrepareSingleFieldInsertBinlog(kCollectionID,
kPartitionID,
kSegmentID,
vec_field_id.get(),
{vec_field_data},
cm,
mmap_dir_path);
segment->LoadFieldData(load_info);
}
protected:
@ -196,13 +182,7 @@ TEST_P(BinlogIndexTest, AccuracyWithLoadFieldData) {
segcore_config.set_enable_interim_segment_index(true);
segcore_config.set_nprobe(32);
// 1. load field data, and build binlog index for binlog data
auto arrow_data_wrapper =
storage::ConvertFieldDataToArrowDataWrapper(vec_field_data);
auto field_data_info = FieldDataInfo{
vec_field_id.get(),
data_n,
std::vector<std::shared_ptr<ArrowDataWrapper>>{arrow_data_wrapper}};
segment->LoadFieldData(vec_field_id, field_data_info);
LoadVectorField();
//assert segment has been built binlog index
EXPECT_TRUE(segment->HasIndex(vec_field_id));
@ -294,15 +274,7 @@ TEST_P(BinlogIndexTest, AccuracyWithMapFieldData) {
segcore_config.set_enable_interim_segment_index(true);
segcore_config.set_nprobe(32);
// 1. load field data, and build binlog index for binlog data
FieldDataInfo field_data_info;
field_data_info.field_id = vec_field_id.get();
field_data_info.row_count = data_n;
field_data_info.mmap_dir_path = "./data/mmap-test";
auto arrow_data_wrapper =
storage::ConvertFieldDataToArrowDataWrapper(vec_field_data);
field_data_info.arrow_reader_channel->push(arrow_data_wrapper);
field_data_info.arrow_reader_channel->close();
segment->MapFieldData(vec_field_id, field_data_info);
LoadVectorField("./data/mmap-test");
//assert segment has been built binlog index
EXPECT_TRUE(segment->HasIndex(vec_field_id));
@ -392,13 +364,7 @@ TEST_P(BinlogIndexTest, DisableInterimIndex) {
LoadOtherFields();
SegcoreSetEnableTempSegmentIndex(false);
auto arrow_data_wrapper =
storage::ConvertFieldDataToArrowDataWrapper(vec_field_data);
auto field_data_info = FieldDataInfo{
vec_field_id.get(),
data_n,
std::vector<std::shared_ptr<ArrowDataWrapper>>{arrow_data_wrapper}};
segment->LoadFieldData(vec_field_id, field_data_info);
LoadVectorField();
EXPECT_FALSE(segment->HasIndex(vec_field_id));
EXPECT_EQ(segment->get_row_count(), data_n);
@ -439,15 +405,7 @@ TEST_P(BinlogIndexTest, LoadBingLogWihIDMAP) {
segment = CreateSealedSegment(schema, collection_index_meta);
LoadOtherFields();
auto arrow_data_wrapper =
storage::ConvertFieldDataToArrowDataWrapper(vec_field_data);
auto field_data_info = FieldDataInfo{
vec_field_id.get(),
data_n,
std::vector<std::shared_ptr<ArrowDataWrapper>>{arrow_data_wrapper}};
segment->LoadFieldData(vec_field_id, field_data_info);
LoadVectorField();
EXPECT_FALSE(segment->HasIndex(vec_field_id));
EXPECT_EQ(segment->get_row_count(), data_n);
@ -461,13 +419,7 @@ TEST_P(BinlogIndexTest, LoadBinlogWithoutIndexMeta) {
segment = CreateSealedSegment(schema, collection_index_meta);
SegcoreSetEnableTempSegmentIndex(true);
auto arrow_data_wrapper =
storage::ConvertFieldDataToArrowDataWrapper(vec_field_data);
auto field_data_info = FieldDataInfo{
vec_field_id.get(),
data_n,
std::vector<std::shared_ptr<ArrowDataWrapper>>{arrow_data_wrapper}};
segment->LoadFieldData(vec_field_id, field_data_info);
LoadVectorField();
EXPECT_FALSE(segment->HasIndex(vec_field_id));
EXPECT_EQ(segment->get_row_count(), data_n);

View File

@ -47,11 +47,10 @@
#include "plan/PlanNode.h"
#include "segcore/load_index_c.h"
#include "test_utils/c_api_test_utils.h"
#include "test_utils/DataGen.h"
#include "segcore/vector_index_c.h"
#include "common/jsmn.h"
namespace chrono = std::chrono;
using namespace milvus;
using namespace milvus::test;
using namespace milvus::index;
@ -437,13 +436,8 @@ TEST(CApiTest, DeleteTest) {
auto delete_data = serialize(ids.get());
uint64_t delete_timestamps[] = {0, 0, 0};
auto offset = 0;
auto del_res = Delete(segment,
offset,
3,
delete_data.data(),
delete_data.size(),
delete_timestamps);
auto del_res = Delete(
segment, 3, delete_data.data(), delete_data.size(), delete_timestamps);
ASSERT_EQ(del_res.error_code, Success);
DeleteCollection(collection);
@ -480,9 +474,7 @@ TEST(CApiTest, MultiDeleteGrowingSegment) {
delete_pks.end());
auto delete_data = serialize(ids.get());
std::vector<uint64_t> delete_timestamps(1, dataset.timestamps_[N - 1]);
offset = 0;
auto del_res = Delete(segment,
offset,
1,
delete_data.data(),
delete_data.size(),
@ -546,9 +538,7 @@ TEST(CApiTest, MultiDeleteGrowingSegment) {
delete_pks.end());
delete_data = serialize(ids.get());
delete_timestamps[0]++;
offset = 0;
del_res = Delete(segment,
offset,
1,
delete_data.data(),
delete_data.size(),
@ -598,7 +588,11 @@ TEST(CApiTest, MultiDeleteSealedSegment) {
auto segment_interface = reinterpret_cast<SegmentInterface*>(segment);
auto sealed_segment = dynamic_cast<SegmentSealed*>(segment_interface);
SealedLoadFieldData(dataset, *sealed_segment);
auto cm = milvus::storage::RemoteChunkManagerSingleton::GetInstance()
.GetRemoteChunkManager();
auto load_info = PrepareInsertBinlog(
kCollectionID, kPartitionID, kSegmentID, dataset, cm);
sealed_segment->LoadFieldData(load_info);
// delete data pks = {1}
std::vector<int64_t> delete_pks = {1};
@ -607,9 +601,7 @@ TEST(CApiTest, MultiDeleteSealedSegment) {
delete_pks.end());
auto delete_data = serialize(ids.get());
std::vector<uint64_t> delete_timestamps(1, dataset.timestamps_[N - 1]);
auto offset = 0;
auto del_res = Delete(segment,
offset,
1,
delete_data.data(),
delete_data.size(),
@ -674,9 +666,7 @@ TEST(CApiTest, MultiDeleteSealedSegment) {
delete_pks.end());
delete_data = serialize(ids.get());
delete_timestamps[0]++;
offset = 0;
del_res = Delete(segment,
offset,
1,
delete_data.data(),
delete_data.size(),
@ -774,9 +764,7 @@ TEST(CApiTest, DeleteRepeatedPksFromGrowingSegment) {
auto delete_data = serialize(ids.get());
std::vector<uint64_t> delete_timestamps(3, dataset.timestamps_[N - 1]);
offset = 0;
auto del_res = Delete(segment,
offset,
3,
delete_data.data(),
delete_data.size(),
@ -814,7 +802,11 @@ TEST(CApiTest, DeleteRepeatedPksFromSealedSegment) {
auto segment_interface = reinterpret_cast<SegmentInterface*>(segment);
auto sealed_segment = dynamic_cast<SegmentSealed*>(segment_interface);
SealedLoadFieldData(dataset, *sealed_segment);
auto cm = milvus::storage::RemoteChunkManagerSingleton::GetInstance()
.GetRemoteChunkManager();
auto load_info = PrepareInsertBinlog(
kCollectionID, kPartitionID, kSegmentID, dataset, cm);
sealed_segment->LoadFieldData(load_info);
std::vector<proto::plan::GenericValue> retrive_row_ids;
// create retrieve plan pks in {1, 2, 3}
@ -856,10 +848,7 @@ TEST(CApiTest, DeleteRepeatedPksFromSealedSegment) {
auto delete_data = serialize(ids.get());
std::vector<uint64_t> delete_timestamps(3, dataset.timestamps_[N - 1]);
auto offset = 0;
auto del_res = Delete(segment,
offset,
3,
delete_data.data(),
delete_data.size(),
@ -984,10 +973,7 @@ TEST(CApiTest, InsertSamePkAfterDeleteOnGrowingSegment) {
auto delete_data = serialize(ids.get());
std::vector<uint64_t> delete_timestamps(3, dataset.timestamps_[N - 1]);
offset = 0;
auto del_res = Delete(segment,
offset,
3,
delete_data.data(),
delete_data.size(),
@ -1071,8 +1057,11 @@ TEST(CApiTest, InsertSamePkAfterDeleteOnSealedSegment) {
// insert data with pks = {0, 0, 1, 1, 2, 2, 3, 3, 4, 4} , timestamps = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}
auto segment_interface = reinterpret_cast<SegmentInterface*>(segment);
auto sealed_segment = dynamic_cast<SegmentSealed*>(segment_interface);
SealedLoadFieldData(dataset, *sealed_segment);
sealed_segment->get_insert_record().seal_pks();
auto cm = milvus::storage::RemoteChunkManagerSingleton::GetInstance()
.GetRemoteChunkManager();
auto load_info = PrepareInsertBinlog(
kCollectionID, kPartitionID, kSegmentID, dataset, cm);
sealed_segment->LoadFieldData(load_info);
// delete data pks = {1, 2, 3}, timestamps = {4, 4, 4}
std::vector<int64_t> delete_row_ids = {1, 2, 3};
@ -1082,10 +1071,7 @@ TEST(CApiTest, InsertSamePkAfterDeleteOnSealedSegment) {
auto delete_data = serialize(ids.get());
std::vector<uint64_t> delete_timestamps(3, dataset.timestamps_[4]);
auto offset = 0;
auto del_res = Delete(segment,
offset,
3,
delete_data.data(),
delete_data.size(),
@ -1372,14 +1358,8 @@ TEST(CApiTest, GetDeletedCountTest) {
auto delete_data = serialize(ids.get());
uint64_t delete_timestamps[] = {0, 0, 0};
auto offset = 0;
auto del_res = Delete(segment,
offset,
3,
delete_data.data(),
delete_data.size(),
delete_timestamps);
auto del_res = Delete(
segment, 3, delete_data.data(), delete_data.size(), delete_timestamps);
ASSERT_EQ(del_res.error_code, Success);
// TODO: assert(deleted_count == len(delete_row_ids))
@ -1453,14 +1433,8 @@ TEST(CApiTest, GetRealCount) {
dataset.timestamps_[N - 1] + 2,
dataset.timestamps_[N - 1] + 3};
auto del_offset = 0;
auto del_res = Delete(segment,
del_offset,
3,
delete_data.data(),
delete_data.size(),
delete_timestamps);
auto del_res = Delete(
segment, 3, delete_data.data(), delete_data.size(), delete_timestamps);
ASSERT_EQ(del_res.error_code, Success);
auto real_count = GetRealCount(segment);
@ -2097,7 +2071,7 @@ Test_Indexing_Without_Predicate() {
knowhere::Version::GetCurrentVersion().VersionNumber());
AppendIndex(c_load_index_info, (CBinarySet)&binary_set);
// load index for vec field, load raw data for scalar field
auto sealed_segment = SealedCreator(schema, dataset);
auto sealed_segment = CreateSealedWithFieldDataLoaded(schema, dataset);
sealed_segment->DropFieldData(FieldId(100));
sealed_segment->LoadIndex(*(LoadIndexInfo*)c_load_index_info);
@ -2254,7 +2228,7 @@ TEST(CApiTest, Indexing_Expr_Without_Predicate) {
AppendIndex(c_load_index_info, (CBinarySet)&binary_set);
// load index for vec field, load raw data for scalar field
auto sealed_segment = SealedCreator(schema, dataset);
auto sealed_segment = CreateSealedWithFieldDataLoaded(schema, dataset);
sealed_segment->DropFieldData(FieldId(100));
sealed_segment->LoadIndex(*(LoadIndexInfo*)c_load_index_info);
@ -2432,7 +2406,7 @@ TEST(CApiTest, Indexing_With_float_Predicate_Range) {
AppendIndex(c_load_index_info, (CBinarySet)&binary_set);
// load index for vec field, load raw data for scalar field
auto sealed_segment = SealedCreator(schema, dataset);
auto sealed_segment = CreateSealedWithFieldDataLoaded(schema, dataset);
sealed_segment->DropFieldData(FieldId(100));
sealed_segment->LoadIndex(*(LoadIndexInfo*)c_load_index_info);
@ -2612,7 +2586,7 @@ TEST(CApiTest, Indexing_Expr_With_float_Predicate_Range) {
AppendIndex(c_load_index_info, (CBinarySet)&binary_set);
// load index for vec field, load raw data for scalar field
auto sealed_segment = SealedCreator(schema, dataset);
auto sealed_segment = CreateSealedWithFieldDataLoaded(schema, dataset);
sealed_segment->DropFieldData(FieldId(100));
sealed_segment->LoadIndex(*(LoadIndexInfo*)c_load_index_info);
@ -2784,7 +2758,7 @@ TEST(CApiTest, Indexing_With_float_Predicate_Term) {
AppendIndex(c_load_index_info, (CBinarySet)&binary_set);
// load index for vec field, load raw data for scalar field
auto sealed_segment = SealedCreator(schema, dataset);
auto sealed_segment = CreateSealedWithFieldDataLoaded(schema, dataset);
sealed_segment->DropFieldData(FieldId(100));
sealed_segment->LoadIndex(*(LoadIndexInfo*)c_load_index_info);
@ -2957,7 +2931,7 @@ TEST(CApiTest, Indexing_Expr_With_float_Predicate_Term) {
AppendIndex(c_load_index_info, (CBinarySet)&binary_set);
// load index for vec field, load raw data for scalar field
auto sealed_segment = SealedCreator(schema, dataset);
auto sealed_segment = CreateSealedWithFieldDataLoaded(schema, dataset);
sealed_segment->DropFieldData(FieldId(100));
sealed_segment->LoadIndex(*(LoadIndexInfo*)c_load_index_info);
@ -3138,7 +3112,7 @@ TEST(CApiTest, Indexing_With_binary_Predicate_Range) {
AppendIndex(c_load_index_info, (CBinarySet)&binary_set);
// load index for vec field, load raw data for scalar field
auto sealed_segment = SealedCreator(schema, dataset);
auto sealed_segment = CreateSealedWithFieldDataLoaded(schema, dataset);
sealed_segment->DropFieldData(FieldId(100));
sealed_segment->LoadIndex(*(LoadIndexInfo*)c_load_index_info);
@ -3319,7 +3293,7 @@ TEST(CApiTest, Indexing_Expr_With_binary_Predicate_Range) {
AppendIndex(c_load_index_info, (CBinarySet)&binary_set);
// load index for vec field, load raw data for scalar field
auto sealed_segment = SealedCreator(schema, dataset);
auto sealed_segment = CreateSealedWithFieldDataLoaded(schema, dataset);
sealed_segment->DropFieldData(FieldId(100));
sealed_segment->LoadIndex(*(LoadIndexInfo*)c_load_index_info);
@ -3494,7 +3468,7 @@ TEST(CApiTest, Indexing_With_binary_Predicate_Term) {
AppendIndex(c_load_index_info, (CBinarySet)&binary_set);
// load index for vec field, load raw data for scalar field
auto sealed_segment = SealedCreator(schema, dataset);
auto sealed_segment = CreateSealedWithFieldDataLoaded(schema, dataset);
sealed_segment->DropFieldData(FieldId(100));
sealed_segment->LoadIndex(*(LoadIndexInfo*)c_load_index_info);
@ -3692,7 +3666,7 @@ TEST(CApiTest, Indexing_Expr_With_binary_Predicate_Term) {
AppendIndex(c_load_index_info, (CBinarySet)&binary_set);
// load index for vec field, load raw data for scalar field
auto sealed_segment = SealedCreator(schema, dataset);
auto sealed_segment = CreateSealedWithFieldDataLoaded(schema, dataset);
sealed_segment->DropFieldData(FieldId(100));
sealed_segment->LoadIndex(*(LoadIndexInfo*)c_load_index_info);
@ -3741,27 +3715,6 @@ TEST(CApiTest, Indexing_Expr_With_binary_Predicate_Term) {
DeleteSearchResultDataBlobs(cSearchResultData);
}
TEST(CApiTest, SealedSegmentTest) {
auto collection = NewCollection(get_default_schema_config().c_str());
CSegmentInterface segment;
auto status = NewSegment(collection, Sealed, -1, &segment, false);
ASSERT_EQ(status.error_code, Success);
int N = 1000;
std::default_random_engine e(67);
auto ages = std::vector<int64_t>(N);
for (auto& age : ages) {
age = e() % 2000;
}
auto res = LoadFieldRawData(segment, 101, ages.data(), N);
ASSERT_EQ(res.error_code, Success);
auto count = GetRowCount(segment);
ASSERT_EQ(count, N);
DeleteCollection(collection);
DeleteSegment(segment);
}
TEST(CApiTest, SealedSegment_search_float_Predicate_Range) {
constexpr auto TOPK = 5;
@ -3778,8 +3731,6 @@ TEST(CApiTest, SealedSegment_search_float_Predicate_Range) {
auto vec_col = dataset.get_col<float>(FieldId(100));
auto query_ptr = vec_col.data() + BIAS * DIM;
auto counter_col = dataset.get_col<int64_t>(FieldId(101));
const char* raw_plan = R"(vector_anns: <
field_id: 100
predicates: <
@ -3880,17 +3831,21 @@ TEST(CApiTest, SealedSegment_search_float_Predicate_Range) {
vec_index->Query(query_dataset, search_info, nullptr, result_on_index);
EXPECT_EQ(result_on_index.distances_.size(), num_queries * TOPK);
status = LoadFieldRawData(segment, 101, counter_col.data(), N);
ASSERT_EQ(status.error_code, Success);
status = LoadFieldRawData(segment, 0, dataset.row_ids_.data(), N);
ASSERT_EQ(status.error_code, Success);
status = LoadFieldRawData(segment, 1, dataset.timestamps_.data(), N);
auto cm = milvus::storage::RemoteChunkManagerSingleton::GetInstance()
.GetRemoteChunkManager();
auto excluded_field_ids = GetExcludedFieldIds(dataset.schema_, {0, 1, 101});
auto load_info = PrepareInsertBinlog(kCollectionID,
kPartitionID,
kSegmentID,
dataset,
cm,
"",
excluded_field_ids);
status = LoadFieldData(segment, &load_info);
ASSERT_EQ(status.error_code, Success);
// load index for vec field, load raw data for scalar field
auto sealed_segment = SealedCreator(schema, dataset);
auto sealed_segment = CreateSealedWithFieldDataLoaded(schema, dataset);
sealed_segment->DropFieldData(FieldId(100));
sealed_segment->LoadIndex(*(LoadIndexInfo*)c_load_index_info);
@ -3917,7 +3872,6 @@ TEST(CApiTest, SealedSegment_search_float_Predicate_Range) {
}
TEST(CApiTest, SealedSegment_search_without_predicates) {
constexpr auto TOPK = 5;
std::string schema_string = generate_collection_schema<milvus::FloatVector>(
knowhere::metric::L2, DIM);
auto collection = NewCollection(schema_string.c_str());
@ -3926,16 +3880,8 @@ TEST(CApiTest, SealedSegment_search_without_predicates) {
auto status = NewSegment(collection, Sealed, -1, &segment, false);
ASSERT_EQ(status.error_code, Success);
auto N = ROW_COUNT;
uint64_t ts_offset = 1000;
auto dataset = DataGen(schema, N, ts_offset);
auto vec_col = dataset.get_col<float>(FieldId(100));
auto query_ptr = vec_col.data() + BIAS * DIM;
auto vec_array = dataset.get_col(FieldId(100));
auto vec_data = serialize(vec_array.get());
auto counter_col = dataset.get_col<int64_t>(FieldId(101));
auto dataset = DataGen(schema, ROW_COUNT, ts_offset);
const char* raw_plan = R"(vector_anns: <
field_id: 100
@ -3949,16 +3895,18 @@ TEST(CApiTest, SealedSegment_search_without_predicates) {
>)";
auto plan_str = translate_text_plan_to_binary_plan(raw_plan);
status = LoadFieldRawData(segment, 100, vec_data.data(), N);
ASSERT_EQ(status.error_code, Success);
status = LoadFieldRawData(segment, 101, counter_col.data(), N);
ASSERT_EQ(status.error_code, Success);
status = LoadFieldRawData(segment, 0, dataset.row_ids_.data(), N);
ASSERT_EQ(status.error_code, Success);
status = LoadFieldRawData(segment, 1, dataset.timestamps_.data(), N);
auto cm = milvus::storage::RemoteChunkManagerSingleton::GetInstance()
.GetRemoteChunkManager();
auto excluded_field_ids =
GetExcludedFieldIds(dataset.schema_, {0, 1, 100, 101});
auto load_info = PrepareInsertBinlog(kCollectionID,
kPartitionID,
kSegmentID,
dataset,
cm,
"",
excluded_field_ids);
status = LoadFieldData(segment, &load_info);
ASSERT_EQ(status.error_code, Success);
int num_queries = 10;
@ -3977,14 +3925,17 @@ TEST(CApiTest, SealedSegment_search_without_predicates) {
std::vector<CPlaceholderGroup> placeholderGroups;
placeholderGroups.push_back(placeholderGroup);
CSearchResult search_result;
auto res =
CSearch(segment, plan, placeholderGroup, N + ts_offset, &search_result);
auto res = CSearch(
segment, plan, placeholderGroup, ROW_COUNT + ts_offset, &search_result);
std::cout << res.error_msg << std::endl;
ASSERT_EQ(res.error_code, Success);
CSearchResult search_result2;
auto res2 = CSearch(
segment, plan, placeholderGroup, N + ts_offset, &search_result2);
auto res2 = CSearch(segment,
plan,
placeholderGroup,
ROW_COUNT + ts_offset,
&search_result2);
ASSERT_EQ(res2.error_code, Success);
DeleteSearchPlan(plan);
@ -4002,9 +3953,6 @@ TEST(CApiTest, SealedSegment_search_float_With_Expr_Predicate_Range) {
knowhere::metric::L2, DIM);
auto collection = NewCollection(schema_string.c_str());
auto schema = ((segcore::Collection*)collection)->get_schema();
CSegmentInterface segment;
auto status = NewSegment(collection, Sealed, -1, &segment, false);
ASSERT_EQ(status.error_code, Success);
auto N = ROW_COUNT;
auto dataset = DataGen(schema, N);
@ -4062,7 +4010,7 @@ TEST(CApiTest, SealedSegment_search_float_With_Expr_Predicate_Range) {
// search on segment's small index
void* plan = nullptr;
auto binary_plan = translate_text_plan_to_binary_plan(serialized_expr_plan);
status = CreateSearchPlanByExpr(
auto status = CreateSearchPlanByExpr(
collection, binary_plan.data(), binary_plan.size(), &plan);
ASSERT_EQ(status.error_code, Success);
@ -4103,18 +4051,10 @@ TEST(CApiTest, SealedSegment_search_float_With_Expr_Predicate_Range) {
knowhere::Version::GetCurrentVersion().VersionNumber());
AppendIndex(c_load_index_info, (CBinarySet)&binary_set);
auto segment = CreateSealedWithFieldDataLoaded(schema, dataset);
// load vec index
status = UpdateSealedSegmentIndex(segment, c_load_index_info);
ASSERT_EQ(status.error_code, Success);
// load raw data
status = LoadFieldRawData(segment, 101, counter_col.data(), N);
ASSERT_EQ(status.error_code, Success);
status = LoadFieldRawData(segment, 0, dataset.row_ids_.data(), N);
ASSERT_EQ(status.error_code, Success);
status = LoadFieldRawData(segment, 1, dataset.timestamps_.data(), N);
status = UpdateSealedSegmentIndex(segment.get(), c_load_index_info);
ASSERT_EQ(status.error_code, Success);
// gen query dataset
@ -4133,7 +4073,7 @@ TEST(CApiTest, SealedSegment_search_float_With_Expr_Predicate_Range) {
}
CSearchResult c_search_result_on_bigIndex;
auto res_after_load_index = CSearch(segment,
auto res_after_load_index = CSearch(segment.get(),
plan,
placeholderGroup,
timestamp,
@ -4151,39 +4091,6 @@ TEST(CApiTest, SealedSegment_search_float_With_Expr_Predicate_Range) {
DeletePlaceholderGroup(placeholderGroup);
DeleteSearchResult(c_search_result_on_bigIndex);
DeleteCollection(collection);
DeleteSegment(segment);
}
TEST(CApiTest, SealedSegment_Update_Field_Size) {
auto schema = std::make_shared<Schema>();
auto str_fid = schema->AddDebugField("string", DataType::VARCHAR);
auto vec_fid = schema->AddDebugField(
"vector_float", DataType::VECTOR_FLOAT, DIM, "L2");
schema->set_primary_field_id(str_fid);
auto segment = CreateSealedSegment(schema).release();
auto N = ROW_COUNT;
int row_size = 10;
// update row_size =10 with n rows
auto status =
UpdateFieldRawDataSize(segment, str_fid.get(), N, N * row_size);
ASSERT_EQ(status.error_code, Success);
ASSERT_EQ(segment->get_field_avg_size(str_fid), row_size);
// load data and update avg field size
std::vector<std::string> str_datas;
int64_t total_size = 0;
for (int i = 0; i < N; ++i) {
auto str = "string_data_" + std::to_string(i);
total_size += str.size() + sizeof(uint32_t);
str_datas.emplace_back(str);
}
auto res = LoadFieldRawData(segment, str_fid.get(), str_datas.data(), N);
ASSERT_EQ(res.error_code, Success);
DeleteSegment(segment);
}
TEST(CApiTest, GrowingSegment_Load_Field_Data) {
@ -4207,13 +4114,7 @@ TEST(CApiTest, GrowingSegment_Load_Field_Data) {
auto storage_config = get_default_local_storage_config();
auto cm = storage::CreateChunkManager(storage_config);
auto load_info =
PrepareInsertBinlog(1,
2,
3,
storage_config.root_path + "/" + "test_load_sealed",
raw_data,
cm);
auto load_info = PrepareInsertBinlog(1, 2, 3, raw_data, cm);
auto status = LoadFieldData(segment, &load_info);
ASSERT_EQ(status.error_code, Success);
@ -4266,13 +4167,7 @@ TEST(CApiTest, GrowingSegment_Load_Field_Data_Lack_Binlog_Rows) {
auto storage_config = get_default_local_storage_config();
auto cm = storage::CreateChunkManager(storage_config);
auto load_info =
PrepareInsertBinlog(1,
2,
3,
storage_config.root_path + "/" + "test_load_sealed",
raw_data,
cm);
auto load_info = PrepareInsertBinlog(1, 2, 3, raw_data, cm);
raw_data.raw_->mutable_fields_data()->AddAllocated(array.release());
load_info.field_infos.emplace(
@ -4342,13 +4237,7 @@ TEST(CApiTest, DISABLED_SealedSegment_Load_Field_Data_Lack_Binlog_Rows) {
auto storage_config = get_default_local_storage_config();
auto cm = storage::CreateChunkManager(storage_config);
auto load_info =
PrepareInsertBinlog(1,
2,
3,
storage_config.root_path + "/" + "test_load_sealed",
raw_data,
cm);
auto load_info = PrepareInsertBinlog(1, 2, 3, raw_data, cm);
raw_data.raw_->mutable_fields_data()->AddAllocated(array.release());
load_info.field_infos.emplace(
@ -4375,7 +4264,7 @@ TEST(CApiTest, DISABLED_SealedSegment_Load_Field_Data_Lack_Binlog_Rows) {
DeleteSegment(segment);
}
TEST(CApiTest, RetriveScalarFieldFromSealedSegmentWithIndex) {
TEST(CApiTest, RetrieveScalarFieldFromSealedSegmentWithIndex) {
auto schema = std::make_shared<Schema>();
auto i8_fid = schema->AddDebugField("age8", DataType::INT8);
auto i16_fid = schema->AddDebugField("age16", DataType::INT16);
@ -4391,26 +4280,19 @@ TEST(CApiTest, RetriveScalarFieldFromSealedSegmentWithIndex) {
auto raw_data = DataGen(schema, N);
LoadIndexInfo load_index_info;
// load timestamp field
auto res = LoadFieldRawData(
segment, TimestampFieldID.get(), raw_data.timestamps_.data(), N);
ASSERT_EQ(res.error_code, Success);
auto count = GetRowCount(segment);
ASSERT_EQ(count, N);
// load rowid field
res = LoadFieldRawData(
segment, RowFieldID.get(), raw_data.row_ids_.data(), N);
ASSERT_EQ(res.error_code, Success);
count = GetRowCount(segment);
ASSERT_EQ(count, N);
// load int64 field
res = LoadFieldRawData(
segment, i64_fid.get(), raw_data.get_col<int64_t>(i64_fid).data(), N);
ASSERT_EQ(res.error_code, Success);
count = GetRowCount(segment);
ASSERT_EQ(count, N);
auto cm = milvus::storage::RemoteChunkManagerSingleton::GetInstance()
.GetRemoteChunkManager();
auto excluded_field_ids =
GetExcludedFieldIds(raw_data.schema_, {0, 1, i64_fid.get()});
auto load_info = PrepareInsertBinlog(kCollectionID,
kPartitionID,
kSegmentID,
raw_data,
cm,
"",
excluded_field_ids);
auto status = LoadFieldData(segment, &load_info);
ASSERT_EQ(status.error_code, Success);
// load index for int8 field
auto age8_col = raw_data.get_col<int8_t>(i8_fid);
@ -4492,7 +4374,7 @@ TEST(CApiTest, RetriveScalarFieldFromSealedSegmentWithIndex) {
plan->field_ids_ = target_field_ids;
CRetrieveResult* retrieve_result = nullptr;
res = CRetrieve(
auto res = CRetrieve(
segment, plan.get(), raw_data.timestamps_[N - 1], &retrieve_result);
ASSERT_EQ(res.error_code, Success);
auto query_result = std::make_unique<proto::segcore::RetrieveResults>();

View File

@ -28,6 +28,7 @@
#include "segcore/InsertRecord.h"
#include "mmap/ChunkedColumn.h"
#include "test_utils/DataGen.h"
#include "test_cachinglayer/cachinglayer_test_utils.h"
using namespace milvus;
using namespace milvus::query;
@ -103,6 +104,8 @@ class CachedSearchIteratorTest
static std::unique_ptr<ConcurrentVector<milvus::FloatVector>> vector_base_;
static std::shared_ptr<ChunkedColumn> column_;
static std::vector<std::vector<char>> column_data_;
static std::shared_ptr<Schema> schema_;
static FieldId fakevec_id_;
IndexBase* index_hnsw_ = nullptr;
MetricType metric_type_ = kMetricType;
@ -140,7 +143,7 @@ class CachedSearchIteratorTest
case ConstructorType::ChunkedColumn:
return std::make_unique<CachedSearchIterator>(
column_,
column_.get(),
search_dataset_,
search_info,
std::map<std::string, std::string>{},
@ -245,35 +248,42 @@ class CachedSearchIteratorTest
static void
SetUpChunkedColumn() {
column_ = std::make_unique<ChunkedColumn>();
auto field_meta = schema_->operator[](fakevec_id_);
const size_t num_chunks_ = (nb_ + kSizePerChunk - 1) / kSizePerChunk;
column_data_.resize(num_chunks_);
size_t offset = 0;
std::vector<std::unique_ptr<Chunk>> chunks;
std::vector<int64_t> num_rows_per_chunk;
for (size_t i = 0; i < num_chunks_; ++i) {
const size_t rows =
std::min(static_cast<size_t>(nb_ - offset), kSizePerChunk);
num_rows_per_chunk.push_back(rows);
const size_t buf_size = rows * dim_ * sizeof(float);
auto& chunk_data = column_data_[i];
chunk_data.resize(buf_size);
memcpy(chunk_data.data(),
base_dataset_.cbegin() + offset * dim_,
rows * dim_ * sizeof(float));
column_->AddChunk(std::make_shared<FixedWidthChunk>(
chunks.emplace_back(std::make_unique<FixedWidthChunk>(
rows, dim_, chunk_data.data(), buf_size, sizeof(float), false));
offset += rows;
}
auto translator = std::make_unique<TestChunkTranslator>(
num_rows_per_chunk, "", std::move(chunks));
column_ =
std::make_shared<ChunkedColumn>(std::move(translator), field_meta);
}
static void
SetUpTestSuite() {
auto schema = std::make_shared<Schema>();
auto fakevec_id = schema->AddDebugField(
schema_ = std::make_shared<Schema>();
fakevec_id_ = schema_->AddDebugField(
"fakevec", DataType::VECTOR_FLOAT, dim_, kMetricType);
// generate base dataset
base_dataset_ =
segcore::DataGen(schema, nb_).get_col<float>(fakevec_id);
segcore::DataGen(schema_, nb_).get_col<float>(fakevec_id_);
// generate query dataset
query_dataset_ = {base_dataset_.cbegin(),
@ -348,6 +358,8 @@ std::unique_ptr<ConcurrentVector<milvus::FloatVector>>
CachedSearchIteratorTest::vector_base_ = nullptr;
std::shared_ptr<ChunkedColumn> CachedSearchIteratorTest::column_ = nullptr;
std::vector<std::vector<char>> CachedSearchIteratorTest::column_data_;
std::shared_ptr<Schema> CachedSearchIteratorTest::schema_{nullptr};
FieldId CachedSearchIteratorTest::fakevec_id_(0);
/********* Testcases Start **********/
@ -605,7 +617,6 @@ TEST_P(CachedSearchIteratorTest, NextBatchtAllBatchesNormal) {
SearchInfo search_info = GetDefaultNormalSearchInfo();
const std::vector<size_t> kBatchSizes = {
1, 7, 43, 99, 100, 101, 1000, 1005};
// const std::vector<size_t> kBatchSizes = {1005};
for (size_t batch_size : kBatchSizes) {
search_info.iterator_v2_info_->batch_size = batch_size;
@ -745,7 +756,7 @@ TEST_P(CachedSearchIteratorTest, ConstructorWithInvalidParams) {
data_type_),
SegcoreError);
EXPECT_THROW(auto iterator = std::make_unique<CachedSearchIterator>(
column_,
column_.get(),
dataset::SearchDataset{},
search_info,
std::map<std::string, std::string>{},

View File

@ -0,0 +1,29 @@
# Copyright (C) 2019-2020 Zilliz. All rights reserved.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software distributed under the License
# is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
# or implied. See the License for the specific language governing permissions and limitations under the License
set(CACHINGLAYER_TEST_FILES
../init_gtest.cpp
test_dlist.cpp
test_cache_slot.cpp
)
add_executable(cachinglayer_test
${CACHINGLAYER_TEST_FILES}
)
target_link_libraries(cachinglayer_test
gtest
gmock
milvus_core
milvus-storage
)
install(TARGETS cachinglayer_test DESTINATION unittest)

View File

@ -0,0 +1,171 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#pragma once
#include <gtest/gtest.h>
#include <string>
#include <vector>
#include "cachinglayer/Translator.h"
#include "common/Chunk.h"
#include "segcore/storagev1translator/ChunkTranslator.h"
#include "cachinglayer/lrucache/DList.h"
namespace milvus {
using namespace cachinglayer;
class TestChunkTranslator : public Translator<milvus::Chunk> {
public:
TestChunkTranslator(std::vector<int64_t> num_rows_per_chunk,
std::string key,
std::vector<std::unique_ptr<Chunk>>&& chunks)
: Translator<milvus::Chunk>(),
num_cells_(num_rows_per_chunk.size()),
chunks_(std::move(chunks)),
meta_(segcore::storagev1translator::CTMeta(StorageType::MEMORY)) {
meta_.num_rows_until_chunk_.reserve(num_cells_ + 1);
meta_.num_rows_until_chunk_.push_back(0);
int total_rows = 0;
for (int i = 0; i < num_cells_; ++i) {
meta_.num_rows_until_chunk_.push_back(
meta_.num_rows_until_chunk_[i] + num_rows_per_chunk[i]);
total_rows += num_rows_per_chunk[i];
}
key_ = key;
}
~TestChunkTranslator() override {
}
size_t
num_cells() const override {
return num_cells_;
}
cid_t
cell_id_of(uid_t uid) const override {
return uid;
}
ResourceUsage
estimated_byte_size_of_cell(cid_t cid) const override {
return ResourceUsage(0, 0);
}
const std::string&
key() const override {
return key_;
}
Meta*
meta() override {
return &meta_;
}
std::vector<std::pair<cid_t, std::unique_ptr<milvus::Chunk>>>
get_cells(const std::vector<cid_t>& cids) override {
std::vector<std::pair<cid_t, std::unique_ptr<milvus::Chunk>>> res;
res.reserve(cids.size());
for (auto cid : cids) {
AssertInfo(cid < chunks_.size() && chunks_[cid] != nullptr,
"TestChunkTranslator assumes no eviction.");
res.emplace_back(cid, std::move(chunks_[cid]));
}
return res;
}
private:
size_t num_cells_;
segcore::storagev1translator::CTMeta meta_;
std::string key_;
std::vector<std::unique_ptr<Chunk>> chunks_;
};
namespace cachinglayer::internal {
class DListTestFriend {
public:
static ResourceUsage
get_used_memory(const DList& dlist) {
return dlist.used_memory_.load();
}
static ResourceUsage
get_max_memory(const DList& dlist) {
std::lock_guard lock(dlist.list_mtx_);
return dlist.max_memory_;
}
static ListNode*
get_head(const DList& dlist) {
std::lock_guard lock(dlist.list_mtx_);
return dlist.head_;
}
static ListNode*
get_tail(const DList& dlist) {
std::lock_guard lock(dlist.list_mtx_);
return dlist.tail_;
}
static void
test_push_head(DList* dlist, ListNode* node) {
std::lock_guard lock(dlist->list_mtx_);
dlist->pushHead(node);
}
static void
test_pop_item(DList* dlist, ListNode* node) {
std::lock_guard lock(dlist->list_mtx_);
dlist->popItem(node);
}
static void
test_add_used_memory(DList* dlist, const ResourceUsage& size) {
std::lock_guard lock(dlist->list_mtx_);
dlist->used_memory_ += size;
}
// nodes are from tail to head
static void
verify_list(DList* dlist, std::vector<ListNode*> nodes) {
std::lock_guard lock(dlist->list_mtx_);
EXPECT_EQ(nodes.front(), dlist->tail_);
EXPECT_EQ(nodes.back(), dlist->head_);
for (size_t i = 0; i < nodes.size(); ++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->prev_, expected_prev);
EXPECT_EQ(current->next_, expected_next);
}
}
static void
verify_integrity(DList* dlist) {
std::lock_guard lock(dlist->list_mtx_);
ResourceUsage total_size;
EXPECT_EQ(dlist->tail_->prev_, nullptr);
ListNode* current = dlist->tail_;
ListNode* prev = nullptr;
while (current != nullptr) {
EXPECT_EQ(current->prev_, prev);
total_size += current->size();
prev = current;
current = current->next_;
}
EXPECT_EQ(prev, dlist->head_);
EXPECT_EQ(dlist->head_->next_, nullptr);
EXPECT_EQ(total_size, dlist->used_memory_.load());
}
};
} // namespace cachinglayer::internal
} // namespace milvus

View File

@ -0,0 +1,85 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#pragma once
#include <gmock/gmock.h>
#include <fmt/format.h>
#include "cachinglayer/lrucache/ListNode.h"
#include "cachinglayer/Utils.h"
namespace milvus::cachinglayer::internal {
class MockListNode : public ListNode {
public:
MockListNode(DList* dlist,
ResourceUsage size,
const std::string& key = "mock_key",
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;
});
}
MOCK_METHOD(void, clear_data, (), (override));
std::string
key() const override {
return mock_key_;
}
// Directly manipulate state for test setup (Use carefully!)
void
test_set_state(State new_state) {
std::unique_lock lock(mtx_);
state_ = new_state;
}
State
test_get_state() {
std::shared_lock lock(mtx_);
return state_;
}
void
test_set_pin_count(int count) {
pin_count_.store(count);
}
int
test_get_pin_count() const {
return pin_count_.load();
}
// Expose mutex for lock testing
std::shared_mutex&
test_get_mutex() {
return mtx_;
}
ListNode*
test_get_prev() const {
return prev_;
}
ListNode*
test_get_next() const {
return next_;
}
private:
friend class DListTest;
friend class DListTestFriend;
std::string mock_key_;
};
} // namespace milvus::cachinglayer::internal

View File

@ -0,0 +1,846 @@
#include <gtest/gtest.h>
#include <folly/executors/CPUThreadPoolExecutor.h>
#include <folly/executors/InlineExecutor.h>
#include <folly/futures/Future.h>
#include <chrono>
#include <memory>
#include <random>
#include <stdexcept>
#include <thread>
#include <vector>
#include <utility>
#include <unordered_map>
#include <unordered_set>
#include "cachinglayer/CacheSlot.h"
#include "cachinglayer/Translator.h"
#include "cachinglayer/Utils.h"
#include "cachinglayer/lrucache/DList.h"
#include "cachinglayer/lrucache/ListNode.h"
#include "cachinglayer_test_utils.h"
using namespace milvus::cachinglayer;
using namespace milvus::cachinglayer::internal;
using cl_uid_t = milvus::cachinglayer::uid_t;
struct TestCell {
int data;
cid_t cid;
TestCell(int d, cid_t id) : data(d), cid(id) {
}
size_t
CellByteSize() const {
return sizeof(data) + sizeof(cid);
}
};
class MockTranslator : public Translator<TestCell> {
public:
MockTranslator(std::vector<std::pair<cid_t, int64_t>> cell_sizes,
std::unordered_map<cl_uid_t, cid_t> uid_to_cid_map,
const std::string& key,
StorageType storage_type,
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()) {
cid_set_.reserve(cell_sizes.size());
cell_sizes_.reserve(cell_sizes.size());
for (const auto& pair : cell_sizes) {
cid_t cid = pair.first;
int64_t size = pair.second;
cid_set_.insert(cid);
cell_sizes_[cid] = size;
cid_load_delay_ms_[cid] = 0;
}
}
size_t
num_cells() const override {
return num_unique_cids_;
}
cid_t
cell_id_of(cl_uid_t uid) const override {
auto it = uid_to_cid_map_.find(uid);
if (it != uid_to_cid_map_.end()) {
if (cid_set_.count(it->second)) {
return it->second;
}
}
return static_cast<cid_t>(num_unique_cids_);
}
ResourceUsage
estimated_byte_size_of_cell(cid_t cid) const override {
auto it = cell_sizes_.find(cid);
if (it != cell_sizes_.end()) {
return ResourceUsage{it->second, 0};
}
return ResourceUsage{1, 0};
}
const std::string&
key() const override {
return key_;
}
Meta*
meta() override {
return &meta_;
}
std::vector<std::pair<cid_t, std::unique_ptr<TestCell>>>
get_cells(const std::vector<cid_t>& cids) override {
if (!for_concurrent_test_) {
get_cells_call_count_++;
requested_cids_.push_back(cids);
}
if (load_should_throw_) {
throw std::runtime_error("Simulated load error");
}
std::vector<std::pair<cid_t, std::unique_ptr<TestCell>>> result;
for (cid_t cid : cids) {
auto delay_it = cid_load_delay_ms_.find(cid);
if (delay_it != cid_load_delay_ms_.end() && delay_it->second > 0) {
std::this_thread::sleep_for(
std::chrono::milliseconds(delay_it->second));
}
result.emplace_back(
cid,
std::make_unique<TestCell>(static_cast<int>(cid * 10), cid));
if (auto extra_cids = extra_cids_.find(cid);
extra_cids != extra_cids_.end()) {
for (cid_t extra_cid : extra_cids->second) {
// if extra cid is not explicitly requested, and not yet added as extra cell by other
// cells, add it to the result.
if (std::find(cids.begin(), cids.end(), extra_cid) ==
cids.end() &&
std::find_if(result.begin(),
result.end(),
[extra_cid](const auto& pair) {
return pair.first == extra_cid;
}) == result.end()) {
result.emplace_back(
extra_cid,
std::make_unique<TestCell>(
static_cast<int>(extra_cid * 10), extra_cid));
}
}
}
}
return result;
}
void
SetCidLoadDelay(const std::unordered_map<cid_t, int>& delays) {
for (const auto& pair : delays) {
cid_load_delay_ms_[pair.first] = pair.second;
}
}
void
SetShouldThrow(bool should_throw) {
load_should_throw_ = should_throw;
}
// for some cid, translator will return extra cells.
void
SetExtraReturnCids(
std::unordered_map<cid_t, std::vector<cid_t>> extra_cids) {
extra_cids_ = extra_cids;
}
int
GetCellsCallCount() const {
EXPECT_FALSE(for_concurrent_test_);
return get_cells_call_count_;
}
const std::vector<std::vector<cid_t>>&
GetRequestedCids() const {
EXPECT_FALSE(for_concurrent_test_);
return requested_cids_;
}
void
ResetCounters() {
ASSERT_FALSE(for_concurrent_test_);
get_cells_call_count_ = 0;
requested_cids_.clear();
}
private:
std::unordered_map<cl_uid_t, cid_t> uid_to_cid_map_;
std::unordered_map<cid_t, int64_t> cell_sizes_;
std::unordered_set<cid_t> cid_set_;
const size_t num_unique_cids_;
const std::string key_;
Meta meta_;
std::unordered_map<cid_t, int> cid_load_delay_ms_;
bool load_should_throw_ = false;
std::unordered_map<cid_t, std::vector<cid_t>> extra_cids_;
std::atomic<int> get_cells_call_count_ = 0;
std::vector<std::vector<cid_t>> requested_cids_;
// this class is not concurrent safe, so if for concurrent test, do not track usage
bool for_concurrent_test_ = false;
};
class CacheSlotTest : public ::testing::Test {
protected:
std::unique_ptr<DList> dlist_;
MockTranslator* translator_ = nullptr;
std::shared_ptr<CacheSlot<TestCell>> cache_slot_;
std::vector<std::pair<cid_t, int64_t>> cell_sizes_ = {
{0, 50}, {1, 150}, {2, 100}, {3, 200}, {4, 75}};
std::unordered_map<cl_uid_t, cid_t> uid_to_cid_map_ = {{10, 0},
{11, 0},
{20, 1},
{30, 2},
{31, 2},
{32, 2},
{40, 3},
{50, 4},
{51, 4}};
size_t NUM_UNIQUE_CIDS = 5;
int64_t TOTAL_CELL_SIZE_BYTES = 50 + 150 + 100 + 200 + 75;
int64_t MEMORY_LIMIT = TOTAL_CELL_SIZE_BYTES * 2;
static constexpr int64_t DISK_LIMIT = 0;
const std::string SLOT_KEY = "test_slot";
void
SetUp() override {
dlist_ = std::make_unique<DList>(
ResourceUsage{MEMORY_LIMIT, DISK_LIMIT}, DList::TouchConfig{});
auto temp_translator_uptr = std::make_unique<MockTranslator>(
cell_sizes_, uid_to_cid_map_, SLOT_KEY, StorageType::MEMORY);
translator_ = temp_translator_uptr.get();
cache_slot_ = std::make_shared<CacheSlot<TestCell>>(
std::move(temp_translator_uptr), dlist_.get());
}
void
TearDown() override {
cache_slot_.reset();
dlist_.reset();
}
};
TEST_F(CacheSlotTest, Initialization) {
ASSERT_EQ(cache_slot_->num_cells(), NUM_UNIQUE_CIDS);
}
TEST_F(CacheSlotTest, PinSingleCellSuccess) {
cl_uid_t target_uid = 30;
cid_t expected_cid = 2;
ResourceUsage expected_size =
translator_->estimated_byte_size_of_cell(expected_cid);
translator_->ResetCounters();
auto future = cache_slot_->PinCells({target_uid});
auto accessor = SemiInlineGet(std::move(future));
ASSERT_NE(accessor, nullptr);
ASSERT_EQ(translator_->GetCellsCallCount(), 1);
ASSERT_EQ(translator_->GetRequestedCids().size(), 1);
ASSERT_EQ(translator_->GetRequestedCids()[0].size(), 1);
EXPECT_EQ(translator_->GetRequestedCids()[0][0], expected_cid);
EXPECT_EQ(DListTestFriend::get_used_memory(*dlist_), expected_size);
TestCell* cell = accessor->get_cell_of(target_uid);
ASSERT_NE(cell, nullptr);
EXPECT_EQ(cell->cid, expected_cid);
EXPECT_EQ(cell->data, expected_cid * 10);
TestCell* cell_by_index = accessor->get_ith_cell(expected_cid);
ASSERT_EQ(cell, cell_by_index);
}
TEST_F(CacheSlotTest, PinMultipleCellsSuccess) {
std::vector<cl_uid_t> target_uids = {10, 40, 51};
std::vector<cid_t> expected_cids = {0, 3, 4};
std::sort(expected_cids.begin(), expected_cids.end());
ResourceUsage expected_total_size;
for (cid_t cid : expected_cids) {
expected_total_size += translator_->estimated_byte_size_of_cell(cid);
}
translator_->ResetCounters();
auto future = cache_slot_->PinCells(target_uids);
auto accessor = SemiInlineGet(std::move(future));
ASSERT_NE(accessor, nullptr);
ASSERT_EQ(translator_->GetCellsCallCount(), 1);
ASSERT_EQ(translator_->GetRequestedCids().size(), 1);
auto requested = translator_->GetRequestedCids()[0];
std::sort(requested.begin(), requested.end());
ASSERT_EQ(requested.size(), expected_cids.size());
EXPECT_EQ(requested, expected_cids);
EXPECT_EQ(DListTestFriend::get_used_memory(*dlist_), expected_total_size);
for (cl_uid_t uid : target_uids) {
cid_t cid = uid_to_cid_map_.at(uid);
TestCell* cell = accessor->get_cell_of(uid);
ASSERT_NE(cell, nullptr);
EXPECT_EQ(cell->cid, cid);
EXPECT_EQ(cell->data, cid * 10);
}
}
TEST_F(CacheSlotTest, PinMultipleUidsMappingToSameCid) {
std::vector<cl_uid_t> target_uids = {30, 50, 31, 51, 32};
std::vector<cid_t> expected_unique_cids = {2, 4};
std::sort(expected_unique_cids.begin(), expected_unique_cids.end());
ResourceUsage expected_total_size;
for (cid_t cid : expected_unique_cids) {
expected_total_size += translator_->estimated_byte_size_of_cell(cid);
}
translator_->ResetCounters();
auto future = cache_slot_->PinCells(target_uids);
auto accessor = SemiInlineGet(std::move(future));
ASSERT_NE(accessor, nullptr);
ASSERT_EQ(translator_->GetCellsCallCount(), 1);
ASSERT_EQ(translator_->GetRequestedCids().size(), 1);
auto requested = translator_->GetRequestedCids()[0];
std::sort(requested.begin(), requested.end());
ASSERT_EQ(requested.size(), expected_unique_cids.size());
EXPECT_EQ(requested, expected_unique_cids);
EXPECT_EQ(DListTestFriend::get_used_memory(*dlist_), expected_total_size);
TestCell* cell2_uid30 = accessor->get_cell_of(30);
TestCell* cell2_uid31 = accessor->get_cell_of(31);
TestCell* cell4_uid50 = accessor->get_cell_of(50);
TestCell* cell4_uid51 = accessor->get_cell_of(51);
ASSERT_NE(cell2_uid30, nullptr);
ASSERT_NE(cell4_uid50, nullptr);
EXPECT_EQ(cell2_uid30->cid, 2);
EXPECT_EQ(cell4_uid50->cid, 4);
EXPECT_EQ(cell2_uid30, cell2_uid31);
EXPECT_EQ(cell4_uid50, cell4_uid51);
}
TEST_F(CacheSlotTest, PinInvalidUid) {
cl_uid_t invalid_uid = 999;
cl_uid_t valid_uid = 10;
std::vector<cl_uid_t> target_uids = {valid_uid, invalid_uid};
translator_->ResetCounters();
auto future = cache_slot_->PinCells(target_uids);
EXPECT_THROW(
{
try {
SemiInlineGet(std::move(future));
} catch (const std::invalid_argument& e) {
std::string error_what = e.what();
EXPECT_TRUE(error_what.find("out of range") !=
std::string::npos ||
error_what.find("invalid") != std::string::npos);
throw;
}
},
std::invalid_argument);
EXPECT_EQ(translator_->GetCellsCallCount(), 0);
}
TEST_F(CacheSlotTest, LoadFailure) {
cl_uid_t target_uid = 20;
cid_t expected_cid = 1;
translator_->ResetCounters();
translator_->SetShouldThrow(true);
auto future = cache_slot_->PinCells({target_uid});
EXPECT_THROW(
{
try {
SemiInlineGet(std::move(future));
} catch (const std::runtime_error& e) {
std::string error_what = e.what();
EXPECT_TRUE(error_what.find("Simulated load error") !=
std::string::npos ||
error_what.find("Failed to load") !=
std::string::npos ||
error_what.find("Exception during Future") !=
std::string::npos);
throw;
}
},
std::runtime_error);
ASSERT_EQ(translator_->GetCellsCallCount(), 1);
ASSERT_EQ(translator_->GetRequestedCids().size(), 1);
ASSERT_EQ(translator_->GetRequestedCids()[0].size(), 1);
EXPECT_EQ(translator_->GetRequestedCids()[0][0], expected_cid);
EXPECT_EQ(DListTestFriend::get_used_memory(*dlist_), ResourceUsage{});
}
TEST_F(CacheSlotTest, PinAlreadyLoadedCell) {
cl_uid_t target_uid = 40;
cid_t expected_cid = 3;
ResourceUsage expected_size =
translator_->estimated_byte_size_of_cell(expected_cid);
translator_->ResetCounters();
auto future1 = cache_slot_->PinCells({target_uid});
auto accessor1 = SemiInlineGet(std::move(future1));
ASSERT_NE(accessor1, nullptr);
ASSERT_EQ(translator_->GetCellsCallCount(), 1);
ASSERT_EQ(translator_->GetRequestedCids().size(), 1);
ASSERT_EQ(translator_->GetRequestedCids()[0][0], expected_cid);
EXPECT_EQ(DListTestFriend::get_used_memory(*dlist_), expected_size);
TestCell* cell1 = accessor1->get_cell_of(target_uid);
ASSERT_NE(cell1, nullptr);
translator_->ResetCounters();
auto future2 = cache_slot_->PinCells({target_uid});
auto accessor2 = SemiInlineGet(std::move(future2));
ASSERT_NE(accessor2, nullptr);
EXPECT_EQ(translator_->GetCellsCallCount(), 0);
EXPECT_EQ(DListTestFriend::get_used_memory(*dlist_), expected_size);
TestCell* cell2 = accessor2->get_cell_of(target_uid);
ASSERT_NE(cell2, nullptr);
EXPECT_EQ(cell1, cell2);
accessor1.reset();
EXPECT_EQ(DListTestFriend::get_used_memory(*dlist_), expected_size);
TestCell* cell_after_unpin = accessor2->get_cell_of(target_uid);
ASSERT_NE(cell_after_unpin, nullptr);
EXPECT_EQ(cell_after_unpin, cell2);
}
TEST_F(CacheSlotTest, PinAlreadyLoadedCellViaDifferentUid) {
cl_uid_t uid1 = 30;
cl_uid_t uid2 = 31;
cid_t expected_cid = 2;
ResourceUsage expected_size =
translator_->estimated_byte_size_of_cell(expected_cid);
translator_->ResetCounters();
auto future1 = cache_slot_->PinCells({uid1});
auto accessor1 = SemiInlineGet(std::move(future1));
ASSERT_NE(accessor1, nullptr);
ASSERT_EQ(translator_->GetCellsCallCount(), 1);
ASSERT_EQ(translator_->GetRequestedCids().size(), 1);
ASSERT_EQ(translator_->GetRequestedCids()[0][0], expected_cid);
EXPECT_EQ(DListTestFriend::get_used_memory(*dlist_), expected_size);
TestCell* cell1 = accessor1->get_cell_of(uid1);
ASSERT_NE(cell1, nullptr);
EXPECT_EQ(cell1->cid, expected_cid);
translator_->ResetCounters();
auto future2 = cache_slot_->PinCells({uid2});
auto accessor2 = SemiInlineGet(std::move(future2));
ASSERT_NE(accessor2, nullptr);
EXPECT_EQ(translator_->GetCellsCallCount(), 0);
EXPECT_EQ(DListTestFriend::get_used_memory(*dlist_), expected_size);
TestCell* cell2 = accessor2->get_cell_of(uid2);
ASSERT_NE(cell2, nullptr);
EXPECT_EQ(cell2->cid, expected_cid);
EXPECT_EQ(cell1, cell2);
accessor1.reset();
EXPECT_EQ(DListTestFriend::get_used_memory(*dlist_), expected_size);
TestCell* cell_after_unpin_uid1 = accessor2->get_cell_of(uid1);
TestCell* cell_after_unpin_uid2 = accessor2->get_cell_of(uid2);
ASSERT_NE(cell_after_unpin_uid1, nullptr);
ASSERT_NE(cell_after_unpin_uid2, nullptr);
EXPECT_EQ(cell_after_unpin_uid1, cell2);
EXPECT_EQ(cell_after_unpin_uid2, cell2);
}
TEST_F(CacheSlotTest, TranslatorReturnsExtraCells) {
cl_uid_t requested_uid = 10;
cid_t requested_cid = 0;
cid_t extra_cid = 1;
cl_uid_t extra_uid = 20;
ResourceUsage expected_size =
translator_->estimated_byte_size_of_cell(requested_cid) +
translator_->estimated_byte_size_of_cell(extra_cid);
translator_->ResetCounters();
translator_->SetExtraReturnCids({{requested_cid, {extra_cid}}});
auto future = cache_slot_->PinCells({requested_uid});
auto accessor = SemiInlineGet(std::move(future));
ASSERT_NE(accessor, nullptr);
ASSERT_EQ(translator_->GetCellsCallCount(), 1);
ASSERT_EQ(translator_->GetRequestedCids().size(), 1);
EXPECT_EQ(translator_->GetRequestedCids()[0],
std::vector<cid_t>{requested_cid});
EXPECT_EQ(DListTestFriend::get_used_memory(*dlist_), expected_size);
TestCell* requested_cell = accessor->get_cell_of(requested_uid);
ASSERT_NE(requested_cell, nullptr);
EXPECT_EQ(requested_cell->cid, requested_cid);
translator_->ResetCounters();
auto future_extra = cache_slot_->PinCells({extra_uid});
auto accessor_extra = SemiInlineGet(std::move(future_extra));
ASSERT_NE(accessor_extra, nullptr);
EXPECT_EQ(translator_->GetCellsCallCount(), 0);
EXPECT_EQ(DListTestFriend::get_used_memory(*dlist_), expected_size);
TestCell* extra_cell = accessor_extra->get_cell_of(extra_uid);
ASSERT_NE(extra_cell, nullptr);
EXPECT_EQ(extra_cell->cid, extra_cid);
}
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);
std::vector<cl_uid_t> uids_012 = {10, 20, 30};
std::vector<cid_t> cids_012 = {0, 1, 2};
ResourceUsage size_012 = translator_->estimated_byte_size_of_cell(0) +
translator_->estimated_byte_size_of_cell(1) +
translator_->estimated_byte_size_of_cell(2);
ASSERT_EQ(size_012, ResourceUsage(50 + 150 + 100, 0));
// 1. Load cells 0, 1, 2
translator_->ResetCounters();
auto future1 = cache_slot_->PinCells(uids_012);
auto accessor1 = SemiInlineGet(std::move(future1));
ASSERT_NE(accessor1, nullptr);
EXPECT_EQ(translator_->GetCellsCallCount(), 1);
auto requested1 = translator_->GetRequestedCids()[0];
std::sort(requested1.begin(), requested1.end());
EXPECT_EQ(requested1, cids_012);
EXPECT_EQ(DListTestFriend::get_used_memory(*dlist_), size_012);
// 2. Unpin 0, 1, 2
accessor1.reset();
EXPECT_EQ(DListTestFriend::get_used_memory(*dlist_),
size_012); // Still in cache
// 3. Load cell 3 (size 200), requires eviction
cl_uid_t uid_3 = 40;
cid_t cid_3 = 3;
ResourceUsage size_3 = translator_->estimated_byte_size_of_cell(cid_3);
ASSERT_EQ(size_3, ResourceUsage(200, 0));
translator_->ResetCounters();
auto future2 = cache_slot_->PinCells({uid_3});
auto accessor2 = SemiInlineGet(std::move(future2));
ASSERT_NE(accessor2, nullptr);
EXPECT_EQ(translator_->GetCellsCallCount(),
1); // Load was called for cell 3
ASSERT_EQ(translator_->GetRequestedCids().size(), 1);
EXPECT_EQ(translator_->GetRequestedCids()[0], std::vector<cid_t>{cid_3});
// Verify eviction happened
ResourceUsage used_after_evict1 = DListTestFriend::get_used_memory(*dlist_);
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,
size_012.memory_bytes + size_3.memory_bytes); // Eviction occurred
}
class CacheSlotConcurrentTest : public CacheSlotTest,
public ::testing::WithParamInterface<bool> {};
TEST_P(CacheSlotConcurrentTest, ConcurrentAccessMultipleSlots) {
// Slot 1 Cells: 0-4 (Sizes: 50, 60, 70, 80, 90) -> Total 350
// 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));
EXPECT_EQ(DListTestFriend::get_max_memory(*dlist_).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);
MockTranslator* translator_1 = translator_1_ptr.get();
auto slot1 = std::make_shared<CacheSlot<TestCell>>(
std::move(translator_1_ptr), dlist_.get());
std::vector<std::pair<cid_t, int64_t>> cell_sizes_2 = {
{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);
MockTranslator* translator_2 = translator_2_ptr.get();
auto slot2 = std::make_shared<CacheSlot<TestCell>>(
std::move(translator_2_ptr), dlist_.get());
bool with_bonus_cells = GetParam();
if (with_bonus_cells) {
// Configure translators to return bonus cells
std::unordered_map<cid_t, std::vector<cid_t>> bonus_map_1{
{0, {2}},
{1, {3}},
{2, {1, 4}},
{3, {0}},
{4, {2, 3}},
};
std::unordered_map<cid_t, std::vector<cid_t>> bonus_map_2{
{0, {1, 4}},
{1, {2, 3}},
{2, {0}},
{3, {0, 1}},
{4, {2}},
};
translator_1->SetExtraReturnCids(bonus_map_1);
translator_2->SetExtraReturnCids(bonus_map_2);
}
std::vector<std::shared_ptr<CacheSlot<TestCell>>> slots = {slot1, slot2};
// Store uid maps in a structure easily accessible by slot index
std::vector<std::vector<cl_uid_t>> slot_uids;
slot_uids.resize(slots.size());
std::vector<std::unordered_map<cl_uid_t, cid_t>> uid_to_cid_maps = {
uid_map_1, uid_map_2};
for (const auto& pair : uid_map_1) slot_uids[0].push_back(pair.first);
for (const auto& pair : uid_map_2) slot_uids[1].push_back(pair.first);
// 2. Setup Thread Pool and Concurrency Parameters
// at most 6 cells can be pinned at the same time, thus will never exceed the limit.
int num_threads = 6;
int ops_per_thread = 200;
// 1 extra thread to work with slot3
folly::CPUThreadPoolExecutor executor(num_threads + 1);
std::vector<folly::Future<folly::Unit>> futures;
std::atomic<bool> test_failed{false};
// 3. Launch Threads to Perform Concurrent Pin/Get/Verify/Unpin
for (int i = 0; i < num_threads; ++i) {
futures.push_back(folly::via(&executor, [&, i, tid = i]() {
// Seed random generator uniquely for each thread
std::mt19937 gen(
std::hash<std::thread::id>{}(std::this_thread::get_id()) + tid);
std::uniform_int_distribution<> slot_dist(0, slots.size() - 1);
std::uniform_int_distribution<> sleep_dist(5, 15);
for (int j = 0; j < ops_per_thread && !test_failed.load(); ++j) {
int slot_idx = slot_dist(gen);
auto& current_slot = slots[slot_idx];
auto& current_slot_uids = slot_uids[slot_idx];
auto& current_uid_to_cid_map = uid_to_cid_maps[slot_idx];
std::uniform_int_distribution<> uid_idx_dist(
0, current_slot_uids.size() - 1);
cl_uid_t target_uid = current_slot_uids[uid_idx_dist(gen)];
cid_t expected_cid = current_uid_to_cid_map.at(target_uid);
int expected_data = static_cast<int>(expected_cid * 10);
try {
auto accessor = current_slot->PinCells({target_uid}).get();
if (!accessor) {
ADD_FAILURE()
<< "T" << tid << " Op" << j
<< ": PinCells returned null accessor for UID "
<< target_uid;
test_failed = true;
break;
}
TestCell* cell = accessor->get_cell_of(target_uid);
if (!cell) {
ADD_FAILURE() << "T" << tid << " Op" << j
<< ": get_cell_of returned null for UID "
<< target_uid;
test_failed = true;
break;
}
if (cell->cid != expected_cid) {
ADD_FAILURE() << "T" << tid << " Op" << j
<< ": Incorrect CID for UID "
<< target_uid << ". Slot: " << slot_idx
<< ", Expected: " << expected_cid
<< ", Got: " << cell->cid;
test_failed = true;
break;
}
if (cell->data != expected_data) {
ADD_FAILURE() << "T" << tid << " Op" << j
<< ": Incorrect Data for UID "
<< target_uid << ". Slot: " << slot_idx
<< ", Expected: " << expected_data
<< ", Got: " << cell->data;
test_failed = true;
break;
}
int sleep_ms = sleep_dist(gen);
std::this_thread::sleep_for(
std::chrono::milliseconds(sleep_ms));
} catch (const std::exception& e) {
ADD_FAILURE()
<< "T" << tid << " Op" << j << ": Exception for UID "
<< target_uid << ", Slot: " << slot_idx
<< ". What: " << e.what();
test_failed = true;
} catch (...) {
ADD_FAILURE() << "T" << tid << " Op" << j
<< ": Unknown exception for UID "
<< target_uid << ", Slot: " << slot_idx;
test_failed = true;
}
}
}));
}
// number of ops between recreating slot3
const int recreate_interval = 25;
auto dlist_ptr = dlist_.get();
std::vector<std::pair<cid_t, int64_t>> cell_sizes_3 = {
{0, 40}, {1, 50}, {2, 60}, {3, 70}, {4, 80}};
std::unordered_map<cl_uid_t, cid_t> uid_map_3 = {
{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 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();
futures.push_back(folly::via(&executor, [&, tid = num_threads]() {
std::mt19937 gen(
std::hash<std::thread::id>{}(std::this_thread::get_id()) + tid);
std::uniform_int_distribution<> sleep_dist(5, 15);
std::uniform_int_distribution<> recreate_sleep_dist(20, 30);
std::uniform_int_distribution<> uid_idx_dist(0, slot3_uids.size() - 1);
int ops_since_recreate = 0;
for (int j = 0; j < ops_per_thread && !test_failed.load(); ++j) {
cl_uid_t target_uid = slot3_uids[uid_idx_dist(gen)];
cid_t expected_cid = uid_map_3.at(target_uid);
int expected_data = static_cast<int>(expected_cid * 10);
try {
auto accessor = slot3->PinCells({target_uid}).get();
if (!accessor) {
ADD_FAILURE()
<< "T" << tid << " Op" << j
<< ": PinCells returned null accessor for UID "
<< target_uid;
test_failed = true;
break;
}
TestCell* cell = accessor->get_cell_of(target_uid);
if (!cell) {
ADD_FAILURE()
<< "T" << tid << " Op" << j
<< ": get_cell_of returned null for UID " << target_uid;
test_failed = true;
break;
}
if (cell->cid != expected_cid) {
ADD_FAILURE() << "T" << tid << " Op" << j
<< ": Incorrect CID for UID " << target_uid
<< ". Slot: 3"
<< ", Expected: " << expected_cid
<< ", Got: " << cell->cid;
test_failed = true;
break;
}
if (cell->data != expected_data) {
ADD_FAILURE() << "T" << tid << " Op" << j
<< ": Incorrect Data for UID " << target_uid
<< ". Slot: 3"
<< ", Expected: " << expected_data
<< ", Got: " << cell->data;
test_failed = true;
break;
}
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(
std::chrono::milliseconds(sleep_ms));
slot3 = create_new_slot3();
ops_since_recreate = 0;
} else {
ops_since_recreate++;
int sleep_ms = sleep_dist(gen);
std::this_thread::sleep_for(
std::chrono::milliseconds(sleep_ms));
}
} catch (const std::exception& e) {
ADD_FAILURE()
<< "T" << tid << " Op" << j << ": Exception for UID "
<< target_uid << ", Slot: 3"
<< ". What: " << e.what();
test_failed = true;
} catch (...) {
ADD_FAILURE() << "T" << tid << " Op" << j
<< ": Unknown exception for UID " << target_uid
<< ", Slot: 3";
test_failed = true;
}
}
}));
// 4. Wait for all threads to complete
try {
folly::collectAll(futures).get();
} catch (const std::exception& e) {
FAIL() << "Exception waiting for thread pool completion: " << e.what();
} catch (...) {
FAIL() << "Unknown exception waiting for thread pool completion.";
}
ASSERT_FALSE(test_failed.load())
<< "Test failed due to assertion failures within threads.";
ResourceUsage final_memory_usage =
DListTestFriend::get_used_memory(*dlist_);
// 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)
<< "Final memory usage (" << final_memory_usage.memory_bytes
<< ") exceeds the limit (" << NEW_LIMIT.memory_bytes
<< ") after concurrent access.";
}
DListTestFriend::verify_integrity(dlist_.get());
}
INSTANTIATE_TEST_SUITE_P(BonusCellParam,
CacheSlotConcurrentTest,
::testing::Bool(),
[](const ::testing::TestParamInfo<bool>& info) {
return info.param ? "WithBonusCells"
: "NoBonusCells";
});

View File

@ -0,0 +1,610 @@
#include <gtest/gtest.h>
#include <gmock/gmock.h>
#include <thread>
#include <vector>
#include <memory>
#include "cachinglayer/lrucache/DList.h"
#include "cachinglayer/Utils.h"
#include "mock_list_node.h"
#include "cachinglayer_test_utils.h"
using namespace milvus::cachinglayer;
using namespace milvus::cachinglayer::internal;
using ::testing::StrictMock;
using DLF = DListTestFriend;
class DListTest : public ::testing::Test {
protected:
ResourceUsage initial_limit{100, 50};
DList::TouchConfig touch_config{{std::chrono::seconds(1)}};
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);
managed_nodes.clear();
}
void
TearDown() override {
managed_nodes.clear();
dlist.reset();
}
// Helper to create a mock node, simulate loading it, and add it to the list.
// Returns a raw pointer, but ownership is managed by the shared_ptr in managed_nodes.
MockListNode*
add_and_load_node(ResourceUsage size,
const std::string& key = "key",
cid_t cid = 0,
int pin_count = 0) {
// Check if adding this node would exceed capacity before creating/adding it.
// We want to use add_and_load_node to create a DList in valid state.
ResourceUsage current_usage = get_used_memory();
ResourceUsage limit = DLF::get_max_memory(*dlist);
if (!limit.CanHold(current_usage + size)) {
throw std::invalid_argument(
"Adding this node would exceed capacity");
}
auto node_ptr = std::make_shared<StrictMock<MockListNode>>(
dlist.get(), size, key, cid);
managed_nodes.push_back(node_ptr);
MockListNode* node = node_ptr.get();
node->test_set_state(ListNode::State::LOADED);
node->test_set_pin_count(pin_count);
// Manually adjust used memory and list pointers
DLF::test_add_used_memory(dlist.get(), size);
DLF::test_push_head(dlist.get(), node);
return node;
}
ResourceUsage
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) {
EXPECT_TRUE(dlist->IsEmpty());
EXPECT_EQ(get_used_memory(), ResourceUsage{});
EXPECT_EQ(DLF::get_head(*dlist), nullptr);
EXPECT_EQ(DLF::get_tail(*dlist), nullptr);
}
TEST_F(DListTest, UpdateLimitIncrease) {
MockListNode* node1 = add_and_load_node({10, 5});
EXPECT_EQ(get_used_memory(), node1->size());
ResourceUsage new_limit{200, 100};
EXPECT_TRUE(dlist->UpdateLimit(new_limit));
EXPECT_EQ(get_used_memory(), node1->size());
DLF::verify_list(dlist.get(), {node1});
}
TEST_F(DListTest, UpdateLimitDecreaseNoEviction) {
MockListNode* node1 = add_and_load_node({10, 5});
ResourceUsage current_usage = node1->size();
ASSERT_EQ(get_used_memory(), current_usage);
ResourceUsage new_limit{50, 25};
EXPECT_TRUE(dlist->UpdateLimit(new_limit));
EXPECT_EQ(get_used_memory(), current_usage);
DLF::verify_list(dlist.get(), {node1});
}
TEST_F(DListTest, UpdateLimitDecreaseWithEvictionLRU) {
MockListNode* node1 = add_and_load_node({50, 20}, "key1");
MockListNode* node2 = add_and_load_node({50, 30}, "key2");
ResourceUsage usage_node1 = node1->size();
ResourceUsage usage_node2 = node2->size();
DLF::verify_list(dlist.get(), {node1, node2});
EXPECT_EQ(get_used_memory(), usage_node1 + usage_node2);
EXPECT_EQ(get_used_memory(), DLF::get_max_memory(*dlist));
// Expect node1 to be evicted
EXPECT_CALL(*node1, clear_data()).Times(1);
EXPECT_CALL(*node2, clear_data()).Times(0);
ResourceUsage new_limit{70, 40};
EXPECT_TRUE(dlist->UpdateLimit(new_limit));
EXPECT_EQ(get_used_memory(), usage_node2);
DLF::verify_list(dlist.get(), {node2});
EXPECT_FALSE(dlist->IsEmpty());
}
TEST_F(DListTest, UpdateLimitDecreaseWithEvictionMultiple) {
MockListNode* node1 = add_and_load_node({30, 10}, "key1");
MockListNode* node2 = add_and_load_node({30, 10}, "key2");
MockListNode* node3 = add_and_load_node({30, 10}, "key3");
ResourceUsage usage_node1 = node1->size();
ResourceUsage usage_node2 = node2->size();
ResourceUsage usage_node3 = node3->size();
DLF::verify_list(dlist.get(), {node1, node2, node3});
ASSERT_EQ(get_used_memory(), usage_node1 + usage_node2 + usage_node3);
EXPECT_CALL(*node1, clear_data()).Times(1);
EXPECT_CALL(*node2, clear_data()).Times(1);
EXPECT_CALL(*node3, clear_data()).Times(0);
ResourceUsage new_limit{40, 15};
EXPECT_TRUE(dlist->UpdateLimit(new_limit));
EXPECT_EQ(get_used_memory(), usage_node3);
DLF::verify_list(dlist.get(), {node3});
}
TEST_F(DListTest, UpdateLimitSkipsPinned) {
MockListNode* node1 = add_and_load_node({40, 15}, "key1", 0, 1);
MockListNode* node2 = add_and_load_node({50, 25}, "key2");
ResourceUsage usage_node1 = node1->size();
ResourceUsage usage_node2 = node2->size();
DLF::verify_list(dlist.get(), {node1, node2});
ASSERT_EQ(get_used_memory(), usage_node1 + usage_node2);
EXPECT_CALL(*node1, clear_data()).Times(0);
EXPECT_CALL(*node2, clear_data()).Times(1);
ResourceUsage new_limit{70, 40};
EXPECT_TRUE(dlist->UpdateLimit(new_limit));
EXPECT_EQ(get_used_memory(), usage_node1);
DLF::verify_list(dlist.get(), {node1});
}
TEST_F(DListTest, UpdateLimitToZero) {
MockListNode* node1 = add_and_load_node({10, 0});
MockListNode* node2 = add_and_load_node({0, 5});
EXPECT_CALL(*node1, clear_data()).Times(1);
EXPECT_CALL(*node2, clear_data()).Times(1);
EXPECT_TRUE(dlist->UpdateLimit({1, 1}));
EXPECT_EQ(get_used_memory(), ResourceUsage{});
EXPECT_TRUE(dlist->IsEmpty());
}
TEST_F(DListTest, UpdateLimitInvalid) {
EXPECT_THROW(dlist->UpdateLimit({-10, 0}), std::invalid_argument);
EXPECT_THROW(dlist->UpdateLimit({0, -5}), std::invalid_argument);
}
TEST_F(DListTest, ReserveMemorySufficient) {
ResourceUsage size{20, 10};
EXPECT_TRUE(dlist->reserveMemory(size));
EXPECT_EQ(get_used_memory(), size);
}
TEST_F(DListTest, ReserveMemoryRequiresEviction) {
MockListNode* node1 = add_and_load_node({40, 15}, "key1");
MockListNode* node2 = add_and_load_node({50, 25}, "key2");
ResourceUsage usage_node1 = node1->size();
ResourceUsage usage_node2 = node2->size();
DLF::verify_list(dlist.get(), {node1, node2});
ASSERT_EQ(get_used_memory(), usage_node1 + usage_node2);
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.
// Evicting node1 ({40, 15}) is sufficient.
EXPECT_TRUE(dlist->reserveMemory(reserve_size));
EXPECT_EQ(get_used_memory(), usage_node2 + reserve_size);
DLF::verify_list(dlist.get(), {node2});
}
TEST_F(DListTest, ReserveMemoryEvictPinnedSkipped) {
MockListNode* node_pinned = add_and_load_node({40, 15}, "key_pinned", 0, 1);
MockListNode* node_evict = add_and_load_node({50, 25}, "key_evict");
ResourceUsage usage_pinned = node_pinned->size();
ResourceUsage usage_evict = node_evict->size();
DLF::verify_list(dlist.get(), {node_pinned, node_evict});
ASSERT_EQ(get_used_memory(), usage_pinned + usage_evict);
EXPECT_CALL(*node_pinned, clear_data()).Times(0);
EXPECT_CALL(*node_evict, clear_data()).Times(1);
ResourceUsage reserve_size{20, 20};
EXPECT_TRUE(dlist->reserveMemory(reserve_size));
EXPECT_EQ(get_used_memory(), usage_pinned + reserve_size);
DLF::verify_list(dlist.get(), {node_pinned});
}
TEST_F(DListTest, ReserveMemoryEvictLockedSkipped) {
MockListNode* node_locked = add_and_load_node({40, 15}, "key_locked");
MockListNode* node_evict = add_and_load_node({50, 25}, "key_evict");
ResourceUsage usage_locked = node_locked->size();
ResourceUsage usage_evict = node_evict->size();
DLF::verify_list(dlist.get(), {node_locked, node_evict});
ASSERT_EQ(get_used_memory(), usage_locked + usage_evict);
EXPECT_CALL(*node_locked, clear_data()).Times(0);
EXPECT_CALL(*node_evict, clear_data()).Times(1);
// Simulate locking the node during eviction attempt
std::unique_lock locked_node_lock(node_locked->test_get_mutex());
ResourceUsage reserve_size{20, 20};
EXPECT_TRUE(dlist->reserveMemory(reserve_size));
locked_node_lock.unlock();
EXPECT_EQ(get_used_memory(), usage_locked + reserve_size);
DLF::verify_list(dlist.get(), {node_locked});
}
TEST_F(DListTest, ReserveMemoryInsufficientEvenWithEviction) {
MockListNode* node1 = add_and_load_node({10, 5});
ResourceUsage usage_node1 = node1->size();
ASSERT_EQ(get_used_memory(), usage_node1);
ResourceUsage reserve_size{200, 100};
EXPECT_FALSE(dlist->reserveMemory(reserve_size));
EXPECT_EQ(get_used_memory(), usage_node1);
EXPECT_FALSE(dlist->IsEmpty());
}
TEST_F(DListTest, TouchItemMovesToHead) {
MockListNode* node1 = add_and_load_node({10, 0}, "key1");
MockListNode* node2 = add_and_load_node({10, 0}, "key2");
MockListNode* node3 = add_and_load_node({10, 0}, "key3");
DLF::verify_list(dlist.get(), {node1, node2, node3});
{
std::unique_lock node_lock(node1->test_get_mutex());
dlist->touchItem(node1);
}
DLF::verify_list(dlist.get(), {node2, node3, node1});
}
TEST_F(DListTest, TouchItemRefreshWindow) {
MockListNode* node1 = add_and_load_node({10, 0}, "key1");
MockListNode* node2 = add_and_load_node({10, 0}, "key2");
DLF::verify_list(dlist.get(), {node1, node2});
{
std::unique_lock node_lock(node1->test_get_mutex());
dlist->touchItem(node1);
}
DLF::verify_list(dlist.get(), {node2, node1});
{
std::unique_lock node_lock(node1->test_get_mutex());
dlist->touchItem(node1);
}
DLF::verify_list(dlist.get(), {node2, node1});
std::this_thread::sleep_for(touch_config.refresh_window +
std::chrono::milliseconds(100));
{
std::unique_lock node_lock(node1->test_get_mutex());
dlist->touchItem(node1);
}
DLF::verify_list(dlist.get(), {node2, node1});
std::this_thread::sleep_for(touch_config.refresh_window +
std::chrono::milliseconds(100));
{
std::unique_lock node_lock(node2->test_get_mutex());
dlist->touchItem(node2);
}
DLF::verify_list(dlist.get(), {node1, node2});
}
TEST_F(DListTest, releaseMemory) {
ResourceUsage initial_size{30, 15};
DLF::test_add_used_memory(dlist.get(), initial_size);
ASSERT_EQ(get_used_memory(), initial_size);
ResourceUsage failed_load_size{10, 5};
dlist->releaseMemory(failed_load_size);
EXPECT_EQ(get_used_memory(), initial_size - failed_load_size);
}
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");
ResourceUsage usage_disk = node_disk_only->size();
ResourceUsage usage_mixed = node_mixed->size();
DLF::verify_list(dlist.get(), {node_disk_only, node_mixed});
ASSERT_EQ(get_used_memory(), usage_disk + usage_mixed);
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.
ResourceUsage reserve_size{60, 0};
EXPECT_TRUE(dlist->reserveMemory(reserve_size));
EXPECT_EQ(get_used_memory(), usage_disk + reserve_size);
DLF::verify_list(dlist.get(), {node_disk_only});
}
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();
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);
EXPECT_CALL(*node_mem_only, clear_data()).Times(0);
EXPECT_CALL(*node_mixed, clear_data()).Times(1);
ResourceUsage reserve_size{0, 60};
EXPECT_TRUE(dlist->reserveMemory(reserve_size));
EXPECT_EQ(get_used_memory(), usage_mem + reserve_size);
DLF::verify_list(dlist.get(), {node_mem_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");
ResourceUsage usage1 = node1->size();
ResourceUsage usage2 = node2->size();
ResourceUsage usage3 = node3->size();
DLF::verify_list(dlist.get(), {node1, node2, node3});
ASSERT_EQ(get_used_memory(), usage1 + usage2 + usage3);
EXPECT_CALL(*node1, clear_data()).Times(1);
EXPECT_CALL(*node2, clear_data()).Times(1);
EXPECT_CALL(*node3, clear_data()).Times(0);
ResourceUsage reserve_size{50, 50};
EXPECT_TRUE(dlist->reserveMemory(reserve_size));
EXPECT_EQ(get_used_memory(), usage3 + reserve_size);
DLF::verify_list(dlist.get(), {node3});
}
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);
ResourceUsage usage_node1 = node1->size();
ResourceUsage usage_node2 = node2->size();
DLF::verify_list(dlist.get(), {node1, node2});
ASSERT_EQ(get_used_memory(), usage_node1 + usage_node2);
EXPECT_CALL(*node1, clear_data()).Times(0);
EXPECT_CALL(*node2, clear_data()).Times(0);
ResourceUsage reserve_size{20, 20};
EXPECT_FALSE(dlist->reserveMemory(reserve_size));
EXPECT_EQ(get_used_memory(), usage_node1 + usage_node2);
DLF::verify_list(dlist.get(), {node1, node2});
}
TEST_F(DListTest, ReserveMemoryFailsAllLocked) {
MockListNode* node1 = add_and_load_node({40, 15}, "key1");
MockListNode* node2 = add_and_load_node({50, 25}, "key2");
ResourceUsage usage_node1 = node1->size();
ResourceUsage usage_node2 = node2->size();
DLF::verify_list(dlist.get(), {node1, node2});
ASSERT_EQ(get_used_memory(), usage_node1 + usage_node2);
std::unique_lock lock1(node1->test_get_mutex());
std::unique_lock lock2(node2->test_get_mutex());
EXPECT_CALL(*node1, clear_data()).Times(0);
EXPECT_CALL(*node2, clear_data()).Times(0);
ResourceUsage reserve_size{20, 20};
EXPECT_FALSE(dlist->reserveMemory(reserve_size));
lock1.unlock();
lock2.unlock();
EXPECT_EQ(get_used_memory(), usage_node1 + usage_node2);
DLF::verify_list(dlist.get(), {node1, node2});
}
TEST_F(DListTest, ReserveMemoryFailsSpecificPinned) {
MockListNode* node_evict =
add_and_load_node({80, 40}, "evict_candidate", 0, 1);
MockListNode* node_small = add_and_load_node({10, 5}, "small");
ResourceUsage usage_evict = node_evict->size();
ResourceUsage usage_small = node_small->size();
DLF::verify_list(dlist.get(), {node_evict, node_small});
ASSERT_EQ(get_used_memory(), usage_evict + usage_small);
EXPECT_CALL(*node_evict, clear_data()).Times(0);
EXPECT_CALL(*node_small, clear_data()).Times(0);
ResourceUsage reserve_size{20, 20};
EXPECT_FALSE(dlist->reserveMemory(reserve_size));
EXPECT_EQ(get_used_memory(), usage_evict + usage_small);
DLF::verify_list(dlist.get(), {node_evict, node_small});
}
TEST_F(DListTest, ReserveMemoryFailsSpecificLocked) {
MockListNode* node_evict = add_and_load_node({80, 40}, "evict_candidate");
MockListNode* node_small = add_and_load_node({10, 5}, "small");
ResourceUsage usage_evict = node_evict->size();
ResourceUsage usage_small = node_small->size();
DLF::verify_list(dlist.get(), {node_evict, node_small});
ASSERT_EQ(get_used_memory(), usage_evict + usage_small);
std::unique_lock lock_evict(node_evict->test_get_mutex());
EXPECT_CALL(*node_evict, clear_data()).Times(0);
EXPECT_CALL(*node_small, clear_data()).Times(0);
ResourceUsage reserve_size{20, 20};
EXPECT_FALSE(dlist->reserveMemory(reserve_size));
lock_evict.unlock();
EXPECT_EQ(get_used_memory(), usage_evict + usage_small);
DLF::verify_list(dlist.get(), {node_evict, node_small});
}
TEST_F(DListTest, TouchItemHeadOutsideWindow) {
MockListNode* node1 = add_and_load_node({10, 0}, "key1");
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));
{
std::unique_lock node_lock(node2->test_get_mutex());
dlist->touchItem(node2);
}
DLF::verify_list(dlist.get(), {node1, node2});
}
TEST_F(DListTest, RemoveItemFromList) {
MockListNode* node1 = add_and_load_node({10, 0}, "key1");
MockListNode* node2 = add_and_load_node({10, 0}, "key2");
DLF::verify_list(dlist.get(), {node1, node2});
{
std::unique_lock node_lock(node1->test_get_mutex());
dlist->removeItem(node1, node1->size());
}
DLF::verify_list(dlist.get(), {node2});
EXPECT_EQ(get_used_memory(), node2->size());
}
TEST_F(DListTest, PopItemNotPresent) {
MockListNode* node1 = add_and_load_node({10, 0}, "key1");
MockListNode* node2 = add_and_load_node({10, 0}, "key2");
ResourceUsage initial_usage = get_used_memory();
DLF::verify_list(dlist.get(), {node1, node2});
auto orphan_node_ptr = std::make_unique<StrictMock<MockListNode>>(
dlist.get(), ResourceUsage{10, 0}, "orphan", 0);
MockListNode* orphan_node = orphan_node_ptr.get();
{
std::unique_lock node_lock(orphan_node->test_get_mutex());
EXPECT_NO_THROW(DLF::test_pop_item(dlist.get(), orphan_node));
}
DLF::verify_list(dlist.get(), {node1, node2});
EXPECT_EQ(get_used_memory(), initial_usage);
}
TEST_F(DListTest, UpdateLimitIncreaseMemDecreaseDisk) {
MockListNode* node1 = add_and_load_node({20, 30}, "node1");
MockListNode* node2 = add_and_load_node({30, 10}, "node2");
ResourceUsage usage1 = node1->size();
ResourceUsage usage2 = node2->size();
DLF::verify_list(dlist.get(), {node1, node2});
ASSERT_EQ(get_used_memory(), usage1 + usage2);
EXPECT_CALL(*node1, clear_data()).Times(1);
EXPECT_CALL(*node2, clear_data()).Times(0);
ResourceUsage new_limit{200, 35};
EXPECT_TRUE(dlist->UpdateLimit(new_limit));
EXPECT_EQ(get_used_memory(), usage2);
DLF::verify_list(dlist.get(), {node2});
EXPECT_EQ(DLF::get_max_memory(*dlist), new_limit);
}
TEST_F(DListTest, EvictedNodeDestroyed) {
MockListNode* node1 = add_and_load_node({40, 15}, "node1");
MockListNode* node2 = add_and_load_node({50, 25}, "node2");
ResourceUsage usage1 = node1->size();
ResourceUsage usage2 = node2->size();
DLF::verify_list(dlist.get(), {node1, node2});
ASSERT_EQ(managed_nodes.size(), 2);
ASSERT_EQ(get_used_memory(), usage1 + usage2);
EXPECT_CALL(*node1, clear_data()).Times(1);
EXPECT_CALL(*node2, clear_data()).Times(0);
ResourceUsage new_limit{70, 40};
EXPECT_TRUE(dlist->UpdateLimit(new_limit));
DLF::verify_list(dlist.get(), {node2});
ResourceUsage memory_after_eviction = get_used_memory();
ASSERT_EQ(memory_after_eviction, usage2);
// destroy node1 by removing its shared_ptr
// node1's destructor should not decrement used_memory_ again
auto it = std::find_if(managed_nodes.begin(),
managed_nodes.end(),
[&](const auto& ptr) { return ptr.get() == node1; });
ASSERT_NE(it, managed_nodes.end());
managed_nodes.erase(it);
EXPECT_EQ(get_used_memory(), memory_after_eviction);
DLF::verify_list(dlist.get(), {node2});
}
TEST_F(DListTest, NodeInListDestroyed) {
MockListNode* node1 = add_and_load_node({40, 15}, "node1");
MockListNode* node2 = add_and_load_node({50, 25}, "node2");
ResourceUsage usage1 = node1->size();
ResourceUsage usage2 = node2->size();
DLF::verify_list(dlist.get(), {node1, node2});
ASSERT_EQ(managed_nodes.size(), 2);
ResourceUsage memory_before_destroy = get_used_memory();
ASSERT_EQ(memory_before_destroy, usage1 + usage2);
// destroy node1 by removing its shared_ptr
// node1's destructor should decrement used_memory_ by node1->size() and remove node1 from the list
auto it = std::find_if(managed_nodes.begin(),
managed_nodes.end(),
[&](const auto& ptr) { return ptr.get() == node1; });
ASSERT_NE(it, managed_nodes.end());
managed_nodes.erase(it);
EXPECT_EQ(get_used_memory(), memory_before_destroy - usage1);
DLF::verify_list(dlist.get(), {node2});
}

View File

@ -67,7 +67,8 @@ TEST(chunk, test_int64_field) {
std::nullopt);
arrow::ArrayVector array_vec = read_single_column_batches(rb_reader);
auto chunk = create_chunk(field_meta, 1, array_vec);
auto span = std::dynamic_pointer_cast<FixedWidthChunk>(chunk)->Span();
auto fixed_chunk = static_cast<FixedWidthChunk*>(chunk.get());
auto span = fixed_chunk->Span();
EXPECT_EQ(span.row_count(), data.size());
for (size_t i = 0; i < data.size(); ++i) {
auto n = *(int64_t*)((char*)span.data() + i * span.element_sizeof());
@ -109,8 +110,8 @@ TEST(chunk, test_variable_field) {
std::nullopt);
arrow::ArrayVector array_vec = read_single_column_batches(rb_reader);
auto chunk = create_chunk(field_meta, 1, array_vec);
auto views = std::dynamic_pointer_cast<StringChunk>(chunk)->StringViews(
std::nullopt);
auto string_chunk = static_cast<StringChunk*>(chunk.get());
auto views = string_chunk->StringViews(std::nullopt);
for (size_t i = 0; i < data.size(); ++i) {
EXPECT_EQ(views.first[i], data[i]);
}
@ -154,8 +155,8 @@ TEST(chunk, test_variable_field_nullable) {
std::nullopt);
arrow::ArrayVector array_vec = read_single_column_batches(rb_reader);
auto chunk = create_chunk(field_meta, 1, array_vec);
auto views = std::dynamic_pointer_cast<StringChunk>(chunk)->StringViews(
std::nullopt);
auto string_chunk = static_cast<StringChunk*>(chunk.get());
auto views = string_chunk->StringViews(std::nullopt);
for (size_t i = 0; i < data.size(); ++i) {
EXPECT_EQ(views.second[i], validity[i]);
if (validity[i]) {
@ -211,10 +212,9 @@ TEST(chunk, test_json_field) {
std::nullopt);
arrow::ArrayVector array_vec = read_single_column_batches(rb_reader);
auto chunk = create_chunk(field_meta, 1, array_vec);
auto json_chunk = static_cast<JSONChunk*>(chunk.get());
{
auto [views, valid] =
std::dynamic_pointer_cast<JSONChunk>(chunk)->StringViews(
std::nullopt);
auto [views, valid] = json_chunk->StringViews(std::nullopt);
EXPECT_EQ(row_num, views.size());
for (size_t i = 0; i < row_num; ++i) {
EXPECT_EQ(views[i], data[i].data());
@ -225,8 +225,7 @@ TEST(chunk, test_json_field) {
auto start = 10;
auto len = 20;
auto [views, valid] =
std::dynamic_pointer_cast<JSONChunk>(chunk)->StringViews(
std::make_pair(start, len));
json_chunk->StringViews(std::make_pair(start, len));
EXPECT_EQ(len, views.size());
for (size_t i = 0; i < len; ++i) {
EXPECT_EQ(views[i], data[i].data());
@ -243,10 +242,9 @@ TEST(chunk, test_json_field) {
std::nullopt);
arrow::ArrayVector array_vec = read_single_column_batches(rb_reader);
auto chunk = create_chunk(field_meta, 1, array_vec);
auto json_chunk = static_cast<JSONChunk*>(chunk.get());
{
auto [views, valid] =
std::dynamic_pointer_cast<JSONChunk>(chunk)->StringViews(
std::nullopt);
auto [views, valid] = json_chunk->StringViews(std::nullopt);
EXPECT_EQ(row_num, views.size());
for (size_t i = 0; i < row_num; ++i) {
EXPECT_EQ(views[i], data[i].data());
@ -257,8 +255,7 @@ TEST(chunk, test_json_field) {
auto start = 10;
auto len = 20;
auto [views, valid] =
std::dynamic_pointer_cast<JSONChunk>(chunk)->StringViews(
std::make_pair(start, len));
json_chunk->StringViews(std::make_pair(start, len));
EXPECT_EQ(len, views.size());
for (size_t i = 0; i < len; ++i) {
EXPECT_EQ(views[i], data[i].data());
@ -268,26 +265,20 @@ TEST(chunk, test_json_field) {
{
auto start = -1;
auto len = 5;
EXPECT_THROW(
std::dynamic_pointer_cast<JSONChunk>(chunk)->StringViews(
std::make_pair(start, len)),
milvus::SegcoreError);
EXPECT_THROW(json_chunk->StringViews(std::make_pair(start, len)),
milvus::SegcoreError);
}
{
auto start = 0;
auto len = row_num + 1;
EXPECT_THROW(
std::dynamic_pointer_cast<JSONChunk>(chunk)->StringViews(
std::make_pair(start, len)),
milvus::SegcoreError);
EXPECT_THROW(json_chunk->StringViews(std::make_pair(start, len)),
milvus::SegcoreError);
}
{
auto start = 95;
auto len = 11;
EXPECT_THROW(
std::dynamic_pointer_cast<JSONChunk>(chunk)->StringViews(
std::make_pair(start, len)),
milvus::SegcoreError);
EXPECT_THROW(json_chunk->StringViews(std::make_pair(start, len)),
milvus::SegcoreError);
}
}
}
@ -329,7 +320,7 @@ TEST(chunk, test_null_int64) {
std::nullopt);
arrow::ArrayVector array_vec = read_single_column_batches(rb_reader);
auto chunk = create_chunk(field_meta, 1, array_vec);
auto fixed_chunk = std::dynamic_pointer_cast<FixedWidthChunk>(chunk);
auto fixed_chunk = static_cast<FixedWidthChunk*>(chunk.get());
auto span = fixed_chunk->Span();
EXPECT_EQ(span.row_count(), data.size());
@ -390,8 +381,8 @@ TEST(chunk, test_array) {
std::nullopt);
arrow::ArrayVector array_vec = read_single_column_batches(rb_reader);
auto chunk = create_chunk(field_meta, 1, array_vec);
auto [views, valid] =
std::dynamic_pointer_cast<ArrayChunk>(chunk)->Views(std::nullopt);
auto array_chunk = static_cast<ArrayChunk*>(chunk.get());
auto [views, valid] = array_chunk->Views(std::nullopt);
EXPECT_EQ(views.size(), 1);
auto& arr = views[0];
for (size_t i = 0; i < arr.length(); ++i) {
@ -453,8 +444,8 @@ TEST(chunk, test_null_array) {
std::nullopt);
arrow::ArrayVector array_vec = read_single_column_batches(rb_reader);
auto chunk = create_chunk(field_meta, 1, array_vec);
auto [views, valid] =
std::dynamic_pointer_cast<ArrayChunk>(chunk)->Views(std::nullopt);
auto array_chunk = static_cast<ArrayChunk*>(chunk.get());
auto [views, valid] = array_chunk->Views(std::nullopt);
EXPECT_EQ(views.size(), array_count);
EXPECT_EQ(valid.size(), array_count);
@ -527,10 +518,9 @@ TEST(chunk, test_array_views) {
std::nullopt);
arrow::ArrayVector array_vec = read_single_column_batches(rb_reader);
auto chunk = create_chunk(field_meta, 1, array_vec);
auto array_chunk = static_cast<ArrayChunk*>(chunk.get());
{
auto [views, valid] =
std::dynamic_pointer_cast<ArrayChunk>(chunk)->Views(std::nullopt);
auto [views, valid] = array_chunk->Views(std::nullopt);
EXPECT_EQ(views.size(), array_count);
for (auto i = 0; i < array_count; i++) {
auto& arr = views[i];
@ -543,9 +533,7 @@ TEST(chunk, test_array_views) {
{
auto start = 2;
auto len = 5;
auto [views, valid] =
std::dynamic_pointer_cast<ArrayChunk>(chunk)->Views(
std::make_pair(start, len));
auto [views, valid] = array_chunk->Views(std::make_pair(start, len));
EXPECT_EQ(views.size(), len);
for (auto i = 0; i < len; i++) {
auto& arr = views[i];
@ -558,22 +546,19 @@ TEST(chunk, test_array_views) {
{
auto start = -1;
auto len = 5;
EXPECT_THROW(std::dynamic_pointer_cast<ArrayChunk>(chunk)->Views(
std::make_pair(start, len)),
EXPECT_THROW(array_chunk->Views(std::make_pair(start, len)),
milvus::SegcoreError);
}
{
auto start = 0;
auto len = array_count + 1;
EXPECT_THROW(std::dynamic_pointer_cast<ArrayChunk>(chunk)->Views(
std::make_pair(start, len)),
EXPECT_THROW(array_chunk->Views(std::make_pair(start, len)),
milvus::SegcoreError);
}
{
auto start = 5;
auto len = 7;
EXPECT_THROW(std::dynamic_pointer_cast<ArrayChunk>(chunk)->Views(
std::make_pair(start, len)),
EXPECT_THROW(array_chunk->Views(std::make_pair(start, len)),
milvus::SegcoreError);
}
}
@ -615,7 +600,8 @@ TEST(chunk, test_sparse_float) {
std::nullopt);
arrow::ArrayVector array_vec = read_single_column_batches(rb_reader);
auto chunk = create_chunk(field_meta, kTestSparseDim, array_vec);
auto vec = std::dynamic_pointer_cast<SparseFloatVectorChunk>(chunk)->Vec();
auto vec_chunk = static_cast<SparseFloatVectorChunk*>(chunk.get());
auto vec = vec_chunk->Vec();
for (size_t i = 0; i < n_rows; ++i) {
auto v1 = vec[i];
auto v2 = vecs[i];

View File

@ -1,251 +0,0 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#include <gtest/gtest.h>
#include <iostream>
#include <optional>
#include <string>
#include <utility>
#include <vector>
#include "common/Consts.h"
#include "common/FieldMeta.h"
#include "common/Types.h"
#include "fmt/format.h"
#include "common/Schema.h"
#include "gtest/gtest.h"
#include "knowhere/sparse_utils.h"
#include "mmap/Column.h"
#include "test_utils/DataGen.h"
#include "test_utils/storage_test_utils.h"
#include "storage/ChunkCache.h"
#include "storage/LocalChunkManagerSingleton.h"
#define DEFAULT_READ_AHEAD_POLICY "willneed"
class ChunkCacheTest : public testing::TestWithParam<bool> {
protected:
void
SetUp() override {
mcm = milvus::storage::MmapManager::GetInstance().GetMmapChunkManager();
mcm->Register(descriptor);
N = 10000;
dim = 128;
auto dense_metric_type = knowhere::metric::L2;
auto sparse_metric_type = knowhere::metric::IP;
auto schema = std::make_shared<milvus::Schema>();
auto fake_dense_vec_id = schema->AddDebugField(
"fakevec", milvus::DataType::VECTOR_FLOAT, dim, dense_metric_type);
auto i64_fid =
schema->AddDebugField("counter", milvus::DataType::INT64);
auto fake_sparse_vec_id =
schema->AddDebugField("fakevec_sparse",
milvus::DataType::VECTOR_SPARSE_FLOAT,
dim,
sparse_metric_type);
schema->set_primary_field_id(i64_fid);
auto dataset = milvus::segcore::DataGen(schema, N);
auto dense_field_data_meta =
milvus::storage::FieldDataMeta{1, 2, 3, fake_dense_vec_id.get()};
auto sparse_field_data_meta =
milvus::storage::FieldDataMeta{1, 2, 3, fake_sparse_vec_id.get()};
dense_field_meta = milvus::FieldMeta(milvus::FieldName("fakevec"),
fake_dense_vec_id,
milvus::DataType::VECTOR_FLOAT,
dim,
dense_metric_type,
false,
std::nullopt);
sparse_field_meta =
milvus::FieldMeta(milvus::FieldName("fakevec_sparse"),
fake_sparse_vec_id,
milvus::DataType::VECTOR_SPARSE_FLOAT,
dim,
sparse_metric_type,
false,
std::nullopt);
lcm = milvus::storage::LocalChunkManagerSingleton::GetInstance()
.GetChunkManager();
dense_data = dataset.get_col<float>(fake_dense_vec_id);
sparse_data = dataset.get_col<knowhere::sparse::SparseRow<float>>(
fake_sparse_vec_id);
auto data_slices = std::vector<void*>{dense_data.data()};
auto slice_sizes = std::vector<int64_t>{static_cast<int64_t>(N)};
auto slice_names = std::vector<std::string>{dense_file_name};
PutFieldData(lcm.get(),
data_slices,
slice_sizes,
slice_names,
dense_field_data_meta,
dense_field_meta);
data_slices = std::vector<void*>{sparse_data.data()};
slice_sizes = std::vector<int64_t>{static_cast<int64_t>(N)};
slice_names = std::vector<std::string>{sparse_file_name};
PutFieldData(lcm.get(),
data_slices,
slice_sizes,
slice_names,
sparse_field_data_meta,
sparse_field_meta);
}
void
TearDown() override {
mcm->UnRegister(descriptor);
}
const char* dense_file_name = "chunk_cache_test/insert_log/2/101/1000000";
const char* sparse_file_name = "chunk_cache_test/insert_log/2/102/1000000";
milvus::storage::MmapChunkManagerPtr mcm;
milvus::segcore::SegcoreConfig config;
milvus::storage::MmapChunkDescriptorPtr descriptor =
std::shared_ptr<milvus::storage::MmapChunkDescriptor>(
new milvus::storage::MmapChunkDescriptor(
{101, SegmentType::Sealed}));
int N;
int dim;
milvus::FieldMeta dense_field_meta = milvus::FieldMeta::RowIdMeta;
milvus::FixedVector<float> dense_data;
milvus::FieldMeta sparse_field_meta = milvus::FieldMeta::RowIdMeta;
milvus::FixedVector<knowhere::sparse::SparseRow<float>> sparse_data;
std::shared_ptr<milvus::storage::LocalChunkManager> lcm;
};
INSTANTIATE_TEST_SUITE_P(ChunkCacheTestSuite, ChunkCacheTest, testing::Bool());
TEST_P(ChunkCacheTest, Read) {
auto cc = milvus::storage::MmapManager::GetInstance().GetChunkCache();
// validate dense data
std::shared_ptr<milvus::ColumnBase> dense_column;
auto mmap_enabled = GetParam();
dense_column = cc->Read(dense_file_name, dense_field_meta, mmap_enabled);
auto actual_dense = (const float*)(dense_column->Data(0));
for (auto i = 0; i < N * dim; i++) {
AssertInfo(dense_data[i] == actual_dense[i],
fmt::format(
"expect {}, actual {}", dense_data[i], actual_dense[i]));
}
// validate sparse data
std::shared_ptr<milvus::ColumnBase> sparse_column;
sparse_column = cc->Read(sparse_file_name, sparse_field_meta, mmap_enabled);
auto expected_sparse_size = 0;
auto actual_sparse =
(const knowhere::sparse::SparseRow<float>*)(sparse_column->Data(0));
for (auto i = 0; i < N; i++) {
const auto& actual_sparse_row = actual_sparse[i];
const auto& expect_sparse_row = sparse_data[i];
AssertInfo(
actual_sparse_row.size() == expect_sparse_row.size(),
fmt::format("Incorrect size of sparse row: expect {}, actual {}",
expect_sparse_row.size(),
actual_sparse_row.size()));
auto bytes = actual_sparse_row.data_byte_size();
AssertInfo(
memcmp(actual_sparse_row.data(), expect_sparse_row.data(), bytes) ==
0,
fmt::format("Incorrect data of sparse row: expect {}, actual {}",
expect_sparse_row.data(),
actual_sparse_row.data()));
expected_sparse_size += bytes;
}
expected_sparse_size += (N + 7) / 8;
expected_sparse_size += sizeof(int64_t) * (N + 1);
if (mmap_enabled) {
const uint32_t page_size = sysconf(_SC_PAGE_SIZE);
auto padding_size = (expected_sparse_size / page_size +
(expected_sparse_size % page_size != 0)) *
page_size -
expected_sparse_size;
expected_sparse_size += padding_size;
}
auto actual_sparse_size = sparse_column->DataByteSize();
Assert(actual_sparse_size == expected_sparse_size);
cc->Remove(dense_file_name);
cc->Remove(sparse_file_name);
lcm->Remove(dense_file_name);
lcm->Remove(sparse_file_name);
}
TEST_P(ChunkCacheTest, TestMultithreads) {
auto cc = milvus::storage::MmapManager::GetInstance().GetChunkCache();
constexpr int threads = 16;
std::vector<int64_t> total_counts(threads);
auto mmap_enabled = GetParam();
auto executor = [&](int thread_id) {
std::shared_ptr<milvus::ColumnBase> dense_column;
dense_column =
cc->Read(dense_file_name, dense_field_meta, mmap_enabled);
auto actual_dense = (const float*)dense_column->Data(0);
for (auto i = 0; i < N * dim; i++) {
AssertInfo(
dense_data[i] == actual_dense[i],
fmt::format(
"expect {}, actual {}", dense_data[i], actual_dense[i]));
}
std::shared_ptr<milvus::ColumnBase> sparse_column;
sparse_column =
cc->Read(sparse_file_name, sparse_field_meta, mmap_enabled);
auto actual_sparse =
(const knowhere::sparse::SparseRow<float>*)sparse_column->Data(0);
for (auto i = 0; i < N; i++) {
const auto& actual_sparse_row = actual_sparse[i];
const auto& expect_sparse_row = sparse_data[i];
AssertInfo(actual_sparse_row.size() == expect_sparse_row.size(),
fmt::format(
"Incorrect size of sparse row: expect {}, actual {}",
expect_sparse_row.size(),
actual_sparse_row.size()));
auto bytes = actual_sparse_row.data_byte_size();
AssertInfo(memcmp(actual_sparse_row.data(),
expect_sparse_row.data(),
bytes) == 0,
fmt::format(
"Incorrect data of sparse row: expect {}, actual {}",
expect_sparse_row.data(),
actual_sparse_row.data()));
}
};
std::vector<std::thread> pool;
for (int i = 0; i < threads; ++i) {
pool.emplace_back(executor, i);
}
for (auto& thread : pool) {
thread.join();
}
cc->Remove(dense_file_name);
cc->Remove(sparse_file_name);
lcm->Remove(dense_file_name);
lcm->Remove(sparse_file_name);
}

View File

@ -9,22 +9,33 @@
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#include <cachinglayer/Translator.h>
#include "common/Chunk.h"
#include "gtest/gtest.h"
#include "mmap/ChunkedColumn.h"
#include "test_cachinglayer/cachinglayer_test_utils.h"
namespace milvus {
TEST(test_chunked_column, test_get_chunkid) {
ChunkedColumn column;
std::vector<size_t> chunk_row_nums = {10, 20, 30};
for (auto row_num : chunk_row_nums) {
std::vector<int64_t> num_rows_per_chunk = {0, 10, 20, 30};
auto num_chunks = num_rows_per_chunk.size();
std::vector<std::unique_ptr<Chunk>> chunks;
for (auto i = 0; i < num_chunks; ++i) {
auto row_num = num_rows_per_chunk[i];
auto chunk =
std::make_shared<FixedWidthChunk>(row_num, 1, nullptr, 0, 4, false);
column.AddChunk(chunk);
std::make_unique<FixedWidthChunk>(row_num, 1, nullptr, 0, 4, false);
chunks.push_back(std::move(chunk));
}
auto translator = std::make_unique<TestChunkTranslator>(
num_rows_per_chunk, "test", std::move(chunks));
FieldMeta field_meta(
FieldName("test"), FieldId(1), DataType::INT64, false, std::nullopt);
ChunkedColumn column(std::move(translator), field_meta);
int offset = 0;
for (int i = 0; i < chunk_row_nums.size(); ++i) {
for (int j = 0; j < chunk_row_nums[i]; ++j) {
for (int i = 0; i < num_chunks; ++i) {
for (int j = 0; j < num_rows_per_chunk[i]; ++j) {
auto [chunk_id, offset_in_chunk] =
column.GetChunkIDByOffset(offset);
ASSERT_EQ(chunk_id, i);
@ -33,4 +44,4 @@ TEST(test_chunked_column, test_get_chunkid) {
}
}
}
} // namespace milvus
} // namespace milvus

Some files were not shown because too many files have changed in this diff Show More