mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Hardening debug build (experimental) (#4632)
* Hardening debug build: more granular memory mappings and ASLR; add memory protection for mark cache and index * Addition to prev. revision * Addition to prev. revision * Addition to prev. revision
This commit is contained in:
parent
b81f73bb13
commit
6db73152d2
@ -85,7 +85,7 @@ class QuantileTDigest
|
||||
Params params;
|
||||
|
||||
/// The memory will be allocated to several elements at once, so that the state occupies 64 bytes.
|
||||
static constexpr size_t bytes_in_arena = 64 - sizeof(PODArray<Centroid>) - sizeof(Count) - sizeof(UInt32);
|
||||
static constexpr size_t bytes_in_arena = 128 - sizeof(PODArray<Centroid>) - sizeof(Count) - sizeof(UInt32);
|
||||
|
||||
using Summary = PODArray<Centroid, bytes_in_arena / sizeof(Centroid), AllocatorWithStackMemory<Allocator<false>, bytes_in_arena>>;
|
||||
|
||||
|
@ -255,6 +255,11 @@ size_t ColumnAggregateFunction::allocatedBytes() const
|
||||
return res;
|
||||
}
|
||||
|
||||
void ColumnAggregateFunction::protect()
|
||||
{
|
||||
data.protect();
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnAggregateFunction::cloneEmpty() const
|
||||
{
|
||||
return create(func, Arenas(1, std::make_shared<Arena>()));
|
||||
|
@ -157,6 +157,8 @@ public:
|
||||
|
||||
size_t allocatedBytes() const override;
|
||||
|
||||
void protect() override;
|
||||
|
||||
void insertRangeFrom(const IColumn & from, size_t start, size_t length) override;
|
||||
|
||||
void popBack(size_t n) override;
|
||||
|
@ -311,6 +311,13 @@ size_t ColumnArray::allocatedBytes() const
|
||||
}
|
||||
|
||||
|
||||
void ColumnArray::protect()
|
||||
{
|
||||
getData().protect();
|
||||
getOffsets().protect();
|
||||
}
|
||||
|
||||
|
||||
bool ColumnArray::hasEqualOffsets(const ColumnArray & other) const
|
||||
{
|
||||
if (offsets == other.offsets)
|
||||
|
@ -78,6 +78,7 @@ public:
|
||||
void reserve(size_t n) override;
|
||||
size_t byteSize() const override;
|
||||
size_t allocatedBytes() const override;
|
||||
void protect() override;
|
||||
ColumnPtr replicate(const Offsets & replicate_offsets) const override;
|
||||
ColumnPtr convertToFullColumnIfConst() const override;
|
||||
void getExtremes(Field & min, Field & max) const override;
|
||||
|
@ -87,6 +87,7 @@ public:
|
||||
size_t size() const override { return data.size(); }
|
||||
size_t byteSize() const override { return data.size() * sizeof(data[0]); }
|
||||
size_t allocatedBytes() const override { return data.allocated_bytes(); }
|
||||
void protect() override { data.protect(); }
|
||||
void reserve(size_t n) override { data.reserve(n); }
|
||||
|
||||
void insertFrom(const IColumn & src, size_t n) override { data.push_back(static_cast<const Self &>(src).getData()[n]); }
|
||||
|
@ -57,6 +57,11 @@ public:
|
||||
return chars.allocated_bytes() + sizeof(n);
|
||||
}
|
||||
|
||||
void protect() override
|
||||
{
|
||||
chars.protect();
|
||||
}
|
||||
|
||||
Field operator[](size_t index) const override
|
||||
{
|
||||
return String(reinterpret_cast<const char *>(&chars[n * index]), n);
|
||||
|
@ -363,7 +363,6 @@ ColumnPtr ColumnLowCardinality::countKeys() const
|
||||
}
|
||||
|
||||
|
||||
|
||||
ColumnLowCardinality::Index::Index() : positions(ColumnUInt8::create()), size_of_type(sizeof(UInt8)) {}
|
||||
|
||||
ColumnLowCardinality::Index::Index(MutableColumnPtr && positions) : positions(std::move(positions))
|
||||
|
@ -291,6 +291,12 @@ size_t ColumnNullable::allocatedBytes() const
|
||||
return getNestedColumn().allocatedBytes() + getNullMapColumn().allocatedBytes();
|
||||
}
|
||||
|
||||
void ColumnNullable::protect()
|
||||
{
|
||||
getNestedColumn().protect();
|
||||
getNullMapColumn().protect();
|
||||
}
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
|
@ -71,6 +71,7 @@ public:
|
||||
void reserve(size_t n) override;
|
||||
size_t byteSize() const override;
|
||||
size_t allocatedBytes() const override;
|
||||
void protect() override;
|
||||
ColumnPtr replicate(const Offsets & replicate_offsets) const override;
|
||||
void updateHashWithValue(size_t n, SipHash & hash) const override;
|
||||
void getExtremes(Field & min, Field & max) const override;
|
||||
|
@ -412,4 +412,11 @@ void ColumnString::getPermutationWithCollation(const Collator & collator, bool r
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void ColumnString::protect()
|
||||
{
|
||||
getChars().protect();
|
||||
getOffsets().protect();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -68,6 +68,8 @@ public:
|
||||
return chars.allocated_bytes() + offsets.allocated_bytes();
|
||||
}
|
||||
|
||||
void protect() override;
|
||||
|
||||
MutableColumnPtr cloneResized(size_t to_size) const override;
|
||||
|
||||
Field operator[](size_t n) const override
|
||||
|
@ -315,6 +315,12 @@ size_t ColumnTuple::allocatedBytes() const
|
||||
return res;
|
||||
}
|
||||
|
||||
void ColumnTuple::protect()
|
||||
{
|
||||
for (auto & column : columns)
|
||||
column->assumeMutableRef().protect();
|
||||
}
|
||||
|
||||
void ColumnTuple::getExtremes(Field & min, Field & max) const
|
||||
{
|
||||
const size_t tuple_size = columns.size();
|
||||
|
@ -71,6 +71,7 @@ public:
|
||||
void reserve(size_t n) override;
|
||||
size_t byteSize() const override;
|
||||
size_t allocatedBytes() const override;
|
||||
void protect() override;
|
||||
void forEachSubcolumn(ColumnCallback callback) override;
|
||||
|
||||
size_t tupleSize() const { return columns.size(); }
|
||||
|
@ -80,6 +80,7 @@ public:
|
||||
bool isNumeric() const override { return column_holder->isNumeric(); }
|
||||
|
||||
size_t byteSize() const override { return column_holder->byteSize(); }
|
||||
void protect() override { column_holder->assumeMutableRef().protect(); }
|
||||
size_t allocatedBytes() const override
|
||||
{
|
||||
return column_holder->allocatedBytes()
|
||||
|
@ -163,6 +163,11 @@ public:
|
||||
return data.allocated_bytes();
|
||||
}
|
||||
|
||||
void protect() override
|
||||
{
|
||||
data.protect();
|
||||
}
|
||||
|
||||
void insertValue(const T value)
|
||||
{
|
||||
data.push_back(value);
|
||||
|
@ -24,9 +24,10 @@ namespace DB
|
||||
class ColumnVectorHelper : public IColumn
|
||||
{
|
||||
public:
|
||||
template <size_t ELEMENT_SIZE>
|
||||
const char * getRawDataBegin() const
|
||||
{
|
||||
return *reinterpret_cast<const char * const *>(reinterpret_cast<const char *>(this) + sizeof(*this));
|
||||
return reinterpret_cast<const PODArrayBase<ELEMENT_SIZE, 4096, Allocator<false>, 15, 16> *>(reinterpret_cast<const char *>(this) + sizeof(*this))->raw_data();
|
||||
}
|
||||
|
||||
template <size_t ELEMENT_SIZE>
|
||||
|
@ -253,6 +253,10 @@ public:
|
||||
/// Zero, if could be determined.
|
||||
virtual size_t allocatedBytes() const = 0;
|
||||
|
||||
/// Make memory region readonly with mprotect if it is large enough.
|
||||
/// The operation is slow and performed only for debug builds.
|
||||
virtual void protect() {}
|
||||
|
||||
/// If the column contains subcolumns (such as Array, Nullable, etc), do callback on them.
|
||||
/// Shallow: doesn't do recursive calls; don't do call for itself.
|
||||
using ColumnCallback = std::function<void(Ptr&)>;
|
||||
|
@ -43,11 +43,30 @@ namespace ErrorCodes
|
||||
*
|
||||
* PS. This is also required, because tcmalloc can not allocate a chunk of memory greater than 16 GB.
|
||||
*/
|
||||
static constexpr size_t MMAP_THRESHOLD = 64 * (1ULL << 20);
|
||||
#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)
|
||||
{
|
||||
@ -61,7 +80,7 @@ void * Allocator<clear_memory_>::alloc(size_t size, size_t alignment)
|
||||
throw DB::Exception("Too large alignment " + formatReadableSizeWithBinarySuffix(alignment) + ": more than page size when allocating "
|
||||
+ formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
buf = mmap(nullptr, size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0);
|
||||
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);
|
||||
|
||||
|
@ -2,6 +2,19 @@
|
||||
|
||||
#include <string.h>
|
||||
|
||||
#ifdef NDEBUG
|
||||
/// If set to 1 - randomize memory mappings manually (address space layout randomization) to reproduce more memory stomping bugs.
|
||||
/// Note that Linux doesn't do it by default. This may lead to worse TLB performance.
|
||||
#define ALLOCATOR_ASLR 0
|
||||
#else
|
||||
#define ALLOCATOR_ASLR 1
|
||||
#endif
|
||||
|
||||
#if ALLOCATOR_ASLR
|
||||
#include <pcg_random.hpp>
|
||||
#include <Common/randomSeed.h>
|
||||
#endif
|
||||
|
||||
|
||||
/** Responsible for allocating / freeing memory. Used, for example, in PODArray, Arena.
|
||||
* Also used in hash tables.
|
||||
@ -14,6 +27,12 @@
|
||||
template <bool clear_memory_>
|
||||
class Allocator
|
||||
{
|
||||
#if ALLOCATOR_ASLR
|
||||
private:
|
||||
pcg64 rng{randomSeed()};
|
||||
#endif
|
||||
void * mmap_hint();
|
||||
|
||||
protected:
|
||||
static constexpr bool clear_memory = clear_memory_;
|
||||
|
||||
|
@ -419,6 +419,7 @@ namespace ErrorCodes
|
||||
extern const int BAD_DATABASE_FOR_TEMPORARY_TABLE = 442;
|
||||
extern const int NO_COMMON_COLUMNS_WITH_PROTOBUF_SCHEMA = 443;
|
||||
extern const int UNKNOWN_PROTOBUF_FORMAT = 444;
|
||||
extern const int CANNOT_MPROTECT = 445;
|
||||
|
||||
extern const int KEEPER_EXCEPTION = 999;
|
||||
extern const int POCO_EXCEPTION = 1000;
|
||||
|
@ -17,10 +17,19 @@
|
||||
#include <Common/BitHelpers.h>
|
||||
#include <Common/memcpySmall.h>
|
||||
|
||||
#ifndef NDEBUG
|
||||
#include <sys/mman.h>
|
||||
#endif
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_MPROTECT;
|
||||
}
|
||||
|
||||
inline constexpr size_t integerRoundUp(size_t value, size_t dividend)
|
||||
{
|
||||
return ((value + dividend - 1) / dividend) * dividend;
|
||||
@ -108,6 +117,8 @@ protected:
|
||||
if (c_start == null)
|
||||
return;
|
||||
|
||||
unprotect();
|
||||
|
||||
TAllocator::free(c_start - pad_left, allocated_bytes());
|
||||
}
|
||||
|
||||
@ -120,6 +131,8 @@ protected:
|
||||
return;
|
||||
}
|
||||
|
||||
unprotect();
|
||||
|
||||
ptrdiff_t end_diff = c_end - c_start;
|
||||
|
||||
c_start = reinterpret_cast<char *>(
|
||||
@ -155,6 +168,28 @@ protected:
|
||||
realloc(allocated_bytes() * 2, std::forward<TAllocatorParams>(allocator_params)...);
|
||||
}
|
||||
|
||||
#ifndef NDEBUG
|
||||
/// Make memory region readonly with mprotect if it is large enough.
|
||||
/// The operation is slow and performed only for debug builds.
|
||||
void protectImpl(int prot)
|
||||
{
|
||||
static constexpr size_t PAGE_SIZE = 4096;
|
||||
|
||||
char * left_rounded_up = reinterpret_cast<char *>((reinterpret_cast<intptr_t>(c_start) - pad_left + PAGE_SIZE - 1) / PAGE_SIZE * PAGE_SIZE);
|
||||
char * right_rounded_down = reinterpret_cast<char *>((reinterpret_cast<intptr_t>(c_end_of_storage) + pad_right) / PAGE_SIZE * PAGE_SIZE);
|
||||
|
||||
if (right_rounded_down > left_rounded_up)
|
||||
{
|
||||
size_t length = right_rounded_down - left_rounded_up;
|
||||
if (0 != mprotect(left_rounded_up, length, prot))
|
||||
throwFromErrno("Cannot mprotect memory region", ErrorCodes::CANNOT_MPROTECT);
|
||||
}
|
||||
}
|
||||
|
||||
/// Restore memory protection in destructor or realloc for further reuse by allocator.
|
||||
bool mprotected = false;
|
||||
#endif
|
||||
|
||||
public:
|
||||
bool empty() const { return c_end == c_start; }
|
||||
size_t size() const { return (c_end - c_start) / ELEMENT_SIZE; }
|
||||
@ -199,6 +234,23 @@ public:
|
||||
c_end += byte_size(1);
|
||||
}
|
||||
|
||||
void protect()
|
||||
{
|
||||
#ifndef NDEBUG
|
||||
protectImpl(PROT_READ);
|
||||
mprotected = true;
|
||||
#endif
|
||||
}
|
||||
|
||||
void unprotect()
|
||||
{
|
||||
#ifndef NDEBUG
|
||||
if (mprotected)
|
||||
protectImpl(PROT_WRITE);
|
||||
mprotected = false;
|
||||
#endif
|
||||
}
|
||||
|
||||
~PODArrayBase()
|
||||
{
|
||||
dealloc();
|
||||
@ -402,6 +454,11 @@ public:
|
||||
|
||||
void swap(PODArray & rhs)
|
||||
{
|
||||
#ifndef NDEBUG
|
||||
this->unprotect();
|
||||
rhs.unprotect();
|
||||
#endif
|
||||
|
||||
/// Swap two PODArray objects, arr1 and arr2, that satisfy the following conditions:
|
||||
/// - The elements of arr1 are stored on stack.
|
||||
/// - The elements of arr2 are stored on heap.
|
||||
@ -450,7 +507,9 @@ public:
|
||||
};
|
||||
|
||||
if (!this->isInitialized() && !rhs.isInitialized())
|
||||
{
|
||||
return;
|
||||
}
|
||||
else if (!this->isInitialized() && rhs.isInitialized())
|
||||
{
|
||||
do_move(rhs, *this);
|
||||
@ -494,9 +553,13 @@ public:
|
||||
rhs.c_end = rhs.c_start + this->byte_size(lhs_size);
|
||||
}
|
||||
else if (this->isAllocatedFromStack() && !rhs.isAllocatedFromStack())
|
||||
{
|
||||
swap_stack_heap(*this, rhs);
|
||||
}
|
||||
else if (!this->isAllocatedFromStack() && rhs.isAllocatedFromStack())
|
||||
{
|
||||
swap_stack_heap(rhs, *this);
|
||||
}
|
||||
else
|
||||
{
|
||||
std::swap(this->c_start, rhs.c_start);
|
||||
|
@ -102,23 +102,23 @@ static inline T ALWAYS_INLINE packFixed(
|
||||
switch (key_sizes[j])
|
||||
{
|
||||
case 1:
|
||||
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(column)->getRawDataBegin() + index, 1);
|
||||
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(column)->getRawDataBegin<1>() + index, 1);
|
||||
offset += 1;
|
||||
break;
|
||||
case 2:
|
||||
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(column)->getRawDataBegin() + index * 2, 2);
|
||||
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(column)->getRawDataBegin<2>() + index * 2, 2);
|
||||
offset += 2;
|
||||
break;
|
||||
case 4:
|
||||
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(column)->getRawDataBegin() + index * 4, 4);
|
||||
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(column)->getRawDataBegin<4>() + index * 4, 4);
|
||||
offset += 4;
|
||||
break;
|
||||
case 8:
|
||||
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(column)->getRawDataBegin() + index * 8, 8);
|
||||
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(column)->getRawDataBegin<8>() + index * 8, 8);
|
||||
offset += 8;
|
||||
break;
|
||||
default:
|
||||
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(column)->getRawDataBegin() + index * key_sizes[j], key_sizes[j]);
|
||||
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(column)->getRawDataBegin<1>() + index * key_sizes[j], key_sizes[j]);
|
||||
offset += key_sizes[j];
|
||||
}
|
||||
}
|
||||
@ -168,23 +168,23 @@ static inline T ALWAYS_INLINE packFixed(
|
||||
switch (key_sizes[j])
|
||||
{
|
||||
case 1:
|
||||
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(key_columns[j])->getRawDataBegin() + i, 1);
|
||||
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(key_columns[j])->getRawDataBegin<1>() + i, 1);
|
||||
offset += 1;
|
||||
break;
|
||||
case 2:
|
||||
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(key_columns[j])->getRawDataBegin() + i * 2, 2);
|
||||
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(key_columns[j])->getRawDataBegin<2>() + i * 2, 2);
|
||||
offset += 2;
|
||||
break;
|
||||
case 4:
|
||||
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(key_columns[j])->getRawDataBegin() + i * 4, 4);
|
||||
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(key_columns[j])->getRawDataBegin<4>() + i * 4, 4);
|
||||
offset += 4;
|
||||
break;
|
||||
case 8:
|
||||
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(key_columns[j])->getRawDataBegin() + i * 8, 8);
|
||||
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(key_columns[j])->getRawDataBegin<8>() + i * 8, 8);
|
||||
offset += 8;
|
||||
break;
|
||||
default:
|
||||
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(key_columns[j])->getRawDataBegin() + i * key_sizes[j], key_sizes[j]);
|
||||
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(key_columns[j])->getRawDataBegin<1>() + i * key_sizes[j], key_sizes[j]);
|
||||
offset += key_sizes[j];
|
||||
}
|
||||
}
|
||||
|
@ -513,13 +513,16 @@ void MergeTreeDataPart::loadIndex()
|
||||
|
||||
for (size_t i = 0; i < marks_count; ++i) //-V756
|
||||
for (size_t j = 0; j < key_size; ++j)
|
||||
storage.primary_key_data_types[j]->deserializeBinary(*loaded_index[j].get(), index_file);
|
||||
storage.primary_key_data_types[j]->deserializeBinary(*loaded_index[j], index_file);
|
||||
|
||||
for (size_t i = 0; i < key_size; ++i)
|
||||
{
|
||||
loaded_index[i]->protect();
|
||||
if (loaded_index[i]->size() != marks_count)
|
||||
throw Exception("Cannot read all data from index file " + index_path
|
||||
+ "(expected size: " + toString(marks_count) + ", read: " + toString(loaded_index[i]->size()) + ")",
|
||||
ErrorCodes::CANNOT_READ_ALL_DATA);
|
||||
}
|
||||
|
||||
if (!index_file.eof())
|
||||
throw Exception("Index file " + index_path + " is unexpectedly long", ErrorCodes::EXPECTED_END_OF_FILE);
|
||||
|
@ -132,6 +132,7 @@ void MergeTreeReaderStream::loadMarks()
|
||||
if (buffer.eof() || buffer.buffer().size() != file_size)
|
||||
throw Exception("Cannot read all marks from file " + mrk_path, ErrorCodes::CANNOT_READ_ALL_DATA);
|
||||
|
||||
res->protect();
|
||||
return res;
|
||||
};
|
||||
|
||||
|
@ -12,7 +12,8 @@
|
||||
|
||||
#define MREMAP_MAYMOVE 1
|
||||
|
||||
void * mremap(void * old_address,
|
||||
void * mremap(
|
||||
void * old_address,
|
||||
size_t old_size,
|
||||
size_t new_size,
|
||||
int flags = 0,
|
||||
@ -23,7 +24,8 @@ void * mremap(void * old_address,
|
||||
|
||||
#endif
|
||||
|
||||
inline void * clickhouse_mremap(void * old_address,
|
||||
inline void * clickhouse_mremap(
|
||||
void * old_address,
|
||||
size_t old_size,
|
||||
size_t new_size,
|
||||
int flags = 0,
|
||||
@ -32,7 +34,8 @@ inline void * clickhouse_mremap(void * old_address,
|
||||
[[maybe_unused]] int mmap_fd = -1,
|
||||
[[maybe_unused]] off_t mmap_offset = 0)
|
||||
{
|
||||
return mremap(old_address,
|
||||
return mremap(
|
||||
old_address,
|
||||
old_size,
|
||||
new_size,
|
||||
flags
|
||||
|
Loading…
Reference in New Issue
Block a user