[Refactor] Remove core arena mem allocator (#60221)

Branch-3.3 (pr: #51263) has already set the default value of config::chunk_reserved_bytes_limit to 0, and there is no performance issue, so we finally removed the core memory allocator in the main branch.

What I'm doing:
Remove core arena mem allocator

Signed-off-by: trueeyu <lxhhust350@qq.com>
This commit is contained in:
trueeyu 2025-06-25 13:21:52 +08:00 committed by GitHub
parent cc73cf9575
commit 2ae06f73fb
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
15 changed files with 36 additions and 236 deletions

View File

@ -542,11 +542,6 @@ CONF_Bool(disable_mem_pools, "false");
// to a relative large number or the performace is very very bad.
CONF_Bool(use_mmap_allocate_chunk, "false");
// Chunk Allocator's reserved bytes limit,
// Default value is 2GB, increase this variable can improve performance, but will
// acquire more free memory which can not be used by other modules
CONF_Int64(chunk_reserved_bytes_limit, "0");
// for pprof
CONF_String(pprof_profile_dir, "${STARROCKS_HOME}/log");

View File

@ -139,15 +139,15 @@ void calculate_metrics(void* arg_this) {
LOG(INFO) << fmt::format(
"Current memory statistics: process({}), query_pool({}), load({}), "
"metadata({}), compaction({}), schema_change({}), "
"page_cache({}), update({}), chunk_allocator({}), passthrough({}), clone({}), consistency({}), "
"page_cache({}), update({}), passthrough({}), clone({}), consistency({}), "
"datacache({}), jit({})",
mem_metrics->process_mem_bytes.value(), mem_metrics->query_mem_bytes.value(),
mem_metrics->load_mem_bytes.value(), mem_metrics->metadata_mem_bytes.value(),
mem_metrics->compaction_mem_bytes.value(), mem_metrics->schema_change_mem_bytes.value(),
mem_metrics->storage_page_cache_mem_bytes.value(), mem_metrics->update_mem_bytes.value(),
mem_metrics->chunk_allocator_mem_bytes.value(), mem_metrics->passthrough_mem_bytes.value(),
mem_metrics->clone_mem_bytes.value(), mem_metrics->consistency_mem_bytes.value(),
mem_metrics->datacache_mem_bytes.value(), mem_metrics->jit_cache_mem_bytes.value());
mem_metrics->passthrough_mem_bytes.value(), mem_metrics->clone_mem_bytes.value(),
mem_metrics->consistency_mem_bytes.value(), mem_metrics->datacache_mem_bytes.value(),
mem_metrics->jit_cache_mem_bytes.value());
StarRocksMetrics::instance()->table_metrics_mgr()->cleanup();
nap_sleep(15, [daemon] { return daemon->stopped(); });

View File

@ -233,7 +233,6 @@ Status GlobalEnv::_init_mem_tracker() {
int32_t update_mem_percent = std::max(std::min(100, config::update_memory_limit_percent), 0);
_update_mem_tracker = regist_tracker(MemTrackerType::UPDATE, bytes_limit * update_mem_percent / 100, nullptr);
_update_mem_tracker->set_level(2);
_chunk_allocator_mem_tracker = regist_tracker(MemTrackerType::CHUNK_ALLOCATOR, -1, process_mem_tracker());
_passthrough_mem_tracker = regist_tracker(MemTrackerType::PASSTHROUGH, -1, nullptr);
_passthrough_mem_tracker->set_level(2);
_clone_mem_tracker = regist_tracker(MemTrackerType::CLONE, -1, process_mem_tracker());
@ -244,7 +243,7 @@ Status GlobalEnv::_init_mem_tracker() {
_poco_connection_pool_mem_tracker = regist_tracker(MemTrackerType::POCO_CONNECTION_POOL, -1, process_mem_tracker());
_replication_mem_tracker = regist_tracker(MemTrackerType::REPLICATION, -1, process_mem_tracker());
MemChunkAllocator::init_instance(_chunk_allocator_mem_tracker.get(), config::chunk_reserved_bytes_limit);
MemChunkAllocator::init_metrics();
return Status::OK();
}

View File

@ -151,7 +151,6 @@ public:
MemTracker* page_cache_mem_tracker() { return _page_cache_mem_tracker.get(); }
MemTracker* jit_cache_mem_tracker() { return _jit_cache_mem_tracker.get(); }
MemTracker* update_mem_tracker() { return _update_mem_tracker.get(); }
MemTracker* chunk_allocator_mem_tracker() { return _chunk_allocator_mem_tracker.get(); }
MemTracker* passthrough_mem_tracker() { return _passthrough_mem_tracker.get(); }
MemTracker* clone_mem_tracker() { return _clone_mem_tracker.get(); }
MemTracker* consistency_mem_tracker() { return _consistency_mem_tracker.get(); }
@ -220,7 +219,6 @@ private:
// The memory tracker for update manager
std::shared_ptr<MemTracker> _update_mem_tracker;
std::shared_ptr<MemTracker> _chunk_allocator_mem_tracker;
// record mem usage in passthrough
std::shared_ptr<MemTracker> _passthrough_mem_tracker;

View File

@ -61,7 +61,7 @@ MemPool::~MemPool() {
int64_t total_bytes_released = 0;
for (auto& chunk : chunks_) {
total_bytes_released += chunk.chunk.size;
MemChunkAllocator::instance()->free(chunk.chunk);
MemChunkAllocator::free(chunk.chunk);
}
StarRocksMetrics::instance()->memory_pool_bytes_total.increment(-total_bytes_released);
}
@ -80,7 +80,7 @@ void MemPool::free_all() {
int64_t total_bytes_released = 0;
for (auto& chunk : chunks_) {
total_bytes_released += chunk.chunk.size;
MemChunkAllocator::instance()->free(chunk.chunk);
MemChunkAllocator::free(chunk.chunk);
}
chunks_.clear();
next_chunk_size_ = INITIAL_CHUNK_SIZE;
@ -132,7 +132,7 @@ bool MemPool::find_chunk(size_t min_size, bool check_limits) {
// Allocate a new chunk. Return early if allocate fails.
MemChunk chunk;
if (!MemChunkAllocator::instance()->allocate(chunk_size, &chunk)) {
if (!MemChunkAllocator::allocate(chunk_size, &chunk)) {
if (tls_thread_status.is_catched()) {
throw std::bad_alloc();
} else {

View File

@ -70,7 +70,6 @@ static std::vector<std::pair<MemTrackerType, std::string>> s_mem_types = {
{MemTrackerType::PAGE_CACHE, "page_cache"},
{MemTrackerType::JIT_CACHE, "jit_cache"},
{MemTrackerType::UPDATE, "update"},
{MemTrackerType::CHUNK_ALLOCATOR, "chunk_allocator"},
{MemTrackerType::CLONE, "clone"},
{MemTrackerType::DATACACHE, "datacache"},
{MemTrackerType::POCO_CONNECTION_POOL, "poco_connection_pool"},

View File

@ -109,7 +109,6 @@ enum class MemTrackerType {
PAGE_CACHE,
JIT_CACHE,
UPDATE,
CHUNK_ALLOCATOR,
CLONE,
DATACACHE,
POCO_CONNECTION_POOL,

View File

@ -34,211 +34,61 @@
#include "runtime/memory/mem_chunk_allocator.h"
#include <memory>
#include <mutex>
#include "gutil/dynamic_annotations.h"
#include "runtime/current_thread.h"
#include "runtime/memory/mem_chunk.h"
#include "runtime/memory/system_allocator.h"
#include "util/bit_util.h"
#include "util/cpu_info.h"
#include "util/defer_op.h"
#include "util/failpoint/fail_point.h"
#include "util/runtime_profile.h"
#include "util/spinlock.h"
#include "util/starrocks_metrics.h"
namespace starrocks {
MemChunkAllocator* MemChunkAllocator::_s_instance = nullptr;
static IntCounter local_core_alloc_count(MetricUnit::NOUNIT);
static IntCounter other_core_alloc_count(MetricUnit::NOUNIT);
static IntCounter system_alloc_count(MetricUnit::NOUNIT);
static IntCounter system_free_count(MetricUnit::NOUNIT);
static IntCounter system_alloc_cost_ns(MetricUnit::NANOSECONDS);
static IntCounter system_free_cost_ns(MetricUnit::NANOSECONDS);
#ifdef BE_TEST
static std::mutex s_mutex;
MemChunkAllocator* MemChunkAllocator::instance() {
std::lock_guard<std::mutex> l(s_mutex);
if (_s_instance == nullptr) {
CpuInfo::init();
MemChunkAllocator::init_instance(nullptr, 4096);
}
return _s_instance;
}
#endif
// Keep free chunk's ptr in size separated free list.
// This class is thread-safe.
class ChunkArena {
public:
ChunkArena(MemTracker* mem_tracker) : _mem_tracker(mem_tracker), _chunk_lists(64) {}
~ChunkArena() {
for (int i = 0; i < 64; ++i) {
if (_chunk_lists[i].empty()) continue;
size_t size = (uint64_t)1 << i;
for (auto ptr : _chunk_lists[i]) {
SystemAllocator::free(_mem_tracker, ptr, size);
}
}
}
// Try to pop a free chunk from corresponding free list.
// Return true if success
bool pop_free_chunk(size_t size, uint8_t** ptr) {
int idx = BitUtil::Log2Ceiling64(size);
auto& free_list = _chunk_lists[idx];
std::lock_guard<SpinLock> l(_lock);
if (free_list.empty()) {
return false;
}
*ptr = free_list.back();
free_list.pop_back();
ASAN_UNPOISON_MEMORY_REGION(*ptr, size);
return true;
}
void push_free_chunk(uint8_t* ptr, size_t size) {
int idx = BitUtil::Log2Ceiling64(size);
// Poison this chunk to make asan can detect invalid access
ASAN_POISON_MEMORY_REGION(ptr, size);
std::lock_guard<SpinLock> l(_lock);
_chunk_lists[idx].push_back(ptr);
}
private:
MemTracker* _mem_tracker = nullptr;
SpinLock _lock;
std::vector<std::vector<uint8_t*>> _chunk_lists;
};
void MemChunkAllocator::init_instance(MemTracker* mem_tracker, size_t reserve_limit) {
if (_s_instance != nullptr) return;
_s_instance = new MemChunkAllocator(mem_tracker, reserve_limit);
void MemChunkAllocator::init_metrics() {
#define REGISTER_METIRC_WITH_NAME(name, metric) StarRocksMetrics::instance()->metrics()->register_metric(#name, &metric)
#define REGISTER_METIRC_WITH_PREFIX(prefix, name) REGISTER_METIRC_WITH_NAME(prefix##name, name)
#define REGISTER_METIRC(name) REGISTER_METIRC_WITH_PREFIX(chunk_pool_, name)
REGISTER_METIRC(local_core_alloc_count);
REGISTER_METIRC(other_core_alloc_count);
REGISTER_METIRC(system_alloc_count);
REGISTER_METIRC(system_free_count);
REGISTER_METIRC(system_alloc_cost_ns);
REGISTER_METIRC(system_free_cost_ns);
}
MemChunkAllocator::MemChunkAllocator(MemTracker* mem_tracker, size_t reserve_limit)
: _mem_tracker(mem_tracker),
_reserve_bytes_limit(reserve_limit),
_reserved_bytes(0),
_arenas(CpuInfo::get_max_num_cores()) {
for (auto& _arena : _arenas) {
_arena = std::make_unique<ChunkArena>(_mem_tracker);
}
}
bool MemChunkAllocator::allocate(size_t size, MemChunk* chunk) {
FAIL_POINT_TRIGGER_RETURN(random_error, false);
bool ret = true;
#ifndef BE_TEST
MemTracker* prev_tracker = tls_thread_status.set_mem_tracker(_mem_tracker);
DeferOp op([&] {
if (ret) {
if (LIKELY(_mem_tracker != nullptr)) {
_mem_tracker->release(chunk->size);
}
if (LIKELY(prev_tracker != nullptr)) {
prev_tracker->consume(chunk->size);
}
}
tls_thread_status.set_mem_tracker(prev_tracker);
});
#endif
// fast path: allocate from current core arena
int core_id = CpuInfo::get_current_core();
chunk->size = size;
chunk->core_id = core_id;
if (_arenas[core_id]->pop_free_chunk(size, &chunk->data)) {
_reserved_bytes.fetch_sub(size);
local_core_alloc_count.increment(1);
ret = true;
return ret;
}
if (_reserved_bytes > size) {
// try to allocate from other core's arena
++core_id;
for (int i = 1; i < _arenas.size(); ++i, ++core_id) {
if (_arenas[core_id % _arenas.size()]->pop_free_chunk(size, &chunk->data)) {
_reserved_bytes.fetch_sub(size);
other_core_alloc_count.increment(1);
// reset chunk's core_id to other
chunk->core_id = core_id % _arenas.size();
ret = true;
return ret;
}
}
}
int64_t cost_ns = 0;
{
SCOPED_RAW_TIMER(&cost_ns);
// allocate from system allocator
chunk->data = SystemAllocator::allocate(_mem_tracker, size);
chunk->data = SystemAllocator::allocate(tls_thread_status.mem_tracker(), size);
}
system_alloc_count.increment(1);
system_alloc_cost_ns.increment(cost_ns);
if (chunk->data == nullptr) {
ret = false;
return ret;
return false;
} else {
return true;
}
ret = true;
return ret;
}
void MemChunkAllocator::free(const MemChunk& chunk) {
#ifndef BE_TEST
MemTracker* prev_tracker = tls_thread_status.set_mem_tracker(_mem_tracker);
DeferOp op([&] {
int64_t chunk_size = chunk.size;
if (LIKELY(prev_tracker != nullptr)) {
prev_tracker->release(chunk_size);
}
if (LIKELY(_mem_tracker != nullptr)) {
_mem_tracker->consume(chunk_size);
}
tls_thread_status.set_mem_tracker(prev_tracker);
});
#endif
int64_t old_reserved_bytes = _reserved_bytes;
int64_t new_reserved_bytes = 0;
do {
new_reserved_bytes = old_reserved_bytes + chunk.size;
if (new_reserved_bytes > _reserve_bytes_limit) {
int64_t cost_ns = 0;
{
SCOPED_RAW_TIMER(&cost_ns);
SystemAllocator::free(_mem_tracker, chunk.data, chunk.size);
}
system_free_count.increment(1);
system_free_cost_ns.increment(cost_ns);
return;
}
} while (!_reserved_bytes.compare_exchange_weak(old_reserved_bytes, new_reserved_bytes));
_arenas[chunk.core_id]->push_free_chunk(chunk.data, chunk.size);
int64_t cost_ns = 0;
{
SCOPED_RAW_TIMER(&cost_ns);
SystemAllocator::free(tls_thread_status.mem_tracker(), chunk.data, chunk.size);
}
system_free_count.increment(1);
system_free_cost_ns.increment(cost_ns);
}
} // namespace starrocks

View File

@ -47,54 +47,18 @@ class ChunkArena;
class MemTracker;
// Used to allocate memory with power-of-two length.
// This Allocator allocate memory from system and cache free chunks for
// later use.
//
// MemChunkAllocator has one ChunkArena for each CPU core, it will try to allocate
// memory from current core arena firstly. In this way, there will be no lock contention
// between concurrently-running threads. If this fails, MemChunkAllocator will try to allocate
// memory from other core's arena.
//
// Memory Reservation
// MemChunkAllocator has a limit about how much free chunk bytes it can reserve, above which
// chunk will be released to system memory. For the worst case, when the limits is 0, it will
// act as allocating directly from system.
//
// ChunkArena will keep a separate free list for each chunk size. In common case, chunk will
// be allocated from current core arena. In this case, there is no lock contention.
//
// Must call CpuInfo::init() and StarRocksMetrics::instance()->initialize() to achieve good performance
// before first object is created. And call init_instance() before use instance is called.
// This Allocator allocates memory from system.
class MemChunkAllocator {
public:
static void init_instance(MemTracker* mem_tracker, size_t reserve_limit);
#ifdef BE_TEST
static MemChunkAllocator* instance();
#else
static MemChunkAllocator* instance() { return _s_instance; }
#endif
MemChunkAllocator(MemTracker* mem_tracker, size_t reserve_limit);
static void init_metrics();
// Allocate a MemChunk with a power-of-two length "size".
// Return true if success and allocated chunk is saved in "chunk".
// Otherwise return false.
bool allocate(size_t size, MemChunk* chunk);
static bool allocate(size_t size, MemChunk* chunk);
// Free chunk allocated from this allocator
void free(const MemChunk& chunk);
void set_mem_tracker(MemTracker* mem_tracker) { _mem_tracker = mem_tracker; }
private:
static MemChunkAllocator* _s_instance;
MemTracker* _mem_tracker = nullptr;
size_t _reserve_bytes_limit;
std::atomic<int64_t> _reserved_bytes;
// each core has a ChunkArena
std::vector<std::unique_ptr<ChunkArena>> _arenas;
static void free(const MemChunk& chunk);
};
} // namespace starrocks

View File

@ -230,7 +230,6 @@ void bind_exec_env(ForeignModule& m) {
REG_METHOD(GlobalEnv, page_cache_mem_tracker);
REG_METHOD(GlobalEnv, jit_cache_mem_tracker);
REG_METHOD(GlobalEnv, update_mem_tracker);
REG_METHOD(GlobalEnv, chunk_allocator_mem_tracker);
REG_METHOD(GlobalEnv, passthrough_mem_tracker);
REG_METHOD(GlobalEnv, clone_mem_tracker);
REG_METHOD(GlobalEnv, consistency_mem_tracker);

View File

@ -1055,7 +1055,7 @@ int meta_tool_main(int argc, char** argv) {
google::ParseCommandLineFlags(&argc, &argv, true);
starrocks::date::init_date_cache();
starrocks::config::disable_storage_page_cache = true;
starrocks::MemChunkAllocator::init_instance(nullptr, 2ul * 1024 * 1024 * 1024);
starrocks::MemChunkAllocator::init_metrics();
if (empty_args || FLAGS_operation.empty()) {
show_usage();

View File

@ -59,7 +59,7 @@ std::string HyperLogLog::empty() {
HyperLogLog::HyperLogLog(const HyperLogLog& other) : _type(other._type), _hash_set(other._hash_set) {
if (other._registers.data != nullptr) {
MemChunkAllocator::instance()->allocate(HLL_REGISTERS_COUNT, &_registers);
MemChunkAllocator::allocate(HLL_REGISTERS_COUNT, &_registers);
DCHECK_NE(_registers.data, nullptr);
DCHECK_EQ(_registers.size, HLL_REGISTERS_COUNT);
memcpy(_registers.data, other._registers.data, HLL_REGISTERS_COUNT);
@ -72,12 +72,12 @@ HyperLogLog& HyperLogLog::operator=(const HyperLogLog& other) {
this->_hash_set = other._hash_set;
if (_registers.data != nullptr) {
MemChunkAllocator::instance()->free(_registers);
MemChunkAllocator::free(_registers);
_registers.data = nullptr;
}
if (other._registers.data != nullptr) {
MemChunkAllocator::instance()->allocate(HLL_REGISTERS_COUNT, &_registers);
MemChunkAllocator::allocate(HLL_REGISTERS_COUNT, &_registers);
DCHECK_NE(_registers.data, nullptr);
DCHECK_EQ(_registers.size, HLL_REGISTERS_COUNT);
memcpy(_registers.data, other._registers.data, HLL_REGISTERS_COUNT);
@ -99,7 +99,7 @@ HyperLogLog& HyperLogLog::operator=(HyperLogLog&& other) noexcept {
this->_hash_set = std::move(other._hash_set);
if (_registers.data != nullptr) {
MemChunkAllocator::instance()->free(_registers);
MemChunkAllocator::free(_registers);
}
_registers = other._registers;
@ -123,7 +123,7 @@ HyperLogLog::HyperLogLog(const Slice& src) {
HyperLogLog::~HyperLogLog() {
if (_registers.data != nullptr) {
DCHECK_EQ(_registers.size, HLL_REGISTERS_COUNT);
MemChunkAllocator::instance()->free(_registers);
MemChunkAllocator::free(_registers);
}
}
@ -132,7 +132,7 @@ HyperLogLog::~HyperLogLog() {
void HyperLogLog::_convert_explicit_to_register() {
DCHECK(_type == HLL_DATA_EXPLICIT) << "_type(" << _type << ") should be explicit(" << HLL_DATA_EXPLICIT << ")";
DCHECK_EQ(_registers.data, nullptr);
MemChunkAllocator::instance()->allocate(HLL_REGISTERS_COUNT, &_registers);
MemChunkAllocator::allocate(HLL_REGISTERS_COUNT, &_registers);
DCHECK_NE(_registers.data, nullptr);
DCHECK_EQ(_registers.size, HLL_REGISTERS_COUNT);
memset(_registers.data, 0, HLL_REGISTERS_COUNT);
@ -226,7 +226,7 @@ void HyperLogLog::merge(const HyperLogLog& other) {
case HLL_DATA_SPARSE:
case HLL_DATA_FULL:
DCHECK_EQ(_registers.data, nullptr);
MemChunkAllocator::instance()->allocate(HLL_REGISTERS_COUNT, &_registers);
MemChunkAllocator::allocate(HLL_REGISTERS_COUNT, &_registers);
DCHECK_NE(_registers.data, nullptr);
DCHECK_EQ(_registers.size, HLL_REGISTERS_COUNT);
memcpy(_registers.data, other._registers.data, HLL_REGISTERS_COUNT);
@ -423,7 +423,7 @@ bool HyperLogLog::deserialize(const Slice& slice) {
}
case HLL_DATA_SPARSE: {
DCHECK_EQ(_registers.data, nullptr);
MemChunkAllocator::instance()->allocate(HLL_REGISTERS_COUNT, &_registers);
MemChunkAllocator::allocate(HLL_REGISTERS_COUNT, &_registers);
DCHECK_NE(_registers.data, nullptr);
DCHECK_EQ(_registers.size, HLL_REGISTERS_COUNT);
memset(_registers.data, 0, HLL_REGISTERS_COUNT);
@ -442,7 +442,7 @@ bool HyperLogLog::deserialize(const Slice& slice) {
}
case HLL_DATA_FULL: {
DCHECK_EQ(_registers.data, nullptr);
MemChunkAllocator::instance()->allocate(HLL_REGISTERS_COUNT, &_registers);
MemChunkAllocator::allocate(HLL_REGISTERS_COUNT, &_registers);
DCHECK_NE(_registers.data, nullptr);
DCHECK_EQ(_registers.size, HLL_REGISTERS_COUNT);
// 2+ : hll register value

View File

@ -291,7 +291,6 @@ void SystemMetrics::_install_memory_metrics(MetricRegistry* registry) {
registry->register_metric("storage_page_cache_mem_bytes", &_memory_metrics->storage_page_cache_mem_bytes);
registry->register_metric("jit_cache_mem_bytes", &_memory_metrics->jit_cache_mem_bytes);
registry->register_metric("update_mem_bytes", &_memory_metrics->update_mem_bytes);
registry->register_metric("chunk_allocator_mem_bytes", &_memory_metrics->chunk_allocator_mem_bytes);
registry->register_metric("clone_mem_bytes", &_memory_metrics->clone_mem_bytes);
registry->register_metric("consistency_mem_bytes", &_memory_metrics->consistency_mem_bytes);
registry->register_metric("datacache_mem_bytes", &_memory_metrics->datacache_mem_bytes);
@ -386,7 +385,6 @@ void SystemMetrics::_update_memory_metrics() {
SET_MEM_METRIC_VALUE(page_cache_mem_tracker, storage_page_cache_mem_bytes)
SET_MEM_METRIC_VALUE(jit_cache_mem_tracker, jit_cache_mem_bytes)
SET_MEM_METRIC_VALUE(update_mem_tracker, update_mem_bytes)
SET_MEM_METRIC_VALUE(chunk_allocator_mem_tracker, chunk_allocator_mem_bytes)
SET_MEM_METRIC_VALUE(passthrough_mem_tracker, passthrough_mem_bytes)
SET_MEM_METRIC_VALUE(clone_mem_tracker, clone_mem_bytes)
SET_MEM_METRIC_VALUE(consistency_mem_tracker, consistency_mem_bytes)

View File

@ -74,7 +74,6 @@ public:
METRIC_DEFINE_INT_GAUGE(storage_page_cache_mem_bytes, MetricUnit::BYTES);
METRIC_DEFINE_INT_GAUGE(jit_cache_mem_bytes, MetricUnit::BYTES);
METRIC_DEFINE_INT_GAUGE(update_mem_bytes, MetricUnit::BYTES);
METRIC_DEFINE_INT_GAUGE(chunk_allocator_mem_bytes, MetricUnit::BYTES);
METRIC_DEFINE_INT_GAUGE(passthrough_mem_bytes, MetricUnit::BYTES);
METRIC_DEFINE_INT_GAUGE(clone_mem_bytes, MetricUnit::BYTES);
METRIC_DEFINE_INT_GAUGE(consistency_mem_bytes, MetricUnit::BYTES);

View File

@ -45,10 +45,10 @@ TEST(MemChunkAllocatorTest, Normal) {
config::use_mmap_allocate_chunk = true;
for (size_t size = 4096; size <= 1024 * 1024; size <<= 1) {
MemChunk chunk;
ASSERT_TRUE(MemChunkAllocator::instance()->allocate(size, &chunk));
ASSERT_TRUE(MemChunkAllocator::allocate(size, &chunk));
ASSERT_NE(nullptr, chunk.data);
ASSERT_EQ(size, chunk.size);
MemChunkAllocator::instance()->free(chunk);
MemChunkAllocator::free(chunk);
}
}
} // namespace starrocks