Merge pull request #4928 from danlark1/master

MarkCache and UncompressedCache use different allocator to catch segfaults
This commit is contained in:
alexey-milovidov 2019-04-06 23:03:04 +03:00 committed by GitHub
commit 4ffb8372d2
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
9 changed files with 205 additions and 215 deletions

View File

@ -1,190 +0,0 @@
#include <Common/Allocator.h>
#if !defined(__APPLE__) && !defined(__FreeBSD__)
#include <malloc.h>
#endif
#include <cstdlib>
#include <algorithm>
#include <sys/mman.h>
#include <Core/Defines.h>
#ifdef THREAD_SANITIZER
/// Thread sanitizer does not intercept mremap. The usage of mremap will lead to false positives.
#define DISABLE_MREMAP 1
#endif
#include <common/mremap.h>
#include <Common/MemoryTracker.h>
#include <Common/Exception.h>
#include <Common/formatReadable.h>
#include <IO/WriteHelpers.h>
/// Required for older Darwin builds, that lack definition of MAP_ANONYMOUS
#ifndef MAP_ANONYMOUS
#define MAP_ANONYMOUS MAP_ANON
#endif
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;
}
}
/** 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.
*
* PS. This is also required, because tcmalloc can not allocate a chunk of memory greater than 16 GB.
*/
#ifdef NDEBUG
static constexpr size_t MMAP_THRESHOLD = 64 * (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.
static constexpr size_t MMAP_THRESHOLD = 4096;
#endif
static constexpr size_t MMAP_MIN_ALIGNMENT = 4096;
static constexpr size_t MALLOC_MIN_ALIGNMENT = 8;
template <bool clear_memory_>
void * Allocator<clear_memory_>::mmap_hint()
{
#if ALLOCATOR_ASLR
return reinterpret_cast<void *>(std::uniform_int_distribution<intptr_t>(0x100000000000UL, 0x700000000000UL)(rng));
#else
return nullptr;
#endif
}
template <bool clear_memory_>
void * Allocator<clear_memory_>::alloc(size_t size, size_t alignment)
{
CurrentMemoryTracker::alloc(size);
void * buf;
if (size >= MMAP_THRESHOLD)
{
if (alignment > MMAP_MIN_ALIGNMENT)
throw DB::Exception("Too large alignment " + formatReadableSizeWithBinarySuffix(alignment) + ": more than page size when allocating "
+ formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::BAD_ARGUMENTS);
buf = mmap(mmap_hint(), size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0);
if (MAP_FAILED == buf)
DB::throwFromErrno("Allocator: Cannot mmap " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
/// No need for zero-fill, because mmap guarantees it.
}
else
{
if (alignment <= MALLOC_MIN_ALIGNMENT)
{
if (clear_memory)
buf = ::calloc(size, 1);
else
buf = ::malloc(size);
if (nullptr == buf)
DB::throwFromErrno("Allocator: Cannot malloc " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
}
else
{
buf = nullptr;
int res = posix_memalign(&buf, alignment, size);
if (0 != res)
DB::throwFromErrno("Cannot allocate memory (posix_memalign) " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res);
if (clear_memory)
memset(buf, 0, size);
}
}
return buf;
}
template <bool clear_memory_>
void Allocator<clear_memory_>::free(void * buf, size_t size)
{
if (size >= MMAP_THRESHOLD)
{
if (0 != munmap(buf, size))
DB::throwFromErrno("Allocator: Cannot munmap " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_MUNMAP);
}
else
{
::free(buf);
}
CurrentMemoryTracker::free(size);
}
template <bool clear_memory_>
void * Allocator<clear_memory_>::realloc(void * buf, size_t old_size, size_t new_size, size_t alignment)
{
if (old_size == new_size)
{
/// 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)
{
/// Resize malloc'd memory region with no special alignment requirement.
CurrentMemoryTracker::realloc(old_size, new_size);
void * new_buf = ::realloc(buf, new_size);
if (nullptr == new_buf)
DB::throwFromErrno("Allocator: Cannot realloc from " + formatReadableSizeWithBinarySuffix(old_size) + " to " + formatReadableSizeWithBinarySuffix(new_size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
buf = new_buf;
if (clear_memory && 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.
CurrentMemoryTracker::realloc(old_size, new_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, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0);
if (MAP_FAILED == buf)
DB::throwFromErrno("Allocator: Cannot mremap memory chunk from " + formatReadableSizeWithBinarySuffix(old_size) + " to " + formatReadableSizeWithBinarySuffix(new_size) + ".", DB::ErrorCodes::CANNOT_MREMAP);
/// No need for zero-fill, because mmap guarantees it.
}
else
{
/// All other cases 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;
}
return buf;
}
/// Explicit template instantiations.
template class Allocator<true>;
template class Allocator<false>;

View File

@ -10,11 +10,88 @@
#define ALLOCATOR_ASLR 1
#endif
#if ALLOCATOR_ASLR
#include <pcg_random.hpp>
#include <Common/randomSeed.h>
#if !defined(__APPLE__) && !defined(__FreeBSD__)
#include <malloc.h>
#endif
#include <cstdlib>
#include <algorithm>
#include <sys/mman.h>
#include <Core/Defines.h>
#ifdef THREAD_SANITIZER
/// Thread sanitizer does not intercept mremap. The usage of mremap will lead to false positives.
#define DISABLE_MREMAP 1
#endif
#include <common/mremap.h>
#include <Common/MemoryTracker.h>
#include <Common/Exception.h>
#include <Common/formatReadable.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.
*
* PS. This is also required, because tcmalloc can not allocate a chunk of memory greater than 16 GB.
*/
#ifdef NDEBUG
static constexpr size_t MMAP_THRESHOLD = 64 * (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.
static constexpr size_t MMAP_THRESHOLD = 4096;
#endif
static constexpr size_t MMAP_MIN_ALIGNMENT = 4096;
static constexpr size_t MALLOC_MIN_ALIGNMENT = 8;
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;
}
}
namespace AllocatorHints
{
struct DefaultHint
{
void * mmap_hint()
{
return nullptr;
}
};
struct RandomHint
{
void * mmap_hint()
{
return reinterpret_cast<void *>(std::uniform_int_distribution<intptr_t>(0x100000000000UL, 0x700000000000UL)(rng));
}
private:
pcg64 rng{randomSeed()};
};
}
/** Responsible for allocating / freeing memory. Used, for example, in PODArray, Arena.
* Also used in hash tables.
@ -23,31 +100,126 @@
* - passing the size into the `free` method;
* - by the presence of the `alignment` argument;
* - the possibility of zeroing memory (used in hash tables);
* - hint class for mmap
* - mmap_threshold for using mmap less or more
*/
template <bool clear_memory_>
class Allocator
template <bool clear_memory_, typename Hint, size_t mmap_threshold>
class AllocatorWithHint : Hint
{
#if ALLOCATOR_ASLR
private:
pcg64 rng{randomSeed()};
#endif
void * mmap_hint();
protected:
static constexpr bool clear_memory = clear_memory_;
public:
/// Allocate memory range.
void * alloc(size_t size, size_t alignment = 0);
void * alloc(size_t size, size_t alignment = 0)
{
CurrentMemoryTracker::alloc(size);
void * buf;
if (size >= mmap_threshold)
{
if (alignment > MMAP_MIN_ALIGNMENT)
throw DB::Exception("Too large alignment " + formatReadableSizeWithBinarySuffix(alignment) + ": more than page size when allocating "
+ formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::BAD_ARGUMENTS);
buf = mmap(Hint::mmap_hint(), size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0);
if (MAP_FAILED == buf)
DB::throwFromErrno("Allocator: Cannot mmap " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
/// No need for zero-fill, because mmap guarantees it.
}
else
{
if (alignment <= MALLOC_MIN_ALIGNMENT)
{
if constexpr (clear_memory)
buf = ::calloc(size, 1);
else
buf = ::malloc(size);
if (nullptr == buf)
DB::throwFromErrno("Allocator: Cannot malloc " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
}
else
{
buf = nullptr;
int res = posix_memalign(&buf, alignment, size);
if (0 != res)
DB::throwFromErrno("Cannot allocate memory (posix_memalign) " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res);
if (clear_memory)
memset(buf, 0, size);
}
}
return buf;
}
/// Free memory range.
void free(void * buf, size_t size);
void free(void * buf, size_t size)
{
if (size >= mmap_threshold)
{
if (0 != munmap(buf, size))
DB::throwFromErrno("Allocator: Cannot munmap " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_MUNMAP);
}
else
{
::free(buf);
}
CurrentMemoryTracker::free(size);
}
/** Enlarge memory range.
* Data from old range is moved to the beginning of new range.
* Address of memory range could change.
*/
void * realloc(void * buf, size_t old_size, size_t new_size, size_t alignment = 0);
void * realloc(void * buf, size_t old_size, size_t new_size, size_t alignment = 0)
{
if (old_size == new_size)
{
/// 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)
{
/// Resize malloc'd memory region with no special alignment requirement.
CurrentMemoryTracker::realloc(old_size, new_size);
void * new_buf = ::realloc(buf, new_size);
if (nullptr == new_buf)
DB::throwFromErrno("Allocator: Cannot realloc from " + formatReadableSizeWithBinarySuffix(old_size) + " to " + formatReadableSizeWithBinarySuffix(new_size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
buf = new_buf;
if (clear_memory && 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.
CurrentMemoryTracker::realloc(old_size, new_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, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0);
if (MAP_FAILED == buf)
DB::throwFromErrno("Allocator: Cannot mremap memory chunk from " + formatReadableSizeWithBinarySuffix(old_size) + " to " + formatReadableSizeWithBinarySuffix(new_size) + ".", DB::ErrorCodes::CANNOT_MREMAP);
/// No need for zero-fill, because mmap guarantees it.
}
else
{
/// All other cases 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;
}
return buf;
}
protected:
static constexpr size_t getStackThreshold()
@ -56,6 +228,13 @@ protected:
}
};
#if ALLOCATOR_ASLR
template <bool clear_memory>
using Allocator = AllocatorWithHint<clear_memory, AllocatorHints::RandomHint, MMAP_THRESHOLD>;
#else
template <bool clear_memory>
using Allocator = AllocatorWithHint<clear_memory, AllocatorHints::DefaultHint, MMAP_THRESHOLD>;
#endif
/** When using AllocatorWithStackMemory, located on the stack,
* GCC 4.9 mistakenly assumes that we can call `free` from a pointer to the stack.

View File

@ -49,7 +49,7 @@ private:
ProfileEvents::increment(ProfileEvents::ArenaAllocChunks);
ProfileEvents::increment(ProfileEvents::ArenaAllocBytes, size_);
begin = reinterpret_cast<char *>(Allocator::alloc(size_));
begin = reinterpret_cast<char *>(Allocator<false>::alloc(size_));
pos = begin;
end = begin + size_ - pad_right;
prev = prev_;
@ -57,7 +57,7 @@ private:
~Chunk()
{
Allocator::free(begin, size());
Allocator<false>::free(begin, size());
if (prev)
delete prev;

View File

@ -55,7 +55,7 @@ public:
char * alloc(const size_t size)
{
if (size > max_fixed_block_size)
return static_cast<char *>(Allocator::alloc(size));
return static_cast<char *>(Allocator<false>::alloc(size));
/// find list of required size
const auto list_idx = findFreeListIndex(size);
@ -76,7 +76,7 @@ public:
void free(char * ptr, const size_t size)
{
if (size > max_fixed_block_size)
return Allocator::free(ptr, size);
return Allocator<false>::free(ptr, size);
/// find list of required size
const auto list_idx = findFreeListIndex(size);

View File

@ -34,6 +34,6 @@ struct MarkInCompressedFile
}
};
using MarksInCompressedFile = PODArray<MarkInCompressedFile>;
using MarksInCompressedFile = PODArray<MarkInCompressedFile, 4096, AllocatorWithHint<false, AllocatorHints::RandomHint, 0>>;
}

View File

@ -24,7 +24,8 @@ namespace DB
* Differs in that is doesn't do unneeded memset. (And also tries to do as little as possible.)
* Also allows to allocate aligned piece of memory (to use with O_DIRECT, for example).
*/
struct Memory : boost::noncopyable, Allocator<false>
template <typename Allocator = Allocator<false>>
struct Memory : boost::noncopyable, Allocator
{
/// Padding is needed to allow usage of 'memcpySmallAllowReadWriteOverflow15' function with this buffer.
static constexpr size_t pad_right = 15;
@ -136,7 +137,7 @@ template <typename Base>
class BufferWithOwnMemory : public Base
{
protected:
Memory memory;
Memory<> memory;
public:
/// If non-nullptr 'existing_memory' is passed, then buffer will not create its own memory and will use existing_memory without ownership.
BufferWithOwnMemory(size_t size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0)

View File

@ -20,7 +20,7 @@ namespace DB
struct UncompressedCacheCell
{
Memory data;
Memory<AllocatorWithHint<false, AllocatorHints::RandomHint, 0>> data;
size_t compressed_size;
};

View File

@ -44,7 +44,7 @@ void thread(int fd, int mode, size_t min_offset, size_t max_offset, size_t block
{
using namespace DB;
Memory direct_buf(block_size, sysconf(_SC_PAGESIZE));
Memory<> direct_buf(block_size, sysconf(_SC_PAGESIZE));
std::vector<char> simple_buf(block_size);
char * buf;

View File

@ -51,9 +51,9 @@ void thread(int fd, int mode, size_t min_offset, size_t max_offset, size_t block
AIOContext ctx;
std::vector<Memory> buffers(buffers_count);
std::vector<Memory<>> buffers(buffers_count);
for (size_t i = 0; i < buffers_count; ++i)
buffers[i] = Memory(block_size, sysconf(_SC_PAGESIZE));
buffers[i] = Memory<>(block_size, sysconf(_SC_PAGESIZE));
drand48_data rand_data;
timespec times;