mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #58237 from azat/build/fwd-decl-exception
Some code refactoring (was an attempt to improve build time, but failed)
This commit is contained in:
commit
8d984df135
@ -12,6 +12,8 @@ elseif (CMAKE_SYSTEM_NAME MATCHES "FreeBSD")
|
||||
elseif (CMAKE_SYSTEM_NAME MATCHES "Darwin")
|
||||
set (OS_DARWIN 1)
|
||||
add_definitions(-D OS_DARWIN)
|
||||
# For MAP_ANON/MAP_ANONYMOUS
|
||||
add_definitions(-D _DARWIN_C_SOURCE)
|
||||
elseif (CMAKE_SYSTEM_NAME MATCHES "SunOS")
|
||||
set (OS_SUNOS 1)
|
||||
add_definitions(-D OS_SUNOS)
|
||||
|
@ -20,6 +20,7 @@
|
||||
|
||||
#include <Common/ArenaAllocator.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/thread_local_rng.h>
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <AggregateFunctions/QuantileTDigest.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <iostream>
|
||||
|
||||
int main(int, char **)
|
||||
{
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Columns/ColumnCompressed.h>
|
||||
#include <Common/formatReadable.h>
|
||||
|
||||
#pragma clang diagnostic ignored "-Wold-style-cast"
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Common/Arena.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Columns/IColumnDummy.h>
|
||||
#include <Columns/ColumnsCommon.h>
|
||||
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <vector>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Common/randomSeed.h>
|
||||
#include <Common/thread_local_rng.h>
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
using namespace DB;
|
||||
|
@ -1,9 +1,190 @@
|
||||
#include "Allocator.h"
|
||||
#include <Common/Allocator.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/formatReadable.h>
|
||||
#include <Common/CurrentMemoryTracker.h>
|
||||
|
||||
#include <base/errnoToString.h>
|
||||
#include <base/getPageSize.h>
|
||||
|
||||
#include <Poco/Logger.h>
|
||||
#include <sys/mman.h> /// MADV_POPULATE_WRITE
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_ALLOCATE_MEMORY;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
using namespace DB;
|
||||
|
||||
#if defined(MADV_POPULATE_WRITE)
|
||||
/// Address passed to madvise is required to be aligned to the page boundary.
|
||||
auto adjustToPageSize(void * buf, size_t len, size_t page_size)
|
||||
{
|
||||
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));
|
||||
}
|
||||
#endif
|
||||
|
||||
void prefaultPages([[maybe_unused]] void * buf_, [[maybe_unused]] size_t len_)
|
||||
{
|
||||
#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
|
||||
}
|
||||
|
||||
template <bool clear_memory, bool populate>
|
||||
void * allocNoTrack(size_t size, size_t alignment)
|
||||
{
|
||||
void * buf;
|
||||
if (alignment <= MALLOC_MIN_ALIGNMENT)
|
||||
{
|
||||
if constexpr (clear_memory)
|
||||
buf = ::calloc(size, 1);
|
||||
else
|
||||
buf = ::malloc(size);
|
||||
|
||||
if (nullptr == buf)
|
||||
throw DB::ErrnoException(DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Allocator: Cannot malloc {}.", ReadableSize(size));
|
||||
}
|
||||
else
|
||||
{
|
||||
buf = nullptr;
|
||||
int res = posix_memalign(&buf, alignment, size);
|
||||
|
||||
if (0 != res)
|
||||
throw DB::ErrnoException(
|
||||
DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Cannot allocate memory (posix_memalign) {}.", ReadableSize(size));
|
||||
|
||||
if constexpr (clear_memory)
|
||||
memset(buf, 0, size);
|
||||
}
|
||||
|
||||
if constexpr (populate)
|
||||
prefaultPages(buf, size);
|
||||
|
||||
return buf;
|
||||
}
|
||||
|
||||
void freeNoTrack(void * buf)
|
||||
{
|
||||
::free(buf);
|
||||
}
|
||||
|
||||
void checkSize(size_t size)
|
||||
{
|
||||
/// More obvious exception in case of possible overflow (instead of just "Cannot mmap").
|
||||
if (size >= 0x8000000000000000ULL)
|
||||
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Too large size ({}) passed to allocator. It indicates an error.", size);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
/// 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 <bool clear_memory_, bool populate>
|
||||
void * Allocator<clear_memory_, populate>::alloc(size_t size, size_t alignment)
|
||||
{
|
||||
checkSize(size);
|
||||
auto trace = CurrentMemoryTracker::alloc(size);
|
||||
void * ptr = allocNoTrack<clear_memory_, populate>(size, alignment);
|
||||
trace.onAlloc(ptr, size);
|
||||
return ptr;
|
||||
}
|
||||
|
||||
|
||||
template <bool clear_memory_, bool populate>
|
||||
void Allocator<clear_memory_, populate>::free(void * buf, size_t size)
|
||||
{
|
||||
try
|
||||
{
|
||||
checkSize(size);
|
||||
freeNoTrack(buf);
|
||||
auto trace = CurrentMemoryTracker::free(size);
|
||||
trace.onFree(buf, size);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
DB::tryLogCurrentException("Allocator::free");
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
template <bool clear_memory_, bool populate>
|
||||
void * Allocator<clear_memory_, populate>::realloc(void * buf, size_t old_size, size_t new_size, size_t alignment)
|
||||
{
|
||||
checkSize(new_size);
|
||||
|
||||
if (old_size == new_size)
|
||||
{
|
||||
/// nothing to do.
|
||||
/// BTW, it's not possible to change alignment while doing realloc.
|
||||
}
|
||||
else if (alignment <= MALLOC_MIN_ALIGNMENT)
|
||||
{
|
||||
/// Resize malloc'd memory region with no special alignment requirement.
|
||||
auto trace_free = CurrentMemoryTracker::free(old_size);
|
||||
auto trace_alloc = CurrentMemoryTracker::alloc(new_size);
|
||||
trace_free.onFree(buf, old_size);
|
||||
|
||||
void * new_buf = ::realloc(buf, new_size);
|
||||
if (nullptr == new_buf)
|
||||
{
|
||||
throw DB::ErrnoException(
|
||||
DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY,
|
||||
"Allocator: Cannot realloc from {} to {}",
|
||||
ReadableSize(old_size),
|
||||
ReadableSize(new_size));
|
||||
}
|
||||
|
||||
buf = new_buf;
|
||||
trace_alloc.onAlloc(buf, new_size);
|
||||
|
||||
if constexpr (clear_memory)
|
||||
if (new_size > old_size)
|
||||
memset(reinterpret_cast<char *>(buf) + old_size, 0, new_size - old_size);
|
||||
}
|
||||
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;
|
||||
}
|
||||
|
||||
|
||||
template class Allocator<false, false>;
|
||||
template class Allocator<true, false>;
|
||||
template class Allocator<false, true>;
|
||||
|
@ -8,47 +8,19 @@
|
||||
#define ALLOCATOR_ASLR 1
|
||||
#endif
|
||||
|
||||
#include <pcg_random.hpp>
|
||||
#include <Common/thread_local_rng.h>
|
||||
|
||||
#if !defined(OS_DARWIN) && !defined(OS_FREEBSD)
|
||||
#include <malloc.h>
|
||||
#endif
|
||||
|
||||
#include <cstdlib>
|
||||
#include <algorithm>
|
||||
#include <sys/mman.h>
|
||||
|
||||
#include <Core/Defines.h>
|
||||
#include <base/getPageSize.h>
|
||||
|
||||
#include <Common/CurrentMemoryTracker.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/formatReadable.h>
|
||||
|
||||
#include <Common/Allocator_fwd.h>
|
||||
|
||||
#include <base/errnoToString.h>
|
||||
#include <Poco/Logger.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <cstdlib>
|
||||
|
||||
|
||||
extern const size_t POPULATE_THRESHOLD;
|
||||
|
||||
static constexpr size_t MALLOC_MIN_ALIGNMENT = 8;
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_ALLOCATE_MEMORY;
|
||||
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.
|
||||
@ -69,83 +41,16 @@ class Allocator
|
||||
{
|
||||
public:
|
||||
/// Allocate memory range.
|
||||
void * alloc(size_t size, size_t alignment = 0)
|
||||
{
|
||||
checkSize(size);
|
||||
auto trace = CurrentMemoryTracker::alloc(size);
|
||||
void * ptr = allocNoTrack(size, alignment);
|
||||
trace.onAlloc(ptr, size);
|
||||
return ptr;
|
||||
}
|
||||
void * alloc(size_t size, size_t alignment = 0);
|
||||
|
||||
/// Free memory range.
|
||||
void free(void * buf, size_t size)
|
||||
{
|
||||
try
|
||||
{
|
||||
checkSize(size);
|
||||
freeNoTrack(buf);
|
||||
auto trace = CurrentMemoryTracker::free(size);
|
||||
trace.onFree(buf, size);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
DB::tryLogCurrentException("Allocator::free");
|
||||
throw;
|
||||
}
|
||||
}
|
||||
void free(void * buf, size_t 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)
|
||||
{
|
||||
checkSize(new_size);
|
||||
|
||||
if (old_size == new_size)
|
||||
{
|
||||
/// nothing to do.
|
||||
/// BTW, it's not possible to change alignment while doing realloc.
|
||||
}
|
||||
else if (alignment <= MALLOC_MIN_ALIGNMENT)
|
||||
{
|
||||
/// Resize malloc'd memory region with no special alignment requirement.
|
||||
auto trace_free = CurrentMemoryTracker::free(old_size);
|
||||
auto trace_alloc = CurrentMemoryTracker::alloc(new_size);
|
||||
trace_free.onFree(buf, old_size);
|
||||
|
||||
void * new_buf = ::realloc(buf, new_size);
|
||||
if (nullptr == new_buf)
|
||||
{
|
||||
throw DB::ErrnoException(
|
||||
DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY,
|
||||
"Allocator: Cannot realloc from {} to {}",
|
||||
ReadableSize(old_size),
|
||||
ReadableSize(new_size));
|
||||
}
|
||||
|
||||
buf = new_buf;
|
||||
trace_alloc.onAlloc(buf, new_size);
|
||||
|
||||
if constexpr (clear_memory)
|
||||
if (new_size > old_size)
|
||||
memset(reinterpret_cast<char *>(buf) + old_size, 0, new_size - old_size);
|
||||
}
|
||||
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;
|
||||
}
|
||||
void * realloc(void * buf, size_t old_size, size_t new_size, size_t alignment = 0);
|
||||
|
||||
protected:
|
||||
static constexpr size_t getStackThreshold()
|
||||
@ -156,76 +61,6 @@ protected:
|
||||
static constexpr bool clear_memory = clear_memory_;
|
||||
|
||||
private:
|
||||
void * allocNoTrack(size_t size, size_t alignment)
|
||||
{
|
||||
void * buf;
|
||||
if (alignment <= MALLOC_MIN_ALIGNMENT)
|
||||
{
|
||||
if constexpr (clear_memory)
|
||||
buf = ::calloc(size, 1);
|
||||
else
|
||||
buf = ::malloc(size);
|
||||
|
||||
if (nullptr == buf)
|
||||
throw DB::ErrnoException(DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Allocator: Cannot malloc {}.", ReadableSize(size));
|
||||
}
|
||||
else
|
||||
{
|
||||
buf = nullptr;
|
||||
int res = posix_memalign(&buf, alignment, size);
|
||||
|
||||
if (0 != res)
|
||||
throw DB::ErrnoException(
|
||||
DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Cannot allocate memory (posix_memalign) {}.", ReadableSize(size));
|
||||
|
||||
if constexpr (clear_memory)
|
||||
memset(buf, 0, size);
|
||||
}
|
||||
|
||||
if constexpr (populate)
|
||||
prefaultPages(buf, size);
|
||||
|
||||
return buf;
|
||||
}
|
||||
|
||||
void freeNoTrack(void * buf)
|
||||
{
|
||||
::free(buf);
|
||||
}
|
||||
|
||||
void checkSize(size_t size)
|
||||
{
|
||||
/// More obvious exception in case of possible overflow (instead of just "Cannot mmap").
|
||||
if (size >= 0x8000000000000000ULL)
|
||||
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Too large size ({}) passed to allocator. It indicates an error.", size);
|
||||
}
|
||||
|
||||
/// Address passed to madvise is required to be aligned to the page boundary.
|
||||
auto adjustToPageSize(void * buf, size_t len, size_t page_size)
|
||||
{
|
||||
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));
|
||||
}
|
||||
|
||||
void prefaultPages([[maybe_unused]] void * buf_, [[maybe_unused]] size_t len_)
|
||||
{
|
||||
#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
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Common/Allocator.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/memcpySmall.h>
|
||||
#include <base/getPageSize.h>
|
||||
|
||||
#if __has_include(<sanitizer/asan_interface.h>) && defined(ADDRESS_SANITIZER)
|
||||
# include <sanitizer/asan_interface.h>
|
||||
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <mutex>
|
||||
#include <Core/Defines.h>
|
||||
#if __has_include(<sanitizer/asan_interface.h>) && defined(ADDRESS_SANITIZER)
|
||||
# include <sanitizer/asan_interface.h>
|
||||
|
@ -19,11 +19,6 @@
|
||||
#include <Common/randomSeed.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
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <Common/formatReadable.h>
|
||||
#include <Common/AsynchronousMetrics.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/setThreadName.h>
|
||||
@ -8,6 +9,7 @@
|
||||
#include <IO/MMappedFileCache.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <base/errnoToString.h>
|
||||
#include <base/getPageSize.h>
|
||||
#include <sys/resource.h>
|
||||
#include <chrono>
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <algorithm>
|
||||
#include <cassert>
|
||||
#include <concepts>
|
||||
#include <cstddef>
|
||||
|
@ -13,6 +13,11 @@
|
||||
#include <valgrind/valgrind.h>
|
||||
#endif
|
||||
|
||||
/// Required for older Darwin builds, that lack definition of MAP_ANONYMOUS
|
||||
#ifndef MAP_ANONYMOUS
|
||||
#define MAP_ANONYMOUS MAP_ANON
|
||||
#endif
|
||||
|
||||
namespace DB::ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_ALLOCATE_MEMORY;
|
||||
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/types.h>
|
||||
#include <Common/PODArray.h>
|
||||
#include <Common/levenshteinDistance.h>
|
||||
|
||||
#include <algorithm>
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <random>
|
||||
#include <base/getThreadId.h>
|
||||
#include <Common/thread_local_rng.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <base/hex.h>
|
||||
#include <Core/Settings.h>
|
||||
|
@ -1,8 +1,46 @@
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/PODArray.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_MPROTECT;
|
||||
extern const int CANNOT_ALLOCATE_MEMORY;
|
||||
}
|
||||
|
||||
namespace PODArrayDetails
|
||||
{
|
||||
|
||||
#ifndef NDEBUG
|
||||
void protectMemoryRegion(void * addr, size_t len, int prot)
|
||||
{
|
||||
if (0 != mprotect(addr, len, prot))
|
||||
throw ErrnoException(ErrorCodes::CANNOT_MPROTECT, "Cannot mprotect memory region");
|
||||
}
|
||||
#endif
|
||||
|
||||
size_t byte_size(size_t num_elements, size_t element_size)
|
||||
{
|
||||
size_t amount;
|
||||
if (__builtin_mul_overflow(num_elements, element_size, &amount))
|
||||
throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Amount of memory requested to allocate is more than allowed");
|
||||
return amount;
|
||||
}
|
||||
|
||||
size_t minimum_memory_for_elements(size_t num_elements, size_t element_size, size_t pad_left, size_t pad_right)
|
||||
{
|
||||
size_t amount;
|
||||
if (__builtin_add_overflow(byte_size(num_elements, element_size), pad_left + pad_right, &amount))
|
||||
throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Amount of memory requested to allocate is more than allowed");
|
||||
return amount;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
/// Used for left padding of PODArray when empty
|
||||
const char empty_pod_array[empty_pod_array_size]{};
|
||||
|
||||
@ -25,4 +63,5 @@ template class PODArray<Int8, 4096, Allocator<false>, 0, 0>;
|
||||
template class PODArray<Int16, 4096, Allocator<false>, 0, 0>;
|
||||
template class PODArray<Int32, 4096, Allocator<false>, 0, 0>;
|
||||
template class PODArray<Int64, 4096, Allocator<false>, 0, 0>;
|
||||
|
||||
}
|
||||
|
@ -1,27 +1,21 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/Allocator.h>
|
||||
#include <Common/BitHelpers.h>
|
||||
#include <Common/memcpySmall.h>
|
||||
#include <Common/PODArray_fwd.h>
|
||||
#include <base/getPageSize.h>
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <cstring>
|
||||
#include <cstddef>
|
||||
#include <cassert>
|
||||
#include <algorithm>
|
||||
#include <memory>
|
||||
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
||||
#include <base/strong_typedef.h>
|
||||
#include <base/getPageSize.h>
|
||||
|
||||
#include <Common/Allocator.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/BitHelpers.h>
|
||||
#include <Common/memcpySmall.h>
|
||||
|
||||
#ifndef NDEBUG
|
||||
#include <sys/mman.h>
|
||||
#include <sys/mman.h>
|
||||
#endif
|
||||
|
||||
#include <Common/PODArray_fwd.h>
|
||||
|
||||
/** Whether we can use memcpy instead of a loop with assignment to T from U.
|
||||
* It is Ok if types are the same. And if types are integral and of the same size,
|
||||
* example: char, signed char, unsigned char.
|
||||
@ -35,12 +29,6 @@ constexpr bool memcpy_can_be_used_for_assignment = std::is_same_v<T, U>
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_MPROTECT;
|
||||
extern const int CANNOT_ALLOCATE_MEMORY;
|
||||
}
|
||||
|
||||
/** A dynamic array for POD types.
|
||||
* Designed for a small number of large arrays (rather than a lot of small ones).
|
||||
* To be more precise - for use in ColumnVector.
|
||||
@ -77,6 +65,19 @@ namespace ErrorCodes
|
||||
static constexpr size_t empty_pod_array_size = 1024;
|
||||
extern const char empty_pod_array[empty_pod_array_size];
|
||||
|
||||
namespace PODArrayDetails
|
||||
{
|
||||
|
||||
void protectMemoryRegion(void * addr, size_t len, int prot);
|
||||
|
||||
/// The amount of memory occupied by the num_elements of the elements.
|
||||
size_t byte_size(size_t num_elements, size_t element_size); /// NOLINT
|
||||
|
||||
/// Minimum amount of memory to allocate for num_elements, including padding.
|
||||
size_t minimum_memory_for_elements(size_t num_elements, size_t element_size, size_t pad_left, size_t pad_right); /// NOLINT
|
||||
|
||||
};
|
||||
|
||||
/** Base class that depend only on size of element, not on element itself.
|
||||
* You can static_cast to this class if you want to insert some data regardless to the actual type T.
|
||||
*/
|
||||
@ -102,27 +103,9 @@ protected:
|
||||
char * c_end = null;
|
||||
char * c_end_of_storage = null; /// Does not include pad_right.
|
||||
|
||||
/// The amount of memory occupied by the num_elements of the elements.
|
||||
static size_t byte_size(size_t num_elements) /// NOLINT
|
||||
{
|
||||
size_t amount;
|
||||
if (__builtin_mul_overflow(num_elements, ELEMENT_SIZE, &amount))
|
||||
throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Amount of memory requested to allocate is more than allowed");
|
||||
return amount;
|
||||
}
|
||||
|
||||
/// Minimum amount of memory to allocate for num_elements, including padding.
|
||||
static size_t minimum_memory_for_elements(size_t num_elements)
|
||||
{
|
||||
size_t amount;
|
||||
if (__builtin_add_overflow(byte_size(num_elements), pad_left + pad_right, &amount))
|
||||
throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Amount of memory requested to allocate is more than allowed");
|
||||
return amount;
|
||||
}
|
||||
|
||||
void alloc_for_num_elements(size_t num_elements) /// NOLINT
|
||||
{
|
||||
alloc(minimum_memory_for_elements(num_elements));
|
||||
alloc(PODArrayDetails::minimum_memory_for_elements(num_elements, ELEMENT_SIZE, pad_left, pad_right));
|
||||
}
|
||||
|
||||
template <typename ... TAllocatorParams>
|
||||
@ -188,7 +171,7 @@ protected:
|
||||
// The allocated memory should be multiplication of ELEMENT_SIZE to hold the element, otherwise,
|
||||
// memory issue such as corruption could appear in edge case.
|
||||
realloc(std::max(integerRoundUp(initial_bytes, ELEMENT_SIZE),
|
||||
minimum_memory_for_elements(1)),
|
||||
PODArrayDetails::minimum_memory_for_elements(1, ELEMENT_SIZE, pad_left, pad_right)),
|
||||
std::forward<TAllocatorParams>(allocator_params)...);
|
||||
}
|
||||
else
|
||||
@ -208,8 +191,7 @@ protected:
|
||||
if (right_rounded_down > left_rounded_up)
|
||||
{
|
||||
size_t length = right_rounded_down - left_rounded_up;
|
||||
if (0 != mprotect(left_rounded_up, length, prot))
|
||||
throw ErrnoException(ErrorCodes::CANNOT_MPROTECT, "Cannot mprotect memory region");
|
||||
PODArrayDetails::protectMemoryRegion(left_rounded_up, length, prot);
|
||||
}
|
||||
}
|
||||
|
||||
@ -232,14 +214,14 @@ public:
|
||||
void reserve(size_t n, TAllocatorParams &&... allocator_params)
|
||||
{
|
||||
if (n > capacity())
|
||||
realloc(roundUpToPowerOfTwoOrZero(minimum_memory_for_elements(n)), std::forward<TAllocatorParams>(allocator_params)...);
|
||||
realloc(roundUpToPowerOfTwoOrZero(PODArrayDetails::minimum_memory_for_elements(n, ELEMENT_SIZE, pad_left, pad_right)), std::forward<TAllocatorParams>(allocator_params)...);
|
||||
}
|
||||
|
||||
template <typename ... TAllocatorParams>
|
||||
void reserve_exact(size_t n, TAllocatorParams &&... allocator_params) /// NOLINT
|
||||
{
|
||||
if (n > capacity())
|
||||
realloc(minimum_memory_for_elements(n), std::forward<TAllocatorParams>(allocator_params)...);
|
||||
realloc(PODArrayDetails::minimum_memory_for_elements(n, ELEMENT_SIZE, pad_left, pad_right), std::forward<TAllocatorParams>(allocator_params)...);
|
||||
}
|
||||
|
||||
template <typename ... TAllocatorParams>
|
||||
@ -258,7 +240,7 @@ public:
|
||||
|
||||
void resize_assume_reserved(const size_t n) /// NOLINT
|
||||
{
|
||||
c_end = c_start + byte_size(n);
|
||||
c_end = c_start + PODArrayDetails::byte_size(n, ELEMENT_SIZE);
|
||||
}
|
||||
|
||||
const char * raw_data() const /// NOLINT
|
||||
@ -339,7 +321,7 @@ public:
|
||||
explicit PODArray(size_t n)
|
||||
{
|
||||
this->alloc_for_num_elements(n);
|
||||
this->c_end += this->byte_size(n);
|
||||
this->c_end += PODArrayDetails::byte_size(n, sizeof(T));
|
||||
}
|
||||
|
||||
PODArray(size_t n, const T & x)
|
||||
@ -411,9 +393,9 @@ public:
|
||||
if (n > old_size)
|
||||
{
|
||||
this->reserve(n);
|
||||
memset(this->c_end, 0, this->byte_size(n - old_size));
|
||||
memset(this->c_end, 0, PODArrayDetails::byte_size(n - old_size, sizeof(T)));
|
||||
}
|
||||
this->c_end = this->c_start + this->byte_size(n);
|
||||
this->c_end = this->c_start + PODArrayDetails::byte_size(n, sizeof(T));
|
||||
}
|
||||
|
||||
void resize_fill(size_t n, const T & value) /// NOLINT
|
||||
@ -424,7 +406,7 @@ public:
|
||||
this->reserve(n);
|
||||
std::fill(t_end(), t_end() + n - old_size, value);
|
||||
}
|
||||
this->c_end = this->c_start + this->byte_size(n);
|
||||
this->c_end = this->c_start + PODArrayDetails::byte_size(n, sizeof(T));
|
||||
}
|
||||
|
||||
template <typename U, typename ... TAllocatorParams>
|
||||
@ -487,7 +469,7 @@ public:
|
||||
if (required_capacity > this->capacity())
|
||||
this->reserve(roundUpToPowerOfTwoOrZero(required_capacity), std::forward<TAllocatorParams>(allocator_params)...);
|
||||
|
||||
size_t bytes_to_copy = this->byte_size(from_end - from_begin);
|
||||
size_t bytes_to_copy = PODArrayDetails::byte_size(from_end - from_begin, sizeof(T));
|
||||
if (bytes_to_copy)
|
||||
{
|
||||
memcpy(this->c_end, reinterpret_cast<const void *>(rhs.begin() + from_begin), bytes_to_copy);
|
||||
@ -502,7 +484,7 @@ public:
|
||||
static_assert(pad_right_ >= PADDING_FOR_SIMD - 1);
|
||||
static_assert(sizeof(T) == sizeof(*from_begin));
|
||||
insertPrepare(from_begin, from_end, std::forward<TAllocatorParams>(allocator_params)...);
|
||||
size_t bytes_to_copy = this->byte_size(from_end - from_begin);
|
||||
size_t bytes_to_copy = PODArrayDetails::byte_size(from_end - from_begin, sizeof(T));
|
||||
memcpySmallAllowReadWriteOverflow15(this->c_end, reinterpret_cast<const void *>(&*from_begin), bytes_to_copy);
|
||||
this->c_end += bytes_to_copy;
|
||||
}
|
||||
@ -513,11 +495,11 @@ public:
|
||||
{
|
||||
static_assert(memcpy_can_be_used_for_assignment<std::decay_t<T>, std::decay_t<decltype(*from_begin)>>);
|
||||
|
||||
size_t bytes_to_copy = this->byte_size(from_end - from_begin);
|
||||
size_t bytes_to_copy = PODArrayDetails::byte_size(from_end - from_begin, sizeof(T));
|
||||
if (!bytes_to_copy)
|
||||
return;
|
||||
|
||||
size_t bytes_to_move = this->byte_size(end() - it);
|
||||
size_t bytes_to_move = PODArrayDetails::byte_size(end() - it, sizeof(T));
|
||||
|
||||
insertPrepare(from_begin, from_end);
|
||||
|
||||
@ -545,10 +527,10 @@ public:
|
||||
if (required_capacity > this->capacity())
|
||||
this->reserve(roundUpToPowerOfTwoOrZero(required_capacity), std::forward<TAllocatorParams>(allocator_params)...);
|
||||
|
||||
size_t bytes_to_copy = this->byte_size(copy_size);
|
||||
size_t bytes_to_copy = PODArrayDetails::byte_size(copy_size, sizeof(T));
|
||||
if (bytes_to_copy)
|
||||
{
|
||||
auto begin = this->c_start + this->byte_size(start_index);
|
||||
auto begin = this->c_start + PODArrayDetails::byte_size(start_index, sizeof(T));
|
||||
memcpy(this->c_end, reinterpret_cast<const void *>(&*begin), bytes_to_copy);
|
||||
this->c_end += bytes_to_copy;
|
||||
}
|
||||
@ -560,7 +542,7 @@ public:
|
||||
static_assert(memcpy_can_be_used_for_assignment<std::decay_t<T>, std::decay_t<decltype(*from_begin)>>);
|
||||
this->assertNotIntersects(from_begin, from_end);
|
||||
|
||||
size_t bytes_to_copy = this->byte_size(from_end - from_begin);
|
||||
size_t bytes_to_copy = PODArrayDetails::byte_size(from_end - from_begin, sizeof(T));
|
||||
if (bytes_to_copy)
|
||||
{
|
||||
memcpy(this->c_end, reinterpret_cast<const void *>(&*from_begin), bytes_to_copy);
|
||||
@ -593,13 +575,13 @@ public:
|
||||
/// arr1 takes ownership of the heap memory of arr2.
|
||||
arr1.c_start = arr2.c_start;
|
||||
arr1.c_end_of_storage = arr1.c_start + heap_allocated - arr2.pad_right - arr2.pad_left;
|
||||
arr1.c_end = arr1.c_start + this->byte_size(heap_size);
|
||||
arr1.c_end = arr1.c_start + PODArrayDetails::byte_size(heap_size, sizeof(T));
|
||||
|
||||
/// Allocate stack space for arr2.
|
||||
arr2.alloc(stack_allocated, std::forward<TAllocatorParams>(allocator_params)...);
|
||||
/// Copy the stack content.
|
||||
memcpy(arr2.c_start, stack_c_start, this->byte_size(stack_size));
|
||||
arr2.c_end = arr2.c_start + this->byte_size(stack_size);
|
||||
memcpy(arr2.c_start, stack_c_start, PODArrayDetails::byte_size(stack_size, sizeof(T)));
|
||||
arr2.c_end = arr2.c_start + PODArrayDetails::byte_size(stack_size, sizeof(T));
|
||||
};
|
||||
|
||||
auto do_move = [&](PODArray & src, PODArray & dest)
|
||||
@ -608,8 +590,8 @@ public:
|
||||
{
|
||||
dest.dealloc();
|
||||
dest.alloc(src.allocated_bytes(), std::forward<TAllocatorParams>(allocator_params)...);
|
||||
memcpy(dest.c_start, src.c_start, this->byte_size(src.size()));
|
||||
dest.c_end = dest.c_start + this->byte_size(src.size());
|
||||
memcpy(dest.c_start, src.c_start, PODArrayDetails::byte_size(src.size(), sizeof(T)));
|
||||
dest.c_end = dest.c_start + PODArrayDetails::byte_size(src.size(), sizeof(T));
|
||||
|
||||
src.c_start = Base::null;
|
||||
src.c_end = Base::null;
|
||||
@ -666,8 +648,8 @@ public:
|
||||
this->c_end_of_storage = this->c_start + rhs_allocated - Base::pad_right - Base::pad_left;
|
||||
rhs.c_end_of_storage = rhs.c_start + lhs_allocated - Base::pad_right - Base::pad_left;
|
||||
|
||||
this->c_end = this->c_start + this->byte_size(rhs_size);
|
||||
rhs.c_end = rhs.c_start + this->byte_size(lhs_size);
|
||||
this->c_end = this->c_start + PODArrayDetails::byte_size(rhs_size, sizeof(T));
|
||||
rhs.c_end = rhs.c_start + PODArrayDetails::byte_size(lhs_size, sizeof(T));
|
||||
}
|
||||
else if (this->isAllocatedFromStack() && !rhs.isAllocatedFromStack())
|
||||
{
|
||||
@ -702,7 +684,7 @@ public:
|
||||
if (required_capacity > this->capacity())
|
||||
this->reserve_exact(required_capacity, std::forward<TAllocatorParams>(allocator_params)...);
|
||||
|
||||
size_t bytes_to_copy = this->byte_size(required_capacity);
|
||||
size_t bytes_to_copy = PODArrayDetails::byte_size(required_capacity, sizeof(T));
|
||||
if (bytes_to_copy)
|
||||
memcpy(this->c_start, reinterpret_cast<const void *>(&*from_begin), bytes_to_copy);
|
||||
|
||||
|
@ -28,6 +28,40 @@ namespace CurrentMetrics
|
||||
extern const Metric GlobalThreadScheduled;
|
||||
}
|
||||
|
||||
class JobWithPriority
|
||||
{
|
||||
public:
|
||||
using Job = std::function<void()>;
|
||||
|
||||
Job job;
|
||||
Priority priority;
|
||||
CurrentMetrics::Increment metric_increment;
|
||||
DB::OpenTelemetry::TracingContextOnThread thread_trace_context;
|
||||
|
||||
/// Call stacks of all jobs' schedulings leading to this one
|
||||
std::vector<StackTrace::FramePointers> frame_pointers;
|
||||
bool enable_job_stack_trace = false;
|
||||
|
||||
JobWithPriority(
|
||||
Job job_, Priority priority_, CurrentMetrics::Metric metric,
|
||||
const DB::OpenTelemetry::TracingContextOnThread & thread_trace_context_,
|
||||
bool capture_frame_pointers)
|
||||
: job(job_), priority(priority_), metric_increment(metric),
|
||||
thread_trace_context(thread_trace_context_), enable_job_stack_trace(capture_frame_pointers)
|
||||
{
|
||||
if (!capture_frame_pointers)
|
||||
return;
|
||||
/// Save all previous jobs call stacks and append with current
|
||||
frame_pointers = DB::Exception::thread_frame_pointers;
|
||||
frame_pointers.push_back(StackTrace().getFramePointers());
|
||||
}
|
||||
|
||||
bool operator<(const JobWithPriority & rhs) const
|
||||
{
|
||||
return priority > rhs.priority; // Reversed for `priority_queue` max-heap to yield minimum value (i.e. highest priority) first
|
||||
}
|
||||
};
|
||||
|
||||
static constexpr auto DEFAULT_THREAD_NAME = "ThreadPool";
|
||||
|
||||
template <typename Thread>
|
||||
|
@ -20,9 +20,10 @@
|
||||
#include <Common/ThreadPool_fwd.h>
|
||||
#include <Common/Priority.h>
|
||||
#include <Common/StackTrace.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <base/scope_guard.h>
|
||||
|
||||
class JobWithPriority;
|
||||
|
||||
/** Very simple thread pool similar to boost::threadpool.
|
||||
* Advantages:
|
||||
* - catches exceptions and rethrows on wait.
|
||||
@ -128,37 +129,6 @@ private:
|
||||
bool threads_remove_themselves = true;
|
||||
const bool shutdown_on_exception = true;
|
||||
|
||||
struct JobWithPriority
|
||||
{
|
||||
Job job;
|
||||
Priority priority;
|
||||
CurrentMetrics::Increment metric_increment;
|
||||
DB::OpenTelemetry::TracingContextOnThread thread_trace_context;
|
||||
|
||||
/// Call stacks of all jobs' schedulings leading to this one
|
||||
std::vector<StackTrace::FramePointers> frame_pointers;
|
||||
bool enable_job_stack_trace = false;
|
||||
|
||||
JobWithPriority(
|
||||
Job job_, Priority priority_, CurrentMetrics::Metric metric,
|
||||
const DB::OpenTelemetry::TracingContextOnThread & thread_trace_context_,
|
||||
bool capture_frame_pointers)
|
||||
: job(job_), priority(priority_), metric_increment(metric),
|
||||
thread_trace_context(thread_trace_context_), enable_job_stack_trace(capture_frame_pointers)
|
||||
{
|
||||
if (!capture_frame_pointers)
|
||||
return;
|
||||
/// Save all previous jobs call stacks and append with current
|
||||
frame_pointers = DB::Exception::thread_frame_pointers;
|
||||
frame_pointers.push_back(StackTrace().getFramePointers());
|
||||
}
|
||||
|
||||
bool operator<(const JobWithPriority & rhs) const
|
||||
{
|
||||
return priority > rhs.priority; // Reversed for `priority_queue` max-heap to yield minimum value (i.e. highest priority) first
|
||||
}
|
||||
};
|
||||
|
||||
boost::heap::priority_queue<JobWithPriority> jobs;
|
||||
std::list<Thread> threads;
|
||||
std::exception_ptr first_exception;
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Common/ThreadStatus.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <base/getPageSize.h>
|
||||
#include <base/errnoToString.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include "Common/ZooKeeper/ZooKeeperConstants.h"
|
||||
#include <Common/ZooKeeper/ZooKeeperConstants.h>
|
||||
#include <Common/thread_local_rng.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperImpl.h>
|
||||
|
||||
#include <IO/Operators.h>
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <IO/ReadBufferFromFileDescriptor.h>
|
||||
#include <IO/WriteBufferFromFileDescriptor.h>
|
||||
#include <IO/copyData.h>
|
||||
#include <iostream>
|
||||
|
||||
/** This example shows how we can proxy stdin to ShellCommand and obtain stdout in streaming fashion. */
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <cstring>
|
||||
#include <sys/types.h> /// ssize_t
|
||||
|
||||
#ifdef __SSE2__
|
||||
# include <emmintrin.h>
|
||||
|
@ -11,6 +11,7 @@
|
||||
#include "libaccel_config.h"
|
||||
#include <Common/MemorySanitizer.h>
|
||||
#include <base/scope_guard.h>
|
||||
#include <base/getPageSize.h>
|
||||
#include <immintrin.h>
|
||||
|
||||
|
||||
|
@ -207,7 +207,6 @@ PostgreSQLTableStructure::ColumnsInfoPtr readNamesAndTypesList(
|
||||
|
||||
columns.push_back(NameAndTypePair(column_name, data_type));
|
||||
auto attgenerated = std::get<6>(row);
|
||||
LOG_TEST(&Poco::Logger::get("kssenii"), "KSSENII: attgenerated: {}", attgenerated);
|
||||
|
||||
attributes.emplace(
|
||||
column_name,
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Common/filesystemHelpers.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/atomicRename.h>
|
||||
#include <Common/formatReadable.h>
|
||||
#include <Disks/IO/createReadBufferFromFileBase.h>
|
||||
#include <Disks/loadLocalDiskConfig.h>
|
||||
#include <Disks/TemporaryFileOnDisk.h>
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/ReadBufferFromEmptyFile.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <Common/formatReadable.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/formatReadable.h>
|
||||
#include <Common/formatReadable.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/formatReadable.h>
|
||||
#include <Common/formatReadable.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/formatReadable.h>
|
||||
#include <Common/formatReadable.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,7 +1,8 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include "Common/Exception.h"
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/thread_local_rng.h>
|
||||
#include <Common/NaNUtils.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
|
@ -6,6 +6,8 @@
|
||||
#include <base/errnoToString.h>
|
||||
#include <zip.h>
|
||||
#include <boost/algorithm/string/predicate.hpp>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Poco/Logger.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <algorithm>
|
||||
#include <bit>
|
||||
#include <base/types.h>
|
||||
#include <Common/BitHelpers.h>
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <IO/MMapReadBufferFromFileWithCache.h>
|
||||
#include <base/getPageSize.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -41,6 +41,7 @@
|
||||
#include <IO/PeekableReadBuffer.h>
|
||||
#include <IO/VarInt.h>
|
||||
|
||||
#include <pcg_random.hpp>
|
||||
#include <double-conversion/double-conversion.h>
|
||||
|
||||
static constexpr auto DEFAULT_MAX_STRING_SIZE = 1_GiB;
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <base/scope_guard.h>
|
||||
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/formatReadable.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Interpreters/TemporaryDataOnDisk.h>
|
||||
|
||||
#include <Compression/CompressedWriteBuffer.h>
|
||||
#include <Common/formatReadable.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/thread_local_rng.h>
|
||||
|
||||
|
@ -30,6 +30,7 @@
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/formatReadable.h>
|
||||
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <Common/thread_local_rng.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Common/PoolId.h>
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Processors/Formats/Impl/MySQLOutputFormat.h>
|
||||
#include <Common/formatReadable.h>
|
||||
#include <Core/MySQL/PacketsGeneric.h>
|
||||
#include <Core/MySQL/PacketsProtocolBinary.h>
|
||||
#include <Core/MySQL/PacketsProtocolText.h>
|
||||
|
@ -13,6 +13,7 @@
|
||||
#include <Columns/ColumnMap.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Common/config_version.h>
|
||||
#include <Common/formatReadable.h>
|
||||
|
||||
#if USE_SNAPPY
|
||||
#include <snappy.h>
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <IO/WriteBuffer.h>
|
||||
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/formatReadable.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
#include <Core/SortCursor.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/formatReadable.h>
|
||||
#include <Interpreters/sortBlock.h>
|
||||
#include <base/range.h>
|
||||
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h>
|
||||
#include <Core/ProtocolDefines.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/formatReadable.h>
|
||||
|
||||
#include <Processors/Transforms/SquashingChunksTransform.h>
|
||||
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/formatReadable.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Storages/MergeTree/localBackup.h>
|
||||
#include <Backups/BackupEntryFromSmallFile.h>
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Common/parseRemoteDescription.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/NamedCollections/NamedCollections.h>
|
||||
#include <Common/thread_local_rng.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
#include <Core/PostgreSQL/PoolWithFailover.h>
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Common/NamedCollections/NamedCollections.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/Throttler.h>
|
||||
#include <Common/formatReadable.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <boost/algorithm/string/predicate.hpp>
|
||||
|
||||
|
@ -32,12 +32,13 @@
|
||||
#include <Common/NamedCollections/NamedCollections.h>
|
||||
#include <Common/ProxyConfigurationResolverProvider.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/thread_local_rng.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <IO/ReadWriteBufferFromHTTP.h>
|
||||
#include <IO/HTTPHeaderEntries.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Poco/Net/HTTPRequest.h>
|
||||
#include <regex>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
|
Loading…
Reference in New Issue
Block a user