[Refactor] Remove some names related to ObjectCache (#63768)
Signed-off-by: trueeyu <lxhhust350@qq.com>
This commit is contained in:
parent
bf8516dc7b
commit
35e984856e
|
|
@ -146,7 +146,7 @@ void ObjectCacheBench::prepare_sequence_data(StoragePageCache* cache, int64_t co
|
|||
auto* ptr = new std::vector<uint8_t>(_page_size);
|
||||
(*ptr)[0] = 1;
|
||||
PageCacheHandle handle;
|
||||
ObjectCacheWriteOptions options;
|
||||
MemCacheWriteOptions options;
|
||||
Status st = cache->insert(key, ptr, options, &handle);
|
||||
if (!st.ok()) {
|
||||
if (!st.is_already_exist()) {
|
||||
|
|
@ -162,7 +162,7 @@ void ObjectCacheBench::prepare_data(StoragePageCache* cache, int64_t count) {
|
|||
auto* ptr = new std::vector<uint8_t>(_page_size);
|
||||
(*ptr)[0] = 1;
|
||||
PageCacheHandle handle;
|
||||
ObjectCacheWriteOptions options;
|
||||
MemCacheWriteOptions options;
|
||||
Status st = cache->insert(key, ptr, options, &handle);
|
||||
if (!st.ok()) {
|
||||
if (!st.is_already_exist()) {
|
||||
|
|
@ -211,7 +211,7 @@ void ObjectCacheBench::random_insert_multi_threads(benchmark::State* state, Stor
|
|||
auto* ptr = new std::vector<uint8_t>(page_size);
|
||||
(*ptr)[0] = 1;
|
||||
PageCacheHandle handle;
|
||||
ObjectCacheWriteOptions options;
|
||||
MemCacheWriteOptions options;
|
||||
Status st = cache->insert(key, ptr, options, &handle);
|
||||
if (!st.ok()) {
|
||||
if (!st.is_already_exist()) {
|
||||
|
|
|
|||
|
|
@ -14,17 +14,8 @@
|
|||
|
||||
#pragma once
|
||||
|
||||
#include <cstdint>
|
||||
#include <functional>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
#include "cache/cache_metrics.h"
|
||||
#include "common/status.h"
|
||||
|
||||
namespace starrocks {
|
||||
|
||||
// Options to control how to create DataCache instance
|
||||
struct DataCacheOptions {
|
||||
bool enable_datacache = false;
|
||||
bool enable_cache_select = false;
|
||||
|
|
@ -37,82 +28,4 @@ struct DataCacheOptions {
|
|||
int64_t datacache_ttl_seconds = 0;
|
||||
};
|
||||
|
||||
struct DirSpace {
|
||||
std::string path;
|
||||
size_t size;
|
||||
};
|
||||
|
||||
struct RemoteCacheOptions {
|
||||
double skip_read_factor = 0;
|
||||
};
|
||||
|
||||
struct MemCacheOptions {
|
||||
size_t mem_space_size = 0;
|
||||
};
|
||||
|
||||
struct DiskCacheOptions {
|
||||
// basic
|
||||
size_t mem_space_size = 0;
|
||||
std::vector<DirSpace> dir_spaces;
|
||||
std::string meta_path;
|
||||
|
||||
// advanced
|
||||
size_t block_size = 0;
|
||||
bool enable_checksum = false;
|
||||
bool enable_direct_io = false;
|
||||
bool enable_tiered_cache = true;
|
||||
bool enable_datacache_persistence = false;
|
||||
size_t max_concurrent_inserts = 0;
|
||||
size_t max_flying_memory_mb = 0;
|
||||
double scheduler_threads_per_cpu = 0;
|
||||
double skip_read_factor = 0;
|
||||
uint32_t inline_item_count_limit = 0;
|
||||
std::string eviction_policy;
|
||||
};
|
||||
|
||||
struct BlockCacheOptions {
|
||||
size_t block_size = 0;
|
||||
};
|
||||
|
||||
struct WriteCacheOptions {
|
||||
int8_t priority = 0;
|
||||
// If ttl_seconds=0 (default), no ttl restriction will be set. If an old one exists, remove it.
|
||||
uint64_t ttl_seconds = 0;
|
||||
// If overwrite=true, the cache value will be replaced if it already exists.
|
||||
bool overwrite = false;
|
||||
bool async = false;
|
||||
// When allow_zero_copy=true, it means the caller can ensure the target buffer not be released before
|
||||
// to write finish. So the cache library can use the buffer directly without copying it to another buffer.
|
||||
bool allow_zero_copy = false;
|
||||
std::function<void(int, const std::string&)> callback = nullptr;
|
||||
|
||||
// The probability to evict other items if the cache space is full, which can help avoid frequent cache replacement
|
||||
// and improve cache hit rate sometimes.
|
||||
// It is expressed as a percentage. If evict_probability is 10, it means the probability to evict other data is 10%.
|
||||
int32_t evict_probability = 100;
|
||||
|
||||
// The base frequency for target cache.
|
||||
// When using multiple segment lru, a higher frequency may cause the cache is written to warm segment directly.
|
||||
// For the default cache options, that `lru_segment_freq_bits` is 0:
|
||||
// * The default `frequency=0` indicates the cache will be written to cold segment.
|
||||
// * A frequency value greater than 0 indicates writing this cache directly to the warm segment.
|
||||
int8_t frequency = 0;
|
||||
|
||||
struct Stats {
|
||||
int64_t write_mem_bytes = 0;
|
||||
int64_t write_disk_bytes = 0;
|
||||
} stats;
|
||||
};
|
||||
|
||||
struct ReadCacheOptions {
|
||||
bool use_adaptor = false;
|
||||
std::string remote_host;
|
||||
int32_t remote_port;
|
||||
|
||||
struct Stats {
|
||||
int64_t read_mem_bytes = 0;
|
||||
int64_t read_disk_bytes = 0;
|
||||
} stats;
|
||||
};
|
||||
|
||||
} // namespace starrocks
|
||||
|
|
|
|||
|
|
@ -45,7 +45,8 @@ Status BlockCache::init(const BlockCacheOptions& options, std::shared_ptr<LocalD
|
|||
return Status::OK();
|
||||
}
|
||||
|
||||
Status BlockCache::write(const CacheKey& cache_key, off_t offset, const IOBuffer& buffer, WriteCacheOptions* options) {
|
||||
Status BlockCache::write(const CacheKey& cache_key, off_t offset, const IOBuffer& buffer,
|
||||
DiskCacheWriteOptions* options) {
|
||||
if (offset % _block_size != 0) {
|
||||
LOG(WARNING) << "write block key: " << cache_key << " with invalid args, offset: " << offset;
|
||||
return Status::InvalidArgument(strings::Substitute("offset must be aligned by block size $0", _block_size));
|
||||
|
|
@ -62,7 +63,7 @@ Status BlockCache::write(const CacheKey& cache_key, off_t offset, const IOBuffer
|
|||
static void empty_deleter(void*) {}
|
||||
|
||||
Status BlockCache::write(const CacheKey& cache_key, off_t offset, size_t size, const char* data,
|
||||
WriteCacheOptions* options) {
|
||||
DiskCacheWriteOptions* options) {
|
||||
if (!data) {
|
||||
return Status::InvalidArgument("invalid data buffer");
|
||||
}
|
||||
|
|
@ -73,7 +74,7 @@ Status BlockCache::write(const CacheKey& cache_key, off_t offset, size_t size, c
|
|||
}
|
||||
|
||||
Status BlockCache::read(const CacheKey& cache_key, off_t offset, size_t size, IOBuffer* buffer,
|
||||
ReadCacheOptions* options) {
|
||||
DiskCacheReadOptions* options) {
|
||||
if (size == 0) {
|
||||
return Status::OK();
|
||||
}
|
||||
|
|
@ -84,7 +85,7 @@ Status BlockCache::read(const CacheKey& cache_key, off_t offset, size_t size, IO
|
|||
}
|
||||
|
||||
StatusOr<size_t> BlockCache::read(const CacheKey& cache_key, off_t offset, size_t size, char* data,
|
||||
ReadCacheOptions* options) {
|
||||
DiskCacheReadOptions* options) {
|
||||
IOBuffer buffer;
|
||||
RETURN_IF_ERROR(read(cache_key, offset, size, &buffer, options));
|
||||
buffer.copy_to(data);
|
||||
|
|
@ -117,7 +118,7 @@ Status BlockCache::remove(const CacheKey& cache_key, off_t offset, size_t size)
|
|||
}
|
||||
|
||||
Status BlockCache::read_buffer_from_remote_cache(const std::string& cache_key, size_t offset, size_t size,
|
||||
IOBuffer* buffer, ReadCacheOptions* options) {
|
||||
IOBuffer* buffer, DiskCacheReadOptions* options) {
|
||||
if (size == 0) {
|
||||
return Status::OK();
|
||||
}
|
||||
|
|
|
|||
|
|
@ -22,6 +22,10 @@
|
|||
|
||||
namespace starrocks {
|
||||
|
||||
struct BlockCacheOptions {
|
||||
size_t block_size = 0;
|
||||
};
|
||||
|
||||
class BlockCache {
|
||||
public:
|
||||
using CacheKey = std::string;
|
||||
|
|
@ -37,18 +41,19 @@ public:
|
|||
std::shared_ptr<RemoteCacheEngine> remote_cache);
|
||||
|
||||
// Write data buffer to cache, the `offset` must be aligned by block size
|
||||
Status write(const CacheKey& cache_key, off_t offset, const IOBuffer& buffer, WriteCacheOptions* options = nullptr);
|
||||
Status write(const CacheKey& cache_key, off_t offset, const IOBuffer& buffer,
|
||||
DiskCacheWriteOptions* options = nullptr);
|
||||
|
||||
Status write(const CacheKey& cache_key, off_t offset, size_t size, const char* data,
|
||||
WriteCacheOptions* options = nullptr);
|
||||
DiskCacheWriteOptions* options = nullptr);
|
||||
|
||||
// Read data from cache, it returns the data size if successful; otherwise the error status
|
||||
// will be returned. The offset and size must be aligned by block size.
|
||||
Status read(const CacheKey& cache_key, off_t offset, size_t size, IOBuffer* buffer,
|
||||
ReadCacheOptions* options = nullptr);
|
||||
DiskCacheReadOptions* options = nullptr);
|
||||
|
||||
StatusOr<size_t> read(const CacheKey& cache_key, off_t offset, size_t size, char* data,
|
||||
ReadCacheOptions* options = nullptr);
|
||||
DiskCacheReadOptions* options = nullptr);
|
||||
|
||||
bool exist(const CacheKey& cache_key, off_t offset, size_t size) const;
|
||||
|
||||
|
|
@ -57,7 +62,7 @@ public:
|
|||
|
||||
// Read data from remote cache
|
||||
Status read_buffer_from_remote_cache(const std::string& cache_key, size_t offset, size_t size, IOBuffer* buffer,
|
||||
ReadCacheOptions* options);
|
||||
DiskCacheReadOptions* options);
|
||||
|
||||
void record_read_local_cache(size_t size, int64_t latency_us);
|
||||
|
||||
|
|
|
|||
|
|
@ -14,13 +14,78 @@
|
|||
|
||||
#pragma once
|
||||
|
||||
#include "cache/cache_options.h"
|
||||
#include "cache/cache_metrics.h"
|
||||
#include "cache/disk_cache/io_buffer.h"
|
||||
#include "cache/mem_cache/cache_types.h"
|
||||
#include "common/status.h"
|
||||
|
||||
namespace starrocks {
|
||||
|
||||
struct DirSpace {
|
||||
std::string path;
|
||||
size_t size;
|
||||
};
|
||||
|
||||
struct DiskCacheOptions {
|
||||
// basic
|
||||
size_t mem_space_size = 0;
|
||||
std::vector<DirSpace> dir_spaces;
|
||||
std::string meta_path;
|
||||
|
||||
// advanced
|
||||
size_t block_size = 0;
|
||||
bool enable_checksum = false;
|
||||
bool enable_direct_io = false;
|
||||
bool enable_tiered_cache = true;
|
||||
bool enable_datacache_persistence = false;
|
||||
size_t max_concurrent_inserts = 0;
|
||||
size_t max_flying_memory_mb = 0;
|
||||
double scheduler_threads_per_cpu = 0;
|
||||
double skip_read_factor = 0;
|
||||
uint32_t inline_item_count_limit = 0;
|
||||
std::string eviction_policy;
|
||||
};
|
||||
|
||||
struct DiskCacheWriteOptions {
|
||||
int8_t priority = 0;
|
||||
// If ttl_seconds=0 (default), no ttl restriction will be set. If an old one exists, remove it.
|
||||
uint64_t ttl_seconds = 0;
|
||||
// If overwrite=true, the cache value will be replaced if it already exists.
|
||||
bool overwrite = false;
|
||||
bool async = false;
|
||||
// When allow_zero_copy=true, it means the caller can ensure the target buffer not be released before
|
||||
// to write finish. So the cache library can use the buffer directly without copying it to another buffer.
|
||||
bool allow_zero_copy = false;
|
||||
std::function<void(int, const std::string&)> callback = nullptr;
|
||||
|
||||
// The probability to evict other items if the cache space is full, which can help avoid frequent cache replacement
|
||||
// and improve cache hit rate sometimes.
|
||||
// It is expressed as a percentage. If evict_probability is 10, it means the probability to evict other data is 10%.
|
||||
int32_t evict_probability = 100;
|
||||
|
||||
// The base frequency for target cache.
|
||||
// When using multiple segment lru, a higher frequency may cause the cache is written to warm segment directly.
|
||||
// For the default cache options, that `lru_segment_freq_bits` is 0:
|
||||
// * The default `frequency=0` indicates the cache will be written to cold segment.
|
||||
// * A frequency value greater than 0 indicates writing this cache directly to the warm segment.
|
||||
int8_t frequency = 0;
|
||||
|
||||
struct Stats {
|
||||
int64_t write_mem_bytes = 0;
|
||||
int64_t write_disk_bytes = 0;
|
||||
} stats;
|
||||
};
|
||||
|
||||
struct DiskCacheReadOptions {
|
||||
bool use_adaptor = false;
|
||||
std::string remote_host;
|
||||
int32_t remote_port;
|
||||
|
||||
struct Stats {
|
||||
int64_t read_mem_bytes = 0;
|
||||
int64_t read_disk_bytes = 0;
|
||||
} stats;
|
||||
};
|
||||
|
||||
class LocalDiskCacheEngine {
|
||||
public:
|
||||
virtual ~LocalDiskCacheEngine() = default;
|
||||
|
|
@ -28,12 +93,12 @@ public:
|
|||
virtual bool is_initialized() const = 0;
|
||||
|
||||
// Write data to cache
|
||||
virtual Status write(const std::string& key, const IOBuffer& buffer, WriteCacheOptions* options) = 0;
|
||||
virtual Status write(const std::string& key, const IOBuffer& buffer, DiskCacheWriteOptions* options) = 0;
|
||||
|
||||
// Read data from cache, it returns the data size if successful; otherwise the error status
|
||||
// will be returned.
|
||||
virtual Status read(const std::string& key, size_t off, size_t size, IOBuffer* buffer,
|
||||
ReadCacheOptions* options) = 0;
|
||||
DiskCacheReadOptions* options) = 0;
|
||||
|
||||
virtual bool exist(const std::string& key) const = 0;
|
||||
|
||||
|
|
@ -64,9 +129,6 @@ public:
|
|||
// Get the cache hit count.
|
||||
virtual size_t hit_count() const = 0;
|
||||
|
||||
// Get all cache metrics together.
|
||||
virtual const ObjectCacheMetrics metrics() const = 0;
|
||||
|
||||
// Remove all cache entries that are not actively in use.
|
||||
virtual Status prune() = 0;
|
||||
};
|
||||
|
|
|
|||
|
|
@ -68,7 +68,7 @@ Status StarCacheEngine::init(const DiskCacheOptions& options) {
|
|||
return Status::OK();
|
||||
}
|
||||
|
||||
Status StarCacheEngine::write(const std::string& key, const IOBuffer& buffer, WriteCacheOptions* options) {
|
||||
Status StarCacheEngine::write(const std::string& key, const IOBuffer& buffer, DiskCacheWriteOptions* options) {
|
||||
if (!options) {
|
||||
return to_status(_cache->set(key, buffer.const_raw_buf(), nullptr));
|
||||
}
|
||||
|
|
@ -104,7 +104,7 @@ Status StarCacheEngine::write(const std::string& key, const IOBuffer& buffer, Wr
|
|||
}
|
||||
|
||||
Status StarCacheEngine::read(const std::string& key, size_t off, size_t size, IOBuffer* buffer,
|
||||
ReadCacheOptions* options) {
|
||||
DiskCacheReadOptions* options) {
|
||||
if (!options) {
|
||||
return to_status(_cache->read(key, off, size, &buffer->raw_buf(), nullptr));
|
||||
}
|
||||
|
|
@ -198,17 +198,6 @@ size_t StarCacheEngine::hit_count() const {
|
|||
return metrics.detail_l1->hit_count;
|
||||
}
|
||||
|
||||
const ObjectCacheMetrics StarCacheEngine::metrics() const {
|
||||
auto starcache_metrics = _cache->metrics(2);
|
||||
ObjectCacheMetrics m;
|
||||
m.capacity = starcache_metrics.mem_quota_bytes;
|
||||
m.usage = starcache_metrics.mem_used_bytes;
|
||||
m.lookup_count = starcache_metrics.detail_l1->hit_count + starcache_metrics.detail_l1->miss_count;
|
||||
m.hit_count = starcache_metrics.detail_l1->hit_count;
|
||||
m.object_item_count = starcache_metrics.detail_l2->object_item_count;
|
||||
return m;
|
||||
}
|
||||
|
||||
Status StarCacheEngine::prune() {
|
||||
return to_status(_cache->update_mem_quota(0, false));
|
||||
}
|
||||
|
|
|
|||
|
|
@ -29,8 +29,9 @@ public:
|
|||
Status init(const DiskCacheOptions& options);
|
||||
bool is_initialized() const override { return _initialized.load(std::memory_order_relaxed); }
|
||||
|
||||
Status write(const std::string& key, const IOBuffer& buffer, WriteCacheOptions* options) override;
|
||||
Status read(const std::string& key, size_t off, size_t size, IOBuffer* buffer, ReadCacheOptions* options) override;
|
||||
Status write(const std::string& key, const IOBuffer& buffer, DiskCacheWriteOptions* options) override;
|
||||
Status read(const std::string& key, size_t off, size_t size, IOBuffer* buffer,
|
||||
DiskCacheReadOptions* options) override;
|
||||
|
||||
bool exist(const std::string& key) const override;
|
||||
|
||||
|
|
@ -60,8 +61,6 @@ public:
|
|||
|
||||
size_t hit_count() const override;
|
||||
|
||||
const ObjectCacheMetrics metrics() const override;
|
||||
|
||||
Status prune() override;
|
||||
|
||||
private:
|
||||
|
|
|
|||
|
|
@ -1,53 +0,0 @@
|
|||
// Copyright 2021-present StarRocks, Inc. 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
|
||||
//
|
||||
// https://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 <functional>
|
||||
#include <ostream>
|
||||
#include <string>
|
||||
|
||||
// Not a good way to import lru cache header here, just for temporary compatibility with old deleters.
|
||||
#include "util/lru_cache.h"
|
||||
|
||||
namespace starrocks {
|
||||
|
||||
struct ObjectCacheWriteOptions {
|
||||
// The priority of the cache object, only support 0 and 1 now.
|
||||
int8_t priority = 0;
|
||||
};
|
||||
|
||||
struct ObjectCacheReadOptions {};
|
||||
|
||||
struct ObjectCacheHandle {};
|
||||
|
||||
struct ObjectCacheMetrics {
|
||||
size_t capacity = 0;
|
||||
size_t usage = 0;
|
||||
size_t lookup_count = 0;
|
||||
size_t hit_count = 0;
|
||||
size_t object_item_count = 0;
|
||||
};
|
||||
|
||||
using ObjectCacheHandlePtr = ObjectCacheHandle*;
|
||||
|
||||
// using CacheDeleter = std::function<void(const std::string&, void*)>;
|
||||
//
|
||||
// We only use the deleter function of the lru cache temporarily.
|
||||
// Maybe a std::function object or a function pointer like `void (*)(std::string&, void*)` which
|
||||
// independent on lru cache is more appropriate, but it is not easy to convert them to the lru
|
||||
// cache deleter when using a lru cache module.
|
||||
using ObjectCacheDeleter = void (*)(const CacheKey&, void*);
|
||||
|
||||
} // namespace starrocks
|
||||
|
|
@ -14,12 +14,35 @@
|
|||
|
||||
#pragma once
|
||||
|
||||
#include "cache/cache_options.h"
|
||||
#include "cache/cache_metrics.h"
|
||||
#include "cache/disk_cache/io_buffer.h"
|
||||
#include "cache/mem_cache/cache_types.h"
|
||||
#include "common/status.h"
|
||||
|
||||
namespace starrocks {
|
||||
class CacheKey;
|
||||
|
||||
struct MemCacheOptions {
|
||||
size_t mem_space_size = 0;
|
||||
};
|
||||
|
||||
struct MemCacheWriteOptions {
|
||||
// The priority of the cache object, only support 0 and 1 now.
|
||||
int8_t priority = 0;
|
||||
};
|
||||
|
||||
struct MemCacheReadOptions {};
|
||||
|
||||
struct MemCacheHandle {};
|
||||
|
||||
using MemCacheHandlePtr = MemCacheHandle*;
|
||||
|
||||
// using CacheDeleter = std::function<void(const std::string&, void*)>;
|
||||
//
|
||||
// We only use the deleter function of the lru cache temporarily.
|
||||
// Maybe a std::function object or a function pointer like `void (*)(std::string&, void*)` which
|
||||
// independent on lru cache is more appropriate, but it is not easy to convert them to the lru
|
||||
// cache deleter when using a lru cache module.
|
||||
using MemCacheDeleter = void (*)(const CacheKey&, void*);
|
||||
|
||||
class LocalMemCacheEngine {
|
||||
public:
|
||||
|
|
@ -28,22 +51,22 @@ public:
|
|||
virtual bool is_initialized() const = 0;
|
||||
|
||||
// Insert object to cache
|
||||
virtual Status insert(const std::string& key, void* value, size_t size, ObjectCacheDeleter deleter,
|
||||
ObjectCacheHandlePtr* handle, const ObjectCacheWriteOptions& options) = 0;
|
||||
virtual Status insert(const std::string& key, void* value, size_t size, MemCacheDeleter deleter,
|
||||
MemCacheHandlePtr* handle, const MemCacheWriteOptions& options) = 0;
|
||||
|
||||
// Lookup object from cache, the `handle` wraps the object pointer.
|
||||
// As long as the handle object is not destroyed and the user does not manually call the `handle->release()`
|
||||
// function, the corresponding pointer will never be freed by the cache system.
|
||||
virtual Status lookup(const std::string& key, ObjectCacheHandlePtr* handle,
|
||||
ObjectCacheReadOptions* options = nullptr) = 0;
|
||||
virtual Status lookup(const std::string& key, MemCacheHandlePtr* handle,
|
||||
MemCacheReadOptions* options = nullptr) = 0;
|
||||
|
||||
// Release a handle returned by a previous insert() or lookup().
|
||||
// The handle must have not been released yet.
|
||||
virtual void release(ObjectCacheHandlePtr handle) = 0;
|
||||
virtual void release(MemCacheHandlePtr handle) = 0;
|
||||
|
||||
// Return the value in the given handle returned by a previous insert() or lookup().
|
||||
// The handle must have not been released yet.
|
||||
virtual const void* value(ObjectCacheHandlePtr handle) = 0;
|
||||
virtual const void* value(MemCacheHandlePtr handle) = 0;
|
||||
|
||||
virtual bool exist(const std::string& key) const = 0;
|
||||
|
||||
|
|
@ -74,9 +97,6 @@ public:
|
|||
// Get the cache hit count.
|
||||
virtual size_t hit_count() const = 0;
|
||||
|
||||
// Get all cache metrics together.
|
||||
virtual const ObjectCacheMetrics metrics() const = 0;
|
||||
|
||||
// Remove all cache entries that are not actively in use.
|
||||
virtual Status prune() = 0;
|
||||
};
|
||||
|
|
|
|||
|
|
@ -23,21 +23,21 @@ Status LRUCacheEngine::init(const MemCacheOptions& options) {
|
|||
return Status::OK();
|
||||
}
|
||||
|
||||
Status LRUCacheEngine::insert(const std::string& key, void* value, size_t size, ObjectCacheDeleter deleter,
|
||||
ObjectCacheHandlePtr* handle, const ObjectCacheWriteOptions& options) {
|
||||
Status LRUCacheEngine::insert(const std::string& key, void* value, size_t size, MemCacheDeleter deleter,
|
||||
MemCacheHandlePtr* handle, const MemCacheWriteOptions& options) {
|
||||
auto* lru_handle = _cache->insert(key, value, size, deleter, static_cast<CachePriority>(options.priority));
|
||||
if (handle) {
|
||||
*handle = reinterpret_cast<ObjectCacheHandlePtr>(lru_handle);
|
||||
*handle = reinterpret_cast<MemCacheHandlePtr>(lru_handle);
|
||||
}
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
Status LRUCacheEngine::lookup(const std::string& key, ObjectCacheHandlePtr* handle, ObjectCacheReadOptions* options) {
|
||||
Status LRUCacheEngine::lookup(const std::string& key, MemCacheHandlePtr* handle, MemCacheReadOptions* options) {
|
||||
auto* lru_handle = _cache->lookup(CacheKey(key));
|
||||
if (!lru_handle) {
|
||||
return Status::NotFound("no such entry");
|
||||
}
|
||||
*handle = reinterpret_cast<ObjectCacheHandlePtr>(lru_handle);
|
||||
*handle = reinterpret_cast<MemCacheHandlePtr>(lru_handle);
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
|
|
@ -80,12 +80,12 @@ Status LRUCacheEngine::prune() {
|
|||
return Status::OK();
|
||||
}
|
||||
|
||||
void LRUCacheEngine::release(ObjectCacheHandlePtr handle) {
|
||||
void LRUCacheEngine::release(MemCacheHandlePtr handle) {
|
||||
auto lru_handle = reinterpret_cast<Cache::Handle*>(handle);
|
||||
_cache->release(lru_handle);
|
||||
}
|
||||
|
||||
const void* LRUCacheEngine::value(ObjectCacheHandlePtr handle) {
|
||||
const void* LRUCacheEngine::value(MemCacheHandlePtr handle) {
|
||||
auto lru_handle = reinterpret_cast<Cache::Handle*>(handle);
|
||||
return _cache->value(lru_handle);
|
||||
}
|
||||
|
|
@ -113,15 +113,4 @@ size_t LRUCacheEngine::hit_count() const {
|
|||
return _cache->get_hit_count();
|
||||
}
|
||||
|
||||
const ObjectCacheMetrics LRUCacheEngine::metrics() const {
|
||||
ObjectCacheMetrics m;
|
||||
m.capacity = _cache->get_capacity();
|
||||
m.usage = _cache->get_memory_usage();
|
||||
m.lookup_count = _cache->get_lookup_count();
|
||||
m.hit_count = _cache->get_hit_count();
|
||||
// Unsupported
|
||||
m.object_item_count = 0;
|
||||
return m;
|
||||
}
|
||||
|
||||
} // namespace starrocks
|
||||
|
|
@ -28,9 +28,9 @@ public:
|
|||
Status init(const MemCacheOptions& options);
|
||||
bool is_initialized() const override { return _initialized.load(std::memory_order_relaxed); }
|
||||
|
||||
Status insert(const std::string& key, void* value, size_t size, ObjectCacheDeleter deleter,
|
||||
ObjectCacheHandlePtr* handle, const ObjectCacheWriteOptions& options) override;
|
||||
Status lookup(const std::string& key, ObjectCacheHandlePtr* handle, ObjectCacheReadOptions* options) override;
|
||||
Status insert(const std::string& key, void* value, size_t size, MemCacheDeleter deleter, MemCacheHandlePtr* handle,
|
||||
const MemCacheWriteOptions& options) override;
|
||||
Status lookup(const std::string& key, MemCacheHandlePtr* handle, MemCacheReadOptions* options) override;
|
||||
|
||||
bool exist(const std::string& key) const override;
|
||||
Status remove(const std::string& key) override;
|
||||
|
|
@ -45,8 +45,8 @@ public:
|
|||
bool available() const override { return is_initialized() && has_mem_cache(); }
|
||||
bool mem_cache_available() const override { return is_initialized() && has_mem_cache(); }
|
||||
|
||||
void release(ObjectCacheHandlePtr handle) override;
|
||||
const void* value(ObjectCacheHandlePtr handle) override;
|
||||
void release(MemCacheHandlePtr handle) override;
|
||||
const void* value(MemCacheHandlePtr handle) override;
|
||||
|
||||
Status adjust_mem_quota(int64_t delta, size_t min_capacity) override;
|
||||
|
||||
|
|
@ -57,8 +57,6 @@ public:
|
|||
|
||||
size_t hit_count() const override;
|
||||
|
||||
const ObjectCacheMetrics metrics() const override;
|
||||
|
||||
Status prune() override;
|
||||
|
||||
private:
|
||||
|
|
|
|||
|
|
@ -105,7 +105,7 @@ size_t StoragePageCache::get_pinned_count() const {
|
|||
}
|
||||
|
||||
bool StoragePageCache::lookup(const std::string& key, PageCacheHandle* handle) {
|
||||
ObjectCacheHandle* obj_handle = nullptr;
|
||||
MemCacheHandle* obj_handle = nullptr;
|
||||
Status st = _cache->lookup(key, &obj_handle);
|
||||
if (!st.ok()) {
|
||||
return false;
|
||||
|
|
@ -115,7 +115,7 @@ bool StoragePageCache::lookup(const std::string& key, PageCacheHandle* handle) {
|
|||
return true;
|
||||
}
|
||||
|
||||
Status StoragePageCache::insert(const std::string& key, std::vector<uint8_t>* data, const ObjectCacheWriteOptions& opts,
|
||||
Status StoragePageCache::insert(const std::string& key, std::vector<uint8_t>* data, const MemCacheWriteOptions& opts,
|
||||
PageCacheHandle* handle) {
|
||||
#ifndef BE_TEST
|
||||
int64_t mem_size = malloc_usable_size(data->data()) + sizeof(*data);
|
||||
|
|
@ -131,7 +131,7 @@ Status StoragePageCache::insert(const std::string& key, std::vector<uint8_t>* da
|
|||
delete cache_item;
|
||||
};
|
||||
|
||||
ObjectCacheHandle* obj_handle = nullptr;
|
||||
MemCacheHandle* obj_handle = nullptr;
|
||||
// Use mem size managed by memory allocator as this record charge size.
|
||||
// At the same time, we should record this record size for data fetching when lookup.
|
||||
Status st = _cache->insert(key, (void*)data, mem_size, deleter, &obj_handle, opts);
|
||||
|
|
@ -142,9 +142,9 @@ Status StoragePageCache::insert(const std::string& key, std::vector<uint8_t>* da
|
|||
return st;
|
||||
}
|
||||
|
||||
Status StoragePageCache::insert(const std::string& key, void* data, int64_t size, ObjectCacheDeleter deleter,
|
||||
const ObjectCacheWriteOptions& opts, PageCacheHandle* handle) {
|
||||
ObjectCacheHandle* obj_handle = nullptr;
|
||||
Status StoragePageCache::insert(const std::string& key, void* data, int64_t size, MemCacheDeleter deleter,
|
||||
const MemCacheWriteOptions& opts, PageCacheHandle* handle) {
|
||||
MemCacheHandle* obj_handle = nullptr;
|
||||
Status st = _cache->insert(key, data, size, deleter, &obj_handle, opts);
|
||||
if (st.ok()) {
|
||||
*handle = PageCacheHandle(_cache, obj_handle);
|
||||
|
|
|
|||
|
|
@ -43,7 +43,7 @@ namespace starrocks {
|
|||
|
||||
class PageCacheHandle;
|
||||
class MemTracker;
|
||||
struct ObjectCacheWriteOptions;
|
||||
struct MemCacheWriteOptions;
|
||||
|
||||
// Page cache min size is 256MB
|
||||
static constexpr int64_t kcacheMinSize = 268435456;
|
||||
|
|
@ -87,11 +87,11 @@ public:
|
|||
// This function is thread-safe, and when two clients insert two same key
|
||||
// concurrently, this function can assure that only one page is cached.
|
||||
// The in_memory page will have higher priority.
|
||||
Status insert(const std::string& key, std::vector<uint8_t>* data, const ObjectCacheWriteOptions& opts,
|
||||
Status insert(const std::string& key, std::vector<uint8_t>* data, const MemCacheWriteOptions& opts,
|
||||
PageCacheHandle* handle);
|
||||
|
||||
Status insert(const std::string& key, void* data, int64_t size, ObjectCacheDeleter deleter,
|
||||
const ObjectCacheWriteOptions& opts, PageCacheHandle* handle);
|
||||
Status insert(const std::string& key, void* data, int64_t size, MemCacheDeleter deleter,
|
||||
const MemCacheWriteOptions& opts, PageCacheHandle* handle);
|
||||
|
||||
size_t memory_usage() const { return _cache->mem_usage(); }
|
||||
|
||||
|
|
@ -125,7 +125,7 @@ private:
|
|||
class PageCacheHandle {
|
||||
public:
|
||||
PageCacheHandle() = default;
|
||||
PageCacheHandle(LocalMemCacheEngine* cache, ObjectCacheHandle* handle) : _cache(cache), _handle(handle) {}
|
||||
PageCacheHandle(LocalMemCacheEngine* cache, MemCacheHandle* handle) : _cache(cache), _handle(handle) {}
|
||||
|
||||
// Don't allow copy and assign
|
||||
PageCacheHandle(const PageCacheHandle&) = delete;
|
||||
|
|
@ -155,7 +155,7 @@ public:
|
|||
|
||||
private:
|
||||
LocalMemCacheEngine* _cache = nullptr;
|
||||
ObjectCacheHandle* _handle = nullptr;
|
||||
MemCacheHandle* _handle = nullptr;
|
||||
};
|
||||
|
||||
} // namespace starrocks
|
||||
|
|
|
|||
|
|
@ -29,7 +29,7 @@ Status PeerCacheEngine::init(const RemoteCacheOptions& options) {
|
|||
}
|
||||
|
||||
Status PeerCacheEngine::read(const std::string& key, size_t off, size_t size, IOBuffer* buffer,
|
||||
ReadCacheOptions* options) {
|
||||
DiskCacheReadOptions* options) {
|
||||
if (options->use_adaptor && !_cache_adaptor->check_read_cache()) {
|
||||
return Status::ResourceBusy("resource is busy");
|
||||
}
|
||||
|
|
|
|||
|
|
@ -26,9 +26,10 @@ public:
|
|||
|
||||
Status init(const RemoteCacheOptions& options) override;
|
||||
|
||||
Status read(const std::string& key, size_t off, size_t size, IOBuffer* buffer, ReadCacheOptions* options) override;
|
||||
Status read(const std::string& key, size_t off, size_t size, IOBuffer* buffer,
|
||||
DiskCacheReadOptions* options) override;
|
||||
|
||||
Status write(const std::string& key, const IOBuffer& buffer, WriteCacheOptions* options) override {
|
||||
Status write(const std::string& key, const IOBuffer& buffer, DiskCacheWriteOptions* options) override {
|
||||
return Status::NotSupported("write data to peer cache is unsupported");
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -14,12 +14,16 @@
|
|||
|
||||
#pragma once
|
||||
|
||||
#include "cache/cache_options.h"
|
||||
#include "cache/disk_cache/io_buffer.h"
|
||||
#include "cache/disk_cache/local_disk_cache_engine.h"
|
||||
#include "common/status.h"
|
||||
|
||||
namespace starrocks {
|
||||
|
||||
struct RemoteCacheOptions {
|
||||
double skip_read_factor = 0;
|
||||
};
|
||||
|
||||
class RemoteCacheEngine {
|
||||
public:
|
||||
virtual ~RemoteCacheEngine() = default;
|
||||
|
|
@ -28,12 +32,12 @@ public:
|
|||
virtual Status init(const RemoteCacheOptions& options) = 0;
|
||||
|
||||
// Write data to remote cache
|
||||
virtual Status write(const std::string& key, const IOBuffer& buffer, WriteCacheOptions* options) = 0;
|
||||
virtual Status write(const std::string& key, const IOBuffer& buffer, DiskCacheWriteOptions* options) = 0;
|
||||
|
||||
// Read data from remote cache, it returns the data size if successful; otherwise the error status
|
||||
// will be returned.
|
||||
virtual Status read(const std::string& key, size_t off, size_t size, IOBuffer* buffer,
|
||||
ReadCacheOptions* options) = 0;
|
||||
DiskCacheReadOptions* options) = 0;
|
||||
|
||||
// Remove data from cache.
|
||||
virtual Status remove(const std::string& key) = 0;
|
||||
|
|
|
|||
|
|
@ -17,6 +17,7 @@
|
|||
#include <atomic>
|
||||
#include <boost/algorithm/string.hpp>
|
||||
|
||||
#include "cache/cache_options.h"
|
||||
#include "connector/deletion_vector/deletion_bitmap.h"
|
||||
#include "exec/olap_scan_prepare.h"
|
||||
#include "exec/pipeline/scan/morsel.h"
|
||||
|
|
|
|||
|
|
@ -470,7 +470,7 @@ StatusOr<FileMetaDataPtr> FileMetaDataParser::get_file_metadata() {
|
|||
RETURN_IF_ERROR(_parse_footer(&file_metadata, &file_metadata_size));
|
||||
if (file_metadata_size > 0) {
|
||||
auto deleter = [](const starrocks::CacheKey& key, void* value) { delete (FileMetaDataPtr*)value; };
|
||||
ObjectCacheWriteOptions options;
|
||||
MemCacheWriteOptions options;
|
||||
auto capture = std::make_unique<FileMetaDataPtr>(file_metadata);
|
||||
Status st = _cache->insert(metacache_key, (void*)(capture.get()), file_metadata_size, deleter, options,
|
||||
&cache_handle);
|
||||
|
|
|
|||
|
|
@ -92,7 +92,7 @@ Status PageReader::_deal_page_with_cache() {
|
|||
return Status::OK();
|
||||
}
|
||||
RETURN_IF_ERROR(_read_and_decompress_internal(true));
|
||||
ObjectCacheWriteOptions opts;
|
||||
MemCacheWriteOptions opts;
|
||||
auto st = _cache->insert(page_cache_key, _cache_buf, opts, &cache_handle);
|
||||
if (st.ok()) {
|
||||
_page_handle = PageHandle(std::move(cache_handle));
|
||||
|
|
|
|||
|
|
@ -125,7 +125,7 @@ Status CacheInputStream::_read_from_cache(const int64_t offset, const int64_t si
|
|||
Status res;
|
||||
int64_t read_local_cache_ns = 0;
|
||||
BlockBuffer block;
|
||||
ReadCacheOptions options;
|
||||
DiskCacheReadOptions options;
|
||||
size_t read_size = 0;
|
||||
{
|
||||
options.use_adaptor = _enable_cache_io_adaptor;
|
||||
|
|
@ -153,7 +153,7 @@ Status CacheInputStream::_read_from_cache(const int64_t offset, const int64_t si
|
|||
read_size = block_size;
|
||||
|
||||
if (res.ok() && _enable_populate_cache) {
|
||||
WriteCacheOptions options;
|
||||
DiskCacheWriteOptions options;
|
||||
options.async = _enable_async_populate_mode;
|
||||
options.evict_probability = _datacache_evict_probability;
|
||||
options.priority = _priority;
|
||||
|
|
@ -203,7 +203,7 @@ Status CacheInputStream::_read_from_cache(const int64_t offset, const int64_t si
|
|||
return res;
|
||||
}
|
||||
|
||||
Status CacheInputStream::_read_peer_cache(off_t offset, size_t size, IOBuffer* iobuf, ReadCacheOptions* options) {
|
||||
Status CacheInputStream::_read_peer_cache(off_t offset, size_t size, IOBuffer* iobuf, DiskCacheReadOptions* options) {
|
||||
options->remote_host = _peer_host;
|
||||
options->remote_port = _peer_port;
|
||||
return _cache->read_buffer_from_remote_cache(_cache_key, offset, size, iobuf, options);
|
||||
|
|
@ -445,7 +445,7 @@ void CacheInputStream::_populate_to_cache(const char* p, int64_t offset, int64_t
|
|||
int64_t end = std::min((offset + count + _block_size - 1) / _block_size * _block_size, _size);
|
||||
p -= (offset - begin);
|
||||
auto f = [sb, this](const char* buf, size_t off, size_t size) {
|
||||
WriteCacheOptions options;
|
||||
DiskCacheWriteOptions options;
|
||||
options.async = _enable_async_populate_mode;
|
||||
options.evict_probability = _datacache_evict_probability;
|
||||
options.priority = _priority;
|
||||
|
|
@ -474,7 +474,7 @@ void CacheInputStream::_populate_to_cache(const char* p, int64_t offset, int64_t
|
|||
return;
|
||||
}
|
||||
|
||||
void CacheInputStream::_write_cache(int64_t offset, const IOBuffer& iobuf, WriteCacheOptions* options) {
|
||||
void CacheInputStream::_write_cache(int64_t offset, const IOBuffer& iobuf, DiskCacheWriteOptions* options) {
|
||||
DCHECK(offset % _block_size == 0);
|
||||
if (_already_populated_blocks.contains(offset / _block_size)) {
|
||||
// Already populate in CacheInputStream's lifecycle, ignore this time
|
||||
|
|
|
|||
|
|
@ -108,9 +108,9 @@ protected:
|
|||
virtual Status _read_blocks_from_remote(const int64_t offset, const int64_t size, char* out);
|
||||
Status _read_from_cache(const int64_t offset, const int64_t size, const int64_t block_offset,
|
||||
const int64_t block_size, char* out);
|
||||
Status _read_peer_cache(off_t offset, size_t size, IOBuffer* iobuf, ReadCacheOptions* options);
|
||||
Status _read_peer_cache(off_t offset, size_t size, IOBuffer* iobuf, DiskCacheReadOptions* options);
|
||||
void _populate_to_cache(const char* src, int64_t offset, int64_t count, const SharedBufferPtr& sb);
|
||||
void _write_cache(int64_t offset, const IOBuffer& iobuf, WriteCacheOptions* options);
|
||||
void _write_cache(int64_t offset, const IOBuffer& iobuf, DiskCacheWriteOptions* options);
|
||||
|
||||
void _deduplicate_shared_buffer(const SharedBufferPtr& sb);
|
||||
bool _can_ignore_populate_error(const Status& status) const;
|
||||
|
|
|
|||
|
|
@ -655,7 +655,7 @@ void PInternalServiceImplBase<T>::_fetch_datacache(google::protobuf::RpcControll
|
|||
if (!block_cache || !block_cache->available()) {
|
||||
st = Status::ServiceUnavailable("block cache is unavailable");
|
||||
} else {
|
||||
ReadCacheOptions options;
|
||||
DiskCacheReadOptions options;
|
||||
IOBuffer buf;
|
||||
st = block_cache->read(request->cache_key(), request->offset(), request->size(), &buf, &options);
|
||||
if (st.ok()) {
|
||||
|
|
|
|||
|
|
@ -20,9 +20,9 @@
|
|||
#include <variant>
|
||||
|
||||
#include "gutil/macros.h"
|
||||
#include "util/lru_cache.h"
|
||||
|
||||
namespace starrocks {
|
||||
class Cache;
|
||||
class CacheKey;
|
||||
class Rowset;
|
||||
|
||||
|
|
|
|||
|
|
@ -257,7 +257,7 @@ Status PageIO::read_and_decompress_page(const PageReadOptions& opts, PageHandle*
|
|||
*body = Slice(page_slice.data, page_slice.size - 4 - footer_size);
|
||||
if (opts.use_page_cache && page_cache_available) {
|
||||
// insert this page into cache and return the cache handle
|
||||
ObjectCacheWriteOptions opts;
|
||||
MemCacheWriteOptions opts;
|
||||
Status st = cache->insert(cache_key, page.get(), opts, &cache_handle);
|
||||
*handle = st.ok() ? PageHandle(std::move(cache_handle)) : PageHandle(page.get());
|
||||
} else {
|
||||
|
|
|
|||
|
|
@ -125,7 +125,7 @@ TEST_F(BlockCacheTest, write_with_overwrite_option) {
|
|||
std::string value(cache_size, 'a');
|
||||
ASSERT_OK(cache->write(cache_key, 0, cache_size, value.c_str()));
|
||||
|
||||
WriteCacheOptions write_options;
|
||||
DiskCacheWriteOptions write_options;
|
||||
std::string value2(cache_size, 'b');
|
||||
Status st = cache->write(cache_key, 0, cache_size, value2.c_str(), &write_options);
|
||||
ASSERT_TRUE(st.is_already_exist());
|
||||
|
|
@ -182,7 +182,7 @@ TEST_F(BlockCacheTest, read_cache_with_adaptor) {
|
|||
char ch = 'a' + i % 26;
|
||||
std::string expect_value(batch_size, ch);
|
||||
char value[batch_size] = {0};
|
||||
ReadCacheOptions opts;
|
||||
DiskCacheReadOptions opts;
|
||||
opts.use_adaptor = true;
|
||||
auto res = cache->read(cache_key + std::to_string(i), 0, batch_size, value, &opts);
|
||||
ASSERT_TRUE(res.status().is_resource_busy());
|
||||
|
|
@ -199,7 +199,7 @@ TEST_F(BlockCacheTest, read_cache_with_adaptor) {
|
|||
char ch = 'a' + i % 26;
|
||||
std::string expect_value(batch_size, ch);
|
||||
char value[batch_size] = {0};
|
||||
ReadCacheOptions opts;
|
||||
DiskCacheReadOptions opts;
|
||||
opts.use_adaptor = true;
|
||||
auto res = cache->read(cache_key + std::to_string(i), 0, batch_size, value, &opts);
|
||||
ASSERT_TRUE(res.status().ok());
|
||||
|
|
@ -286,7 +286,7 @@ TEST_F(BlockCacheTest, read_peer_cache) {
|
|||
auto cache = TestCacheUtils::create_cache(options);
|
||||
|
||||
IOBuffer iobuf;
|
||||
ReadCacheOptions read_options;
|
||||
DiskCacheReadOptions read_options;
|
||||
read_options.remote_host = "127.0.0.1";
|
||||
read_options.remote_port = 0;
|
||||
auto st = cache->read_buffer_from_remote_cache("test_key", 0, 100, &iobuf, &read_options);
|
||||
|
|
|
|||
|
|
@ -36,8 +36,8 @@ protected:
|
|||
}
|
||||
|
||||
std::shared_ptr<LRUCacheEngine> _cache;
|
||||
ObjectCacheWriteOptions _write_opt;
|
||||
ObjectCacheReadOptions _read_opt;
|
||||
MemCacheWriteOptions _write_opt;
|
||||
MemCacheReadOptions _read_opt;
|
||||
|
||||
size_t _capacity = 16384;
|
||||
size_t _key_size = sizeof(LRUHandle) - 1 + 6;
|
||||
|
|
@ -52,7 +52,7 @@ void LRUCacheEngineTest::_insert_data() {
|
|||
int* ptr = (int*)malloc(_value_size);
|
||||
*ptr = i;
|
||||
|
||||
ObjectCacheHandlePtr handle = nullptr;
|
||||
MemCacheHandlePtr handle = nullptr;
|
||||
ASSERT_OK(_cache->insert(key, (void*)ptr, _value_size, &Deleter, &handle, _write_opt));
|
||||
_cache->release(handle);
|
||||
}
|
||||
|
|
@ -66,7 +66,7 @@ void LRUCacheEngineTest::_check_not_found(int value) {
|
|||
|
||||
void LRUCacheEngineTest::_check_found(int value) {
|
||||
std::string key = _int_to_string(6, value);
|
||||
ObjectCacheHandlePtr handle;
|
||||
MemCacheHandlePtr handle;
|
||||
ASSERT_OK(_cache->lookup(key, &handle, &_read_opt));
|
||||
ASSERT_EQ(*(int*)(_cache->value(handle)), value);
|
||||
_cache->release(handle);
|
||||
|
|
@ -106,7 +106,7 @@ TEST_F(LRUCacheEngineTest, test_value) {
|
|||
_insert_data();
|
||||
|
||||
std::string key = _int_to_string(6, 30);
|
||||
ObjectCacheHandlePtr handle = nullptr;
|
||||
MemCacheHandlePtr handle = nullptr;
|
||||
ASSERT_OK(_cache->lookup(key, &handle, &_read_opt));
|
||||
|
||||
const void* value = _cache->value(handle);
|
||||
|
|
@ -169,26 +169,4 @@ TEST_F(LRUCacheEngineTest, test_shutdown) {
|
|||
ASSERT_OK(_cache->shutdown());
|
||||
ASSERT_EQ(_cache->mem_usage(), 0);
|
||||
}
|
||||
|
||||
TEST_F(LRUCacheEngineTest, test_metrics) {
|
||||
// insert
|
||||
_insert_data();
|
||||
|
||||
for (int i = 200; i < 300; i++) {
|
||||
_check_not_found(i);
|
||||
}
|
||||
|
||||
for (int i = 50; i < 100; i++) {
|
||||
_check_found(i);
|
||||
}
|
||||
|
||||
ASSERT_EQ(_cache->lookup_count(), 150);
|
||||
ASSERT_EQ(_cache->hit_count(), 50);
|
||||
ObjectCacheMetrics metrics = _cache->metrics();
|
||||
ASSERT_EQ(metrics.lookup_count, 150);
|
||||
ASSERT_EQ(metrics.hit_count, 50);
|
||||
ASSERT_EQ(metrics.usage, _kv_size * 128);
|
||||
ASSERT_EQ(metrics.capacity, _capacity);
|
||||
ASSERT_EQ(metrics.object_item_count, 0);
|
||||
}
|
||||
} // namespace starrocks
|
||||
|
|
|
|||
|
|
@ -71,7 +71,7 @@ TEST_F(StoragePageCacheTest, insert_with_deleter) {
|
|||
std::string key("123");
|
||||
auto* value = new Value("value_of_123");
|
||||
PageCacheHandle handle;
|
||||
ObjectCacheWriteOptions opts;
|
||||
MemCacheWriteOptions opts;
|
||||
|
||||
ASSERT_OK(_page_cache->insert(key, (void*)value, 15, deleter, opts, &handle));
|
||||
Value* handle_value = (Value*)handle.data();
|
||||
|
|
@ -95,7 +95,7 @@ TEST_F(StoragePageCacheTest, normal) {
|
|||
auto data = std::make_unique<std::vector<uint8_t>>(1024);
|
||||
PageCacheHandle handle;
|
||||
|
||||
ObjectCacheWriteOptions opts;
|
||||
MemCacheWriteOptions opts;
|
||||
ASSERT_OK(_page_cache->insert(key, data.get(), opts, &handle));
|
||||
ASSERT_EQ(handle.data(), data.get());
|
||||
auto* check_data = data.release();
|
||||
|
|
@ -110,7 +110,7 @@ TEST_F(StoragePageCacheTest, normal) {
|
|||
auto data = std::make_unique<std::vector<uint8_t>>(1024);
|
||||
PageCacheHandle handle;
|
||||
|
||||
ObjectCacheWriteOptions opts{.priority = true};
|
||||
MemCacheWriteOptions opts{.priority = true};
|
||||
ASSERT_OK(_page_cache->insert(key, data.get(), opts, &handle));
|
||||
ASSERT_EQ(handle.data(), data.get());
|
||||
data.release();
|
||||
|
|
@ -124,7 +124,7 @@ TEST_F(StoragePageCacheTest, normal) {
|
|||
key.append(std::to_string(i));
|
||||
auto data = std::make_unique<std::vector<uint8_t>>(1024);
|
||||
PageCacheHandle handle;
|
||||
ObjectCacheWriteOptions opts;
|
||||
MemCacheWriteOptions opts;
|
||||
ASSERT_OK(_page_cache->insert(key, data.get(), opts, &handle));
|
||||
data.release();
|
||||
}
|
||||
|
|
@ -191,7 +191,7 @@ TEST_F(StoragePageCacheTest, metrics) {
|
|||
{
|
||||
// Insert a piece of data, but the application layer does not release it.
|
||||
auto data = std::make_unique<std::vector<uint8_t>>(1024);
|
||||
ObjectCacheWriteOptions opts;
|
||||
MemCacheWriteOptions opts;
|
||||
ASSERT_OK(_page_cache->insert(key1, data.get(), opts, &handle1));
|
||||
data.release();
|
||||
}
|
||||
|
|
@ -200,7 +200,7 @@ TEST_F(StoragePageCacheTest, metrics) {
|
|||
// Insert another piece of data and release it from the application layer.
|
||||
auto data = std::make_unique<std::vector<uint8_t>>(1024);
|
||||
PageCacheHandle handle2;
|
||||
ObjectCacheWriteOptions opts;
|
||||
MemCacheWriteOptions opts;
|
||||
ASSERT_OK(_page_cache->insert(key2, data.get(), opts, &handle2));
|
||||
data.release();
|
||||
}
|
||||
|
|
|
|||
|
|
@ -15,8 +15,8 @@
|
|||
#include <fmt/format.h>
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include "cache/cache_options.h"
|
||||
#include "cache/disk_cache/io_buffer.h"
|
||||
#include "cache/disk_cache/local_disk_cache_engine.h"
|
||||
#include "cache/peer_cache_engine.h"
|
||||
#include "common/statusor.h"
|
||||
|
||||
|
|
@ -50,7 +50,7 @@ TEST_F(PeerCacheTest, io_adaptor) {
|
|||
peer_cache.record_read_remote(1024, 1000);
|
||||
}
|
||||
|
||||
ReadCacheOptions read_options;
|
||||
DiskCacheReadOptions read_options;
|
||||
read_options.use_adaptor = true;
|
||||
IOBuffer buf;
|
||||
st = peer_cache.read("test_key", 0, 100, &buf, &read_options);
|
||||
|
|
|
|||
|
|
@ -31,6 +31,7 @@
|
|||
#include "storage/sstable/table.h"
|
||||
#include "storage/sstable/table_builder.h"
|
||||
#include "testutil/assert.h"
|
||||
#include "util/lru_cache.h"
|
||||
#include "util/phmap/btree.h"
|
||||
|
||||
namespace starrocks::lake {
|
||||
|
|
|
|||
|
|
@ -39,7 +39,7 @@ void PageHandleTest::SetUp() {
|
|||
}
|
||||
|
||||
TEST_F(PageHandleTest, test_operator_not_owner) {
|
||||
ObjectCacheWriteOptions opts;
|
||||
MemCacheWriteOptions opts;
|
||||
|
||||
std::vector<uint8_t>* p1 = new std::vector<uint8_t>(2);
|
||||
(*p1)[0] = 0;
|
||||
|
|
|
|||
|
|
@ -296,7 +296,7 @@ TEST_F(StarRocksMetricsTest, PageCacheMetrics) {
|
|||
std::string key("abc0");
|
||||
PageCacheHandle handle;
|
||||
auto data = std::make_unique<std::vector<uint8_t>>(1024);
|
||||
ObjectCacheWriteOptions opts;
|
||||
MemCacheWriteOptions opts;
|
||||
ASSERT_OK(_page_cache->insert(key, data.get(), opts, &handle));
|
||||
ASSERT_TRUE(_page_cache->lookup(key, &handle));
|
||||
data.release();
|
||||
|
|
|
|||
Loading…
Reference in New Issue