mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
[Resubmit] Remove mmap/mremap/munmap from Allocator.h (#52792)
* Revert "Revert "Remove `mmap/mremap/munmap` from Allocator.h" (#52589)"
This reverts commit 0a838dc6d1
.
* pre-fault pages
* upd contrib to merge commit
* limit log rate
* Update Allocator.h
---------
Co-authored-by: Alexey Milovidov <milovidov@clickhouse.com>
This commit is contained in:
parent
bdb52eefc4
commit
079b52fa93
2
contrib/sysroot
vendored
2
contrib/sysroot
vendored
@ -1 +1 @@
|
||||
Subproject commit e0d1b64da666afbfaa6f1ee0487c33f3fd2cd5cb
|
||||
Subproject commit b5fcabb24d28fc33024291b2c6c1abd807c7dba8
|
@ -1,24 +1,8 @@
|
||||
#include "Allocator.h"
|
||||
|
||||
/** Keep definition of this constant in cpp file; otherwise its value
|
||||
* is inlined into allocator code making it impossible to override it
|
||||
* in third-party code.
|
||||
*
|
||||
* Note: extern may seem redundant, but is actually needed due to bug in GCC.
|
||||
* See also: https://gcc.gnu.org/legacy-ml/gcc-help/2017-12/msg00021.html
|
||||
*/
|
||||
#ifdef NDEBUG
|
||||
__attribute__((__weak__)) extern const size_t MMAP_THRESHOLD = 128 * (1ULL << 20);
|
||||
#else
|
||||
/**
|
||||
* In debug build, use small mmap threshold to reproduce more memory
|
||||
* stomping bugs. Along with ASLR it will hopefully detect more issues than
|
||||
* ASan. The program may fail due to the limit on number of memory mappings.
|
||||
*
|
||||
* Not too small to avoid too quick exhaust of memory mappings.
|
||||
*/
|
||||
__attribute__((__weak__)) extern const size_t MMAP_THRESHOLD = 16384;
|
||||
#endif
|
||||
|
||||
/// Constant is chosen almost arbitrarily, what I observed is 128KB is too small, 1MB is almost indistinguishable from 64MB and 1GB is too large.
|
||||
extern const size_t POPULATE_THRESHOLD = 16 * 1024 * 1024;
|
||||
|
||||
template class Allocator<false, false>;
|
||||
template class Allocator<true, false>;
|
||||
|
@ -20,12 +20,6 @@
|
||||
#include <sys/mman.h>
|
||||
|
||||
#include <Core/Defines.h>
|
||||
#if defined(THREAD_SANITIZER) || defined(MEMORY_SANITIZER)
|
||||
/// Thread and memory sanitizers do not intercept mremap. The usage of
|
||||
/// mremap will lead to false positives.
|
||||
#define DISABLE_MREMAP 1
|
||||
#endif
|
||||
#include <base/mremap.h>
|
||||
#include <base/getPageSize.h>
|
||||
|
||||
#include <Common/CurrentMemoryTracker.h>
|
||||
@ -35,52 +29,33 @@
|
||||
|
||||
#include <Common/Allocator_fwd.h>
|
||||
|
||||
#include <base/errnoToString.h>
|
||||
#include <Poco/Logger.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
/// Required for older Darwin builds, that lack definition of MAP_ANONYMOUS
|
||||
#ifndef MAP_ANONYMOUS
|
||||
#define MAP_ANONYMOUS MAP_ANON
|
||||
#endif
|
||||
|
||||
/**
|
||||
* Many modern allocators (for example, tcmalloc) do not do a mremap for
|
||||
* realloc, even in case of large enough chunks of memory. Although this allows
|
||||
* you to increase performance and reduce memory consumption during realloc.
|
||||
* To fix this, we do mremap manually if the chunk of memory is large enough.
|
||||
* The threshold (64 MB) is chosen quite large, since changing the address
|
||||
* space is very slow, especially in the case of a large number of threads. We
|
||||
* expect that the set of operations mmap/something to do/mremap can only be
|
||||
* performed about 1000 times per second.
|
||||
*
|
||||
* P.S. This is also required, because tcmalloc can not allocate a chunk of
|
||||
* memory greater than 16 GB.
|
||||
*
|
||||
* P.P.S. Note that MMAP_THRESHOLD symbol is intentionally made weak. It allows
|
||||
* to override it during linkage when using ClickHouse as a library in
|
||||
* third-party applications which may already use own allocator doing mmaps
|
||||
* in the implementation of alloc/realloc.
|
||||
*/
|
||||
extern const size_t MMAP_THRESHOLD;
|
||||
extern const size_t POPULATE_THRESHOLD;
|
||||
|
||||
static constexpr size_t MALLOC_MIN_ALIGNMENT = 8;
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric MMappedAllocs;
|
||||
extern const Metric MMappedAllocBytes;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int CANNOT_ALLOCATE_MEMORY;
|
||||
extern const int CANNOT_MUNMAP;
|
||||
extern const int CANNOT_MREMAP;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/** Previously there was a code which tried to use manual mmap and mremap (clickhouse_mremap.h) for large allocations/reallocations (64MB+).
|
||||
* Most modern allocators (including jemalloc) don't use mremap, so the idea was to take advantage from mremap system call for large reallocs.
|
||||
* Actually jemalloc had support for mremap, but it was intentionally removed from codebase https://github.com/jemalloc/jemalloc/commit/e2deab7a751c8080c2b2cdcfd7b11887332be1bb.
|
||||
* Our performance tests also shows that without manual mmap/mremap/munmap clickhouse is overall faster for about 1-2% and up to 5-7x for some types of queries.
|
||||
* That is why we don't do manual mmap/mremap/munmap here and completely rely on jemalloc for allocations of any size.
|
||||
*/
|
||||
|
||||
/** Responsible for allocating / freeing memory. Used, for example, in PODArray, Arena.
|
||||
* Also used in hash tables.
|
||||
* The interface is different from std::allocator
|
||||
@ -88,10 +63,8 @@ namespace ErrorCodes
|
||||
* - passing the size into the `free` method;
|
||||
* - by the presence of the `alignment` argument;
|
||||
* - the possibility of zeroing memory (used in hash tables);
|
||||
* - random hint address for mmap
|
||||
* - mmap_threshold for using mmap less or more
|
||||
*/
|
||||
template <bool clear_memory_, bool mmap_populate>
|
||||
template <bool clear_memory_, bool populate>
|
||||
class Allocator
|
||||
{
|
||||
public:
|
||||
@ -111,7 +84,7 @@ public:
|
||||
try
|
||||
{
|
||||
checkSize(size);
|
||||
freeNoTrack(buf, size);
|
||||
freeNoTrack(buf);
|
||||
auto trace = CurrentMemoryTracker::free(size);
|
||||
trace.onFree(buf, size);
|
||||
}
|
||||
@ -135,8 +108,7 @@ public:
|
||||
/// nothing to do.
|
||||
/// BTW, it's not possible to change alignment while doing realloc.
|
||||
}
|
||||
else if (old_size < MMAP_THRESHOLD && new_size < MMAP_THRESHOLD
|
||||
&& alignment <= MALLOC_MIN_ALIGNMENT)
|
||||
else if (alignment <= MALLOC_MIN_ALIGNMENT)
|
||||
{
|
||||
/// Resize malloc'd memory region with no special alignment requirement.
|
||||
auto trace_free = CurrentMemoryTracker::free(old_size);
|
||||
@ -145,7 +117,10 @@ public:
|
||||
|
||||
void * new_buf = ::realloc(buf, new_size);
|
||||
if (nullptr == new_buf)
|
||||
DB::throwFromErrno(fmt::format("Allocator: Cannot realloc from {} to {}.", ReadableSize(old_size), ReadableSize(new_size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
|
||||
{
|
||||
DB::throwFromErrno(
|
||||
fmt::format("Allocator: Cannot realloc from {} to {}.", ReadableSize(old_size), ReadableSize(new_size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
|
||||
}
|
||||
|
||||
buf = new_buf;
|
||||
trace_alloc.onAlloc(buf, new_size);
|
||||
@ -154,46 +129,18 @@ public:
|
||||
if (new_size > old_size)
|
||||
memset(reinterpret_cast<char *>(buf) + old_size, 0, new_size - old_size);
|
||||
}
|
||||
else if (old_size >= MMAP_THRESHOLD && new_size >= MMAP_THRESHOLD)
|
||||
{
|
||||
/// Resize mmap'd memory region.
|
||||
auto trace_free = CurrentMemoryTracker::free(old_size);
|
||||
auto trace_alloc = CurrentMemoryTracker::alloc(new_size);
|
||||
trace_free.onFree(buf, old_size);
|
||||
|
||||
// On apple and freebsd self-implemented mremap used (common/mremap.h)
|
||||
buf = clickhouse_mremap(buf, old_size, new_size, MREMAP_MAYMOVE,
|
||||
PROT_READ | PROT_WRITE, mmap_flags, -1, 0);
|
||||
if (MAP_FAILED == buf)
|
||||
DB::throwFromErrno(fmt::format("Allocator: Cannot mremap memory chunk from {} to {}.",
|
||||
ReadableSize(old_size), ReadableSize(new_size)), DB::ErrorCodes::CANNOT_MREMAP);
|
||||
|
||||
/// No need for zero-fill, because mmap guarantees it.
|
||||
trace_alloc.onAlloc(buf, new_size);
|
||||
}
|
||||
else if (new_size < MMAP_THRESHOLD)
|
||||
{
|
||||
/// Small allocs that requires a copy. Assume there's enough memory in system. Call CurrentMemoryTracker once.
|
||||
auto trace_free = CurrentMemoryTracker::free(old_size);
|
||||
auto trace_alloc = CurrentMemoryTracker::alloc(new_size);
|
||||
trace_free.onFree(buf, old_size);
|
||||
|
||||
void * new_buf = allocNoTrack(new_size, alignment);
|
||||
trace_alloc.onAlloc(buf, new_size);
|
||||
memcpy(new_buf, buf, std::min(old_size, new_size));
|
||||
freeNoTrack(buf, old_size);
|
||||
buf = new_buf;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Big allocs that requires a copy. MemoryTracker is called inside 'alloc', 'free' methods.
|
||||
|
||||
void * new_buf = alloc(new_size, alignment);
|
||||
memcpy(new_buf, buf, std::min(old_size, new_size));
|
||||
free(buf, old_size);
|
||||
buf = new_buf;
|
||||
}
|
||||
|
||||
if constexpr (populate)
|
||||
prefaultPages(buf, new_size);
|
||||
|
||||
return buf;
|
||||
}
|
||||
|
||||
@ -205,83 +152,42 @@ protected:
|
||||
|
||||
static constexpr bool clear_memory = clear_memory_;
|
||||
|
||||
// Freshly mmapped pages are copy-on-write references to a global zero page.
|
||||
// On the first write, a page fault occurs, and an actual writable page is
|
||||
// allocated. If we are going to use this memory soon, such as when resizing
|
||||
// hash tables, it makes sense to pre-fault the pages by passing
|
||||
// MAP_POPULATE to mmap(). This takes some time, but should be faster
|
||||
// overall than having a hot loop interrupted by page faults.
|
||||
// It is only supported on Linux.
|
||||
static constexpr int mmap_flags = MAP_PRIVATE | MAP_ANONYMOUS
|
||||
#if defined(OS_LINUX)
|
||||
| (mmap_populate ? MAP_POPULATE : 0)
|
||||
#endif
|
||||
;
|
||||
|
||||
private:
|
||||
void * allocNoTrack(size_t size, size_t alignment)
|
||||
{
|
||||
void * buf;
|
||||
size_t mmap_min_alignment = ::getPageSize();
|
||||
|
||||
if (size >= MMAP_THRESHOLD)
|
||||
if (alignment <= MALLOC_MIN_ALIGNMENT)
|
||||
{
|
||||
if (alignment > mmap_min_alignment)
|
||||
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS,
|
||||
"Too large alignment {}: more than page size when allocating {}.",
|
||||
ReadableSize(alignment), ReadableSize(size));
|
||||
if constexpr (clear_memory)
|
||||
buf = ::calloc(size, 1);
|
||||
else
|
||||
buf = ::malloc(size);
|
||||
|
||||
buf = mmap(getMmapHint(), size, PROT_READ | PROT_WRITE,
|
||||
mmap_flags, -1, 0);
|
||||
if (MAP_FAILED == buf)
|
||||
DB::throwFromErrno(fmt::format("Allocator: Cannot mmap {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
|
||||
/// No need for zero-fill, because mmap guarantees it.
|
||||
|
||||
CurrentMetrics::add(CurrentMetrics::MMappedAllocs);
|
||||
CurrentMetrics::add(CurrentMetrics::MMappedAllocBytes, size);
|
||||
if (nullptr == buf)
|
||||
DB::throwFromErrno(fmt::format("Allocator: Cannot malloc {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (alignment <= MALLOC_MIN_ALIGNMENT)
|
||||
{
|
||||
if constexpr (clear_memory)
|
||||
buf = ::calloc(size, 1);
|
||||
else
|
||||
buf = ::malloc(size);
|
||||
buf = nullptr;
|
||||
int res = posix_memalign(&buf, alignment, size);
|
||||
|
||||
if (nullptr == buf)
|
||||
DB::throwFromErrno(fmt::format("Allocator: Cannot malloc {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
|
||||
}
|
||||
else
|
||||
{
|
||||
buf = nullptr;
|
||||
int res = posix_memalign(&buf, alignment, size);
|
||||
if (0 != res)
|
||||
DB::throwFromErrno(fmt::format("Cannot allocate memory (posix_memalign) {}.", ReadableSize(size)),
|
||||
DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res);
|
||||
|
||||
if (0 != res)
|
||||
DB::throwFromErrno(fmt::format("Cannot allocate memory (posix_memalign) {}.", ReadableSize(size)),
|
||||
DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res);
|
||||
|
||||
if constexpr (clear_memory)
|
||||
memset(buf, 0, size);
|
||||
}
|
||||
if constexpr (clear_memory)
|
||||
memset(buf, 0, size);
|
||||
}
|
||||
|
||||
if constexpr (populate)
|
||||
prefaultPages(buf, size);
|
||||
|
||||
return buf;
|
||||
}
|
||||
|
||||
void freeNoTrack(void * buf, size_t size)
|
||||
void freeNoTrack(void * buf)
|
||||
{
|
||||
if (size >= MMAP_THRESHOLD)
|
||||
{
|
||||
if (0 != munmap(buf, size))
|
||||
DB::throwFromErrno(fmt::format("Allocator: Cannot munmap {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_MUNMAP);
|
||||
|
||||
CurrentMetrics::sub(CurrentMetrics::MMappedAllocs);
|
||||
CurrentMetrics::sub(CurrentMetrics::MMappedAllocBytes, size);
|
||||
}
|
||||
else
|
||||
{
|
||||
::free(buf);
|
||||
}
|
||||
::free(buf);
|
||||
}
|
||||
|
||||
void checkSize(size_t size)
|
||||
@ -291,20 +197,32 @@ private:
|
||||
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Too large size ({}) passed to allocator. It indicates an error.", size);
|
||||
}
|
||||
|
||||
#ifndef NDEBUG
|
||||
/// In debug builds, request mmap() at random addresses (a kind of ASLR), to
|
||||
/// reproduce more memory stomping bugs. Note that Linux doesn't do it by
|
||||
/// default. This may lead to worse TLB performance.
|
||||
void * getMmapHint()
|
||||
/// Address passed to madvise is required to be aligned to the page boundary.
|
||||
auto adjustToPageSize(void * buf, size_t len, size_t page_size)
|
||||
{
|
||||
return reinterpret_cast<void *>(std::uniform_int_distribution<intptr_t>(0x100000000000UL, 0x700000000000UL)(thread_local_rng));
|
||||
const uintptr_t address_numeric = reinterpret_cast<uintptr_t>(buf);
|
||||
const size_t next_page_start = ((address_numeric + page_size - 1) / page_size) * page_size;
|
||||
return std::make_pair(reinterpret_cast<void *>(next_page_start), len - (next_page_start - address_numeric));
|
||||
}
|
||||
#else
|
||||
void * getMmapHint()
|
||||
|
||||
void prefaultPages([[maybe_unused]] void * buf_, [[maybe_unused]] size_t len_)
|
||||
{
|
||||
return nullptr;
|
||||
}
|
||||
#if defined(MADV_POPULATE_WRITE)
|
||||
if (len_ < POPULATE_THRESHOLD)
|
||||
return;
|
||||
|
||||
static const size_t page_size = ::getPageSize();
|
||||
if (len_ < page_size) /// Rounded address should be still within [buf, buf + len).
|
||||
return;
|
||||
|
||||
auto [buf, len] = adjustToPageSize(buf_, len_, page_size);
|
||||
if (auto res = ::madvise(buf, len, MADV_POPULATE_WRITE); res < 0)
|
||||
LOG_TRACE(
|
||||
LogFrequencyLimiter(&Poco::Logger::get("Allocator"), 1),
|
||||
"Attempt to populate pages failed: {} (EINVAL is expected for kernels < 5.14)",
|
||||
errnoToString(res));
|
||||
#endif
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
|
@ -3,7 +3,7 @@
|
||||
* This file provides forward declarations for Allocator.
|
||||
*/
|
||||
|
||||
template <bool clear_memory_, bool mmap_populate = false>
|
||||
template <bool clear_memory_, bool populate = false>
|
||||
class Allocator;
|
||||
|
||||
template <typename Base, size_t N = 64, size_t Alignment = 1>
|
||||
|
@ -175,8 +175,6 @@
|
||||
M(PartsInMemory, "In-memory parts.") \
|
||||
M(MMappedFiles, "Total number of mmapped files.") \
|
||||
M(MMappedFileBytes, "Sum size of mmapped file regions.") \
|
||||
M(MMappedAllocs, "Total number of mmapped allocations") \
|
||||
M(MMappedAllocBytes, "Sum bytes of mmapped allocations") \
|
||||
M(AsynchronousReadWait, "Number of threads waiting for asynchronous read.") \
|
||||
M(PendingAsyncInsert, "Number of asynchronous inserts that are waiting for flush.") \
|
||||
M(KafkaConsumers, "Number of active Kafka consumers") \
|
||||
|
@ -8,7 +8,7 @@
|
||||
* table, so it makes sense to pre-fault the pages so that page faults don't
|
||||
* interrupt the resize loop. Set the allocator parameter accordingly.
|
||||
*/
|
||||
using HashTableAllocator = Allocator<true /* clear_memory */, true /* mmap_populate */>;
|
||||
using HashTableAllocator = Allocator<true /* clear_memory */, true /* populate */>;
|
||||
|
||||
template <size_t initial_bytes = 64>
|
||||
using HashTableAllocatorWithStackMemory = AllocatorWithStackMemory<HashTableAllocator, initial_bytes>;
|
||||
|
@ -2,7 +2,5 @@ CreatedReadBufferMMap
|
||||
CreatedReadBufferMMapFailed
|
||||
MMappedFileCacheHits
|
||||
MMappedFileCacheMisses
|
||||
MMappedAllocBytes
|
||||
MMappedAllocs
|
||||
MMappedFileBytes
|
||||
MMappedFiles
|
||||
|
Loading…
Reference in New Issue
Block a user