未验证 提交 6db73152 编写于 作者: A alexey-milovidov 提交者: GitHub

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
上级 b81f73bb
......@@ -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
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册