This commit is contained in:
Pavel Kartavyy 2016-01-18 15:00:13 +03:00
commit 83ba8e4dba
299 changed files with 6660 additions and 3908 deletions

View File

@ -38,6 +38,9 @@ public:
{
num_agruments = arguments.size();
if (0 == num_agruments)
throw Exception("Array aggregate functions requires at least one argument", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
DataTypes nested_arguments;
for (int i = 0; i < num_agruments; ++i)
{
@ -46,6 +49,7 @@ public:
else
throw Exception("Illegal type " + arguments[i]->getName() + " of argument #" + toString(i + 1) + " for aggregate function " + getName() + ". Must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
nested_func->setArguments(nested_arguments);
}

View File

@ -17,6 +17,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int TOO_LARGE_ARRAY_SIZE;
}
/// Частный случай - реализация для числовых типов.
template <typename T>

View File

@ -17,6 +17,14 @@
#include <DB/DataTypes/DataTypeArray.h>
namespace DB
{
namespace ErrorCodes
{
extern const int TOO_LARGE_ARRAY_SIZE;
}
}
/** Алгоритм реализовал Алексей Борзенков https://███████████.yandex-team.ru/snaury
* Ему принадлежит авторство кода и половины комментариев в данном namespace,
* за исключением слияния, сериализации и сортировки, а также выбора типов и других изменений.

View File

@ -15,6 +15,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int TOO_SLOW;
extern const int TOO_LESS_ARGUMENTS_FOR_FUNCTION;
extern const int TOO_MUCH_ARGUMENTS_FOR_FUNCTION;
extern const int SYNTAX_ERROR;
}
/// helper type for comparing `std::pair`s using solely the .first member
template <template <typename> class Comparator>
struct ComparePairFirst final

View File

@ -12,9 +12,15 @@
namespace DB
{
namespace ErrorCodes
{
extern const int AGGREGATE_FUNCTION_DOESNT_ALLOW_PARAMETERS;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
typedef char * AggregateDataPtr;
typedef const char * ConstAggregateDataPtr;
using AggregateDataPtr = char *;
using ConstAggregateDataPtr = const char *;
/** Интерфейс для агрегатных функций.

View File

@ -20,6 +20,16 @@
/// Вызов quantile занимает O(sample_count log sample_count), если после предыдущего вызова quantile был хотя бы один вызов insert. Иначе, O(1).
/// То есть, имеет смысл сначала добавлять, потом получать квантили, не добавляя.
namespace DB
{
namespace ErrorCodes
{
extern const int MEMORY_LIMIT_EXCEEDED;
}
}
namespace detail
{
const size_t DEFAULT_SAMPLE_COUNT = 8192;

View File

@ -12,6 +12,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NETWORK_ERROR;
extern const int SOCKET_TIMEOUT;
extern const int LOGICAL_ERROR;
}
/** Пул соединений с отказоустойчивостью.
* Инициализируется несколькими другими IConnectionPool-ами.
* При получении соединения, пытается создать или выбрать живое соединение из какого-нибудь пула,

View File

@ -16,6 +16,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int PARAMETER_OUT_OF_BOUND;
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
}
/** Столбец состояний агрегатных функций.
* Представлен в виде массива указателей на состояния агрегатных функций (data).

View File

@ -5,7 +5,6 @@
#include <Poco/SharedPtr.h>
#include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/Common/Arena.h>
#include <DB/Columns/IColumn.h>
@ -16,6 +15,13 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int NOT_IMPLEMENTED;
extern const int BAD_ARGUMENTS;
}
using Poco::SharedPtr;
/** Cтолбeц значений типа массив.

View File

@ -4,7 +4,6 @@
#include <DB/Core/Field.h>
#include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/Columns/ColumnVector.h>
#include <DB/Columns/IColumn.h>
#include <DB/Columns/ColumnsCommon.h>
@ -14,6 +13,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_INSERT_ELEMENT_INTO_CONSTANT_COLUMN;
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
extern const int NOT_IMPLEMENTED;
extern const int LOGICAL_ERROR;
}
using Poco::SharedPtr;

View File

@ -1,40 +1,34 @@
#pragma once
#include <DB/Core/NamesAndTypes.h>
#include <DB/Columns/IColumnDummy.h>
#include <DB/Interpreters/ExpressionActions.h>
namespace DB
{
class ExpressionActions;
/** Столбец, содержащий лямбда-выражение.
* Ведёт себя как столбец-константа. Содержит выражение, но не входные или выходные данные.
*/
class ColumnExpression final : public IColumnDummy
{
private:
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
public:
ColumnExpression(size_t s_, ExpressionActionsPtr expression_, const NamesAndTypes & arguments_, DataTypePtr return_type_, std::string return_name_)
: IColumnDummy(s_), expression(expression_), arguments(arguments_), return_type(return_type_), return_name(return_name_) {}
ColumnExpression(size_t s_, ExpressionActionsPtr expression_, const NamesAndTypes & arguments_, DataTypePtr return_type_, String return_name_);
ColumnExpression(size_t s_, ExpressionActionsPtr expression_, const NamesAndTypesList & arguments_, DataTypePtr return_type_, String return_name_);
ColumnExpression(size_t s_, ExpressionActionsPtr expression_, const NamesAndTypesList & arguments_, DataTypePtr return_type_, std::string return_name_)
: IColumnDummy(s_), expression(expression_), arguments(arguments_.begin(), arguments_.end()), return_type(return_type_), return_name(return_name_) {}
std::string getName() const override;
ColumnPtr cloneDummy(size_t s_) const override;
std::string getName() const override { return "ColumnExpression"; }
ColumnPtr cloneDummy(size_t s_) const override { return new ColumnExpression(s_, expression, arguments, return_type, return_name); }
const ExpressionActionsPtr & getExpression() const { return expression; }
const DataTypePtr & getReturnType() const { return return_type; }
const std::string & getReturnName() const { return return_name; }
const NamesAndTypes & getArguments() const { return arguments; }
Names getArgumentNames() const
{
Names res(arguments.size());
for (size_t i = 0; i < arguments.size(); ++i)
res[i] = arguments[i].name;
return res;
}
const ExpressionActionsPtr & getExpression() const;
const DataTypePtr & getReturnType() const;
const std::string & getReturnName() const;
const NamesAndTypes & getArguments() const;
Names getArgumentNames() const;
private:
ExpressionActionsPtr expression;

View File

@ -6,11 +6,20 @@
#include <DB/Common/Arena.h>
#include <DB/Columns/IColumn.h>
#include <DB/IO/ReadHelpers.h>
#include <DB/IO/WriteHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int TOO_LARGE_STRING_SIZE;
extern const int SIZE_OF_ARRAY_DOESNT_MATCH_SIZE_OF_FIXEDARRAY_COLUMN;
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
extern const int PARAMETER_OUT_OF_BOUND;
}
/** Cтолбeц значений типа "строка фиксированной длины".
* Если вставить строку меньшей длины, то она будет дополнена нулевыми байтами.
*/

View File

@ -14,6 +14,13 @@
namespace DB
{
namespace ErrorCodes
{
extern const int PARAMETER_OUT_OF_BOUND;
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
}
/** Cтолбeц значений типа "строка".
*/
class ColumnString final : public IColumn

View File

@ -8,6 +8,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
extern const int CANNOT_INSERT_VALUE_OF_DIFFERENT_SIZE_INTO_TUPLE;
}
/** Столбец, который всего лишь группирует вместе несколько других столбцов.
*/

View File

@ -3,7 +3,6 @@
#include <string.h>
#include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/Common/Arena.h>
#include <DB/IO/WriteBuffer.h>
@ -11,10 +10,20 @@
#include <DB/Columns/IColumn.h>
#if defined(__x86_64__)
#include <emmintrin.h>
#endif
namespace DB
{
namespace ErrorCodes
{
extern const int PARAMETER_OUT_OF_BOUND;
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
}
/** Штука для сравнения чисел.
* Целые числа сравниваются как обычно.
@ -283,17 +292,20 @@ public:
if (result_size_hint)
res_data.reserve(result_size_hint > 0 ? result_size_hint : size);
/** Чуть более оптимизированная версия.
* Исходит из допущения, что часто куски последовательно идущих значений
* полностью проходят или полностью не проходят фильтр.
* Поэтому, будем оптимистично проверять куски по 16 значений.
*/
const UInt8 * filt_pos = &filt[0];
const UInt8 * filt_end = filt_pos + size;
const UInt8 * filt_end_sse = filt_pos + size / 16 * 16;
const T * data_pos = &data[0];
#if defined(__x86_64__)
/** Чуть более оптимизированная версия.
* Исходит из допущения, что часто куски последовательно идущих значений
* полностью проходят или полностью не проходят фильтр.
* Поэтому, будем оптимистично проверять куски по SIMD_BYTES значений.
*/
static constexpr size_t SIMD_BYTES = 16;
const __m128i zero16 = _mm_setzero_si128();
const UInt8 * filt_end_sse = filt_pos + size / SIMD_BYTES * SIMD_BYTES;
while (filt_pos < filt_end_sse)
{
@ -305,18 +317,19 @@ public:
}
else if (0xFFFF == mask)
{
res_data.insert(data_pos, data_pos + 16);
res_data.insert(data_pos, data_pos + SIMD_BYTES);
}
else
{
for (size_t i = 0; i < 16; ++i)
for (size_t i = 0; i < SIMD_BYTES; ++i)
if (filt_pos[i])
res_data.push_back(data_pos[i]);
}
filt_pos += 16;
data_pos += 16;
filt_pos += SIMD_BYTES;
data_pos += SIMD_BYTES;
}
#endif
while (filt_pos < filt_end)
{

View File

@ -7,13 +7,18 @@
#include <DB/Core/Field.h>
#include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/Core/StringRef.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_GET_SIZE_OF_FIELD;
extern const int NOT_IMPLEMENTED;
}
using Poco::SharedPtr;
class IColumn;

View File

@ -7,6 +7,13 @@
namespace DB
{
namespace ErrorCodes
{
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
extern const int NOT_IMPLEMENTED;
}
/** Базовый класс для столбцов-констант, содержащих значение, не входящее в Field.
* Не является полноценым столбцом и используется особым образом.
*/

View File

@ -1,6 +1,5 @@
#pragma once
#include <DB/Core/ErrorCodes.h>
#include <DB/Common/Exception.h>
#include <common/logger_useful.h>
#include <common/singleton.h>
@ -59,9 +58,16 @@ struct AIOContext : private boost::noncopyable
}
};
namespace DB
{
namespace ErrorCodes
{
extern const int AIO_COMPLETION_ERROR;
extern const int AIO_SUBMIT_ERROR;
}
class AIOContextPool : public Singleton<AIOContextPool>
{

View File

@ -0,0 +1,11 @@
#pragma once
#if !defined(__x86_64__)
inline unsigned int _bit_scan_reverse(unsigned int x)
{
return sizeof(unsigned int) * 8 - 1 - __builtin_clz(x);
}
#endif

View File

@ -6,7 +6,18 @@
#include <DB/Common/MemoryTracker.h>
#include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h>
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;
}
}
/** При использовании AllocatorWithStackMemory, размещённом на стеке,

View File

@ -1,6 +1,7 @@
#pragma once
#include <DB/Common/Arena.h>
#include <DB/Common/ARMHelpers.h>
namespace DB

View File

@ -2,7 +2,6 @@
#include <unicode/ucol.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/Common/Exception.h>
#include <DB/IO/WriteHelpers.h>
@ -12,6 +11,17 @@
#include <boost/noncopyable.hpp>
namespace DB
{
namespace ErrorCodes
{
extern const int UNSUPPORTED_COLLATION_LOCALE;
extern const int COLLATION_COMPARISON_FAILED;
}
}
class Collator : private boost::noncopyable
{
public:
@ -61,4 +71,4 @@ public:
private:
std::string locale;
UCollator * collator;
};
};

View File

@ -5,11 +5,16 @@
#include <DB/IO/ReadHelpers.h>
#include <DB/IO/WriteHelpers.h>
#include <DB/Core/Defines.h>
#include <DB/Core/ErrorCodes.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NO_AVAILABLE_DATA;
}
/** Компактный массив для хранения данных, размер content_width, в битах, которых составляет
* меньше одного байта. Вместо того, чтобы хранить каждое значение в отдельный
* байт, что приводит к растрате 37.5% пространства для content_width=5, CompactArray хранит

View File

@ -30,9 +30,14 @@ inline DB::UInt64 intHash64(DB::UInt64 x)
*/
inline DB::UInt64 intHashCRC32(DB::UInt64 x)
{
#if defined(__x86_64__)
DB::UInt64 crc = -1ULL;
asm("crc32q %[x], %[crc]\n" : [crc] "+r" (crc) : [x] "rm" (x));
return crc;
#else
/// На других платформах используем не обязательно CRC32. NOTE Это может сбить с толку.
return intHash64(x);
#endif
}

View File

@ -14,7 +14,6 @@
#include <DB/Core/Defines.h>
#include <DB/Core/Types.h>
#include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/IO/WriteBuffer.h>
#include <DB/IO/WriteHelpers.h>
@ -31,6 +30,16 @@
#endif
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int NO_AVAILABLE_DATA;
}
}
/** Состояние хэш-таблицы, которое влияет на свойства её ячеек.
* Используется в качестве параметра шаблона.
* Например, существует реализация мгновенно-очищаемой хэш-таблицы - ClearableHashMap.

View File

@ -1,7 +1,6 @@
#pragma once
#include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <algorithm>
#include <limits>

View File

@ -10,11 +10,20 @@
#include <DB/IO/ReadHelpers.h>
#include <DB/IO/WriteHelpers.h>
#include <DB/Core/Defines.h>
#include <DB/Core/ErrorCodes.h>
#include <cmath>
#include <cstring>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
}
namespace details
{

View File

@ -6,7 +6,6 @@
#include <chrono>
#include <Poco/ScopedLock.h>
#include <Poco/Mutex.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/Common/Exception.h>
#include <common/logger_useful.h>

View File

@ -16,7 +16,14 @@ class MemoryTracker
/// В целях тестирования exception safety - кидать исключение при каждом выделении памяти с указанной вероятностью.
double fault_probability = 0;
/// Односвязный список. Вся информация будет передаваться в следующие MemoryTracker-ы тоже. Они должны жить во время жизни данного MemoryTracker.
MemoryTracker * next = nullptr;
/// Если задано (например, "for user") - в сообщениях в логе будет указываться это описание.
const char * description = nullptr;
public:
MemoryTracker() {}
MemoryTracker(Int64 limit_) : limit(limit_) {}
~MemoryTracker();
@ -32,10 +39,7 @@ public:
/** А эту функцию имеет смысл вызывать после освобождения памяти.
*/
void free(Int64 size)
{
__sync_sub_and_fetch(&amount, size);
}
void free(Int64 size);
Int64 get() const
{
@ -47,10 +51,31 @@ public:
return peak;
}
void setLimit(Int64 limit_)
{
limit = limit_;
}
void setFaultProbability(double value)
{
fault_probability = value;
}
void setNext(MemoryTracker * elem)
{
next = elem;
}
void setDescription(const char * description_)
{
description = description_;
}
/// Обнулить накопленные данные.
void reset();
/// Вывести в лог информацию о пиковом потреблении памяти.
void logPeakMemoryUsage() const;
};

View File

@ -13,7 +13,6 @@
#include <DB/Common/Allocator.h>
#include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h>
namespace DB

View File

@ -5,10 +5,19 @@
#include <DB/Common/ProfileEvents.h>
#include <DB/Common/NetException.h>
#include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/Interpreters/Settings.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ALL_CONNECTION_TRIES_FAILED;
extern const int CANNOT_CLOCK_GETTIME;
}
}
namespace
{
/** Класс, который употребляется для того, чтобы оптимизировать выделение

View File

@ -40,6 +40,7 @@ struct RadixSortMallocAllocator
* что отношение порядка над ключами будет соответствовать отношению порядка над полученными беззнаковыми числами.
* Для float-ов это преобразование делает следующее:
* если выставлен знаковый бит, то переворачивает все остальные биты.
* При этом, NaN-ы оказываются больше всех нормальных чисел.
*/
template <typename KeyBits>
struct RadixSortFloatTransform

View File

@ -1,6 +1,5 @@
#pragma once
#include <DB/Core/ErrorCodes.h>
#include <DB/Common/Arena.h>
#include <common/likely.h>
#include <ext/range.hpp>

View File

@ -7,23 +7,45 @@
#include <stdint.h>
#include <string.h>
#if defined(__x86_64__)
#include <smmintrin.h>
#endif
namespace DB
{
namespace ErrorCodes
{
extern const int UNSUPPORTED_PARAMETER;
}
struct StringSearcherBase
{
#if defined(__x86_64__)
static constexpr auto n = sizeof(__m128i);
const int page_size = getpagesize();
bool page_safe(const void * const ptr) const
{
return ((page_size - 1) & reinterpret_cast<std::uintptr_t>(ptr)) <= page_size - n;
}
#endif
};
/// Performs case-sensitive and case-insensitive search of UTF-8 strings
template <bool CaseSensitive, bool ASCII> class StringSearcher;
/// Case-insensitive UTF-8 searcher
template <> class StringSearcher<false, false>
template <>
class StringSearcher<false, false> : private StringSearcherBase
{
private:
using UTF8SequenceBuffer = UInt8[6];
static constexpr auto n = sizeof(__m128i);
const int page_size = getpagesize();
/// string to be searched for
const UInt8 * const needle;
const std::size_t needle_size;
@ -32,6 +54,8 @@ template <> class StringSearcher<false, false>
bool first_needle_symbol_is_ascii{};
UInt8 l{};
UInt8 u{};
#if defined(__x86_64__)
/// vectors filled with `l` and `u`, for determining leftmost position of the first symbol
__m128i patl, patu;
/// lower and uppercase vectors of first 16 characters of `needle`
@ -39,11 +63,7 @@ template <> class StringSearcher<false, false>
int cachemask{};
std::size_t cache_valid_len{};
std::size_t cache_actual_len{};
bool page_safe(const void * const ptr) const
{
return ((page_size - 1) & reinterpret_cast<std::uintptr_t>(ptr)) <= page_size - n;
}
#endif
public:
StringSearcher(const char * const needle_, const std::size_t needle_size)
@ -74,6 +94,7 @@ public:
u = u_seq[0];
}
#if defined(__x86_64__)
/// for detecting leftmost position of the first symbol
patl = _mm_set1_epi8(l);
patu = _mm_set1_epi8(u);
@ -127,12 +148,14 @@ public:
}
}
}
#endif
}
bool compare(const UInt8 * pos) const
{
static const Poco::UTF8Encoding utf8;
#if defined(__x86_64__)
if (page_safe(pos))
{
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(pos));
@ -166,6 +189,7 @@ public:
return false;
}
#endif
if (*pos == l || *pos == u)
{
@ -196,6 +220,7 @@ public:
while (haystack < haystack_end)
{
#if defined(__x86_64__)
if (haystack + n <= haystack_end && page_safe(haystack))
{
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
@ -251,6 +276,7 @@ public:
continue;
}
}
#endif
if (haystack == haystack_end)
return haystack_end;
@ -280,13 +306,12 @@ public:
}
};
/// Case-insensitive ASCII searcher
template <> class StringSearcher<false, true>
template <>
class StringSearcher<false, true> : private StringSearcherBase
{
static constexpr auto n = sizeof(__m128i);
const int page_size = getpagesize();
private:
/// string to be searched for
const UInt8 * const needle;
const std::size_t needle_size;
@ -294,16 +319,14 @@ template <> class StringSearcher<false, true>
/// lower and uppercase variants of the first character in `needle`
UInt8 l{};
UInt8 u{};
#if defined(__x86_64__)
/// vectors filled with `l` and `u`, for determining leftmost position of the first symbol
__m128i patl, patu;
/// lower and uppercase vectors of first 16 characters of `needle`
__m128i cachel = _mm_setzero_si128(), cacheu = _mm_setzero_si128();
int cachemask{};
bool page_safe(const void * const ptr) const
{
return ((page_size - 1) & reinterpret_cast<std::uintptr_t>(ptr)) <= page_size - n;
}
#endif
public:
StringSearcher(const char * const needle_, const std::size_t needle_size)
@ -315,6 +338,7 @@ public:
l = static_cast<UInt8>(std::tolower(*needle));
u = static_cast<UInt8>(std::toupper(*needle));
#if defined(__x86_64__)
patl = _mm_set1_epi8(l);
patu = _mm_set1_epi8(u);
@ -333,10 +357,12 @@ public:
++needle_pos;
}
}
#endif
}
bool compare(const UInt8 * pos) const
{
#if defined(__x86_64__)
if (page_safe(pos))
{
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(pos));
@ -364,6 +390,7 @@ public:
return false;
}
#endif
if (*pos == l || *pos == u)
{
@ -387,6 +414,7 @@ public:
while (haystack < haystack_end)
{
#if defined(__x86_64__)
if (haystack + n <= haystack_end && page_safe(haystack))
{
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
@ -435,6 +463,7 @@ public:
continue;
}
}
#endif
if (haystack == haystack_end)
return haystack_end;
@ -459,29 +488,26 @@ public:
}
};
/// Case-sensitive searcher (both ASCII and UTF-8)
template <bool ASCII> class StringSearcher<true, ASCII>
template <bool ASCII>
class StringSearcher<true, ASCII> : private StringSearcherBase
{
static constexpr auto n = sizeof(__m128i);
const int page_size = getpagesize();
private:
/// string to be searched for
const UInt8 * const needle;
const std::size_t needle_size;
const UInt8 * const needle_end = needle + needle_size;
/// first character in `needle`
UInt8 first{};
#if defined(__x86_64__)
/// vector filled `first` for determining leftmost position of the first symbol
__m128i pattern;
/// vector of first 16 characters of `needle`
__m128i cache = _mm_setzero_si128();
int cachemask{};
bool page_safe(const void * const ptr) const
{
return ((page_size - 1) & reinterpret_cast<std::uintptr_t>(ptr)) <= page_size - n;
}
#endif
public:
StringSearcher(const char * const needle_, const std::size_t needle_size)
@ -491,6 +517,8 @@ public:
return;
first = *needle;
#if defined(__x86_64__)
pattern = _mm_set1_epi8(first);
auto needle_pos = needle;
@ -506,10 +534,12 @@ public:
++needle_pos;
}
}
#endif
}
bool compare(const UInt8 * pos) const
{
#if defined(__x86_64__)
if (page_safe(pos))
{
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(pos));
@ -535,6 +565,7 @@ public:
return false;
}
#endif
if (*pos == first)
{
@ -558,6 +589,7 @@ public:
while (haystack < haystack_end)
{
#if defined(__x86_64__)
if (haystack + n <= haystack_end && page_safe(haystack))
{
/// find first character
@ -605,6 +637,7 @@ public:
continue;
}
}
#endif
if (haystack == haystack_end)
return haystack_end;

View File

@ -10,6 +10,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LIMIT_EXCEEDED;
}
/** Позволяет ограничить скорость чего либо (в штуках в секунду) с помощью sleep.
* Особенности работы:
* - считается только средняя скорость, от момента первого вызова функции add;

View File

@ -29,6 +29,8 @@ struct UInt128Hash
size_t operator()(UInt128 x) const { return hash64(hash64(x.first) ^ x.second); }
};
#if defined(__x86_64__)
struct UInt128HashCRC32
{
size_t operator()(UInt128 x) const
@ -40,6 +42,13 @@ struct UInt128HashCRC32
}
};
#else
/// На других платформах используем не обязательно CRC32. NOTE Это может сбить с толку.
struct UInt128HashCRC32 : public UInt128Hash {};
#endif
struct UInt128TrivialHash
{
size_t operator()(UInt128 x) const { return x.first; }
@ -80,6 +89,8 @@ struct UInt256
UInt256 & operator= (const UInt64 rhs) { a = rhs; b = 0; c = 0; d = 0; return *this; }
};
#if defined(__x86_64__)
struct UInt256HashCRC32
{
size_t operator()(UInt256 x) const
@ -93,6 +104,21 @@ struct UInt256HashCRC32
}
};
#else
/// На других платформах используем не обязательно CRC32. NOTE Это может сбить с толку.
struct UInt256HashCRC32
{
DefaultHash<UInt64> hash64;
size_t operator()(UInt256 x) const
{
/// TODO Это не оптимально.
return hash64(hash64(hash64(hash64(x.a) ^ x.b) ^ x.c) ^ x.d);
}
};
#endif
inline void readBinary(UInt256 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void writeBinary(const UInt256 & x, WriteBuffer & buf) { writePODBinary(x, buf); }

View File

@ -1,7 +1,12 @@
#pragma once
#include <DB/Core/Types.h>
#include <x86intrin.h>
#if defined(__x86_64__)
#include <x86intrin.h>
#else
#include <DB/Common/ARMHelpers.h>
#endif
namespace DB

View File

@ -4,7 +4,6 @@
#include <Poco/UTF8Encoding.h>
#include <Poco/Unicode.h>
#include <ext/range.hpp>
#include <x86intrin.h>
#include <stdint.h>
#include <string.h>

View File

@ -0,0 +1,4 @@
#pragma once
/// Получить количество ядер CPU без учёта hyper-threading.
unsigned getNumberOfPhysicalCPUCores();

View File

@ -9,7 +9,6 @@
#include <DB/Core/ColumnWithTypeAndName.h>
#include <DB/Core/NamesAndTypes.h>
#include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include "ColumnsWithTypeAndName.h"

View File

@ -2,7 +2,6 @@
#include <DB/Core/Types.h>
#include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/IO/ReadBuffer.h>
#include <DB/IO/WriteBuffer.h>
#include <DB/IO/VarInt.h>
@ -13,6 +12,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_BLOCK_INFO_FIELD;
}
/** Дополнительная информация о блоке.
*/
struct BlockInfo

View File

@ -82,3 +82,9 @@
#define ALWAYS_INLINE __attribute__((__always_inline__))
#define NO_INLINE __attribute__((__noinline__))
#define PLATFORM_NOT_SUPPORTED "The only supported platforms are x86_64 and AArch64 (work in progress)"
#if !defined(__x86_64__) && !defined(__aarch64__)
#error PLATFORM_NOT_SUPPORTED
#endif

View File

@ -1,318 +0,0 @@
#pragma once
namespace DB
{
namespace ErrorCodes
{
enum ErrorCodes
{
UNSUPPORTED_METHOD = 1,
UNSUPPORTED_PARAMETER = 2,
UNEXPECTED_END_OF_FILE = 3,
EXPECTED_END_OF_FILE = 4,
CANNOT_PARSE_TEXT = 6,
INCORRECT_NUMBER_OF_COLUMNS = 7,
THERE_IS_NO_COLUMN = 8,
SIZES_OF_COLUMNS_DOESNT_MATCH = 9,
NOT_FOUND_COLUMN_IN_BLOCK = 10,
POSITION_OUT_OF_BOUND = 11,
PARAMETER_OUT_OF_BOUND = 12,
SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH = 13,
DUPLICATE_COLUMN = 15,
NO_SUCH_COLUMN_IN_TABLE = 16,
DELIMITER_IN_STRING_LITERAL_DOESNT_MATCH = 17,
CANNOT_INSERT_ELEMENT_INTO_CONSTANT_COLUMN = 18,
SIZE_OF_ARRAY_DOESNT_MATCH_SIZE_OF_FIXEDARRAY_COLUMN = 19,
NUMBER_OF_COLUMNS_DOESNT_MATCH = 20,
CANNOT_READ_ALL_DATA_FROM_TAB_SEPARATED_INPUT = 21,
CANNOT_PARSE_ALL_VALUE_FROM_TAB_SEPARATED_INPUT = 22,
CANNOT_READ_FROM_ISTREAM = 23,
CANNOT_WRITE_TO_OSTREAM = 24,
CANNOT_PARSE_ESCAPE_SEQUENCE = 25,
CANNOT_PARSE_QUOTED_STRING = 26,
CANNOT_PARSE_INPUT_ASSERTION_FAILED = 27,
CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER = 28,
CANNOT_PRINT_INTEGER = 29,
CANNOT_READ_SIZE_OF_COMPRESSED_CHUNK = 30,
CANNOT_READ_COMPRESSED_CHUNK = 31,
ATTEMPT_TO_READ_AFTER_EOF = 32,
CANNOT_READ_ALL_DATA = 33,
TOO_MUCH_ARGUMENTS_FOR_FUNCTION = 34,
TOO_LESS_ARGUMENTS_FOR_FUNCTION = 35,
BAD_ARGUMENTS = 36,
UNKNOWN_ELEMENT_IN_AST = 37,
CANNOT_PARSE_DATE = 38,
TOO_LARGE_SIZE_COMPRESSED = 39,
CHECKSUM_DOESNT_MATCH = 40,
CANNOT_PARSE_DATETIME = 41,
NUMBER_OF_ARGUMENTS_DOESNT_MATCH = 42,
ILLEGAL_TYPE_OF_ARGUMENT = 43,
ILLEGAL_COLUMN = 44,
ILLEGAL_NUMBER_OF_RESULT_COLUMNS = 45,
UNKNOWN_FUNCTION = 46,
UNKNOWN_IDENTIFIER = 47,
NOT_IMPLEMENTED = 48,
LOGICAL_ERROR = 49,
UNKNOWN_TYPE = 50,
EMPTY_LIST_OF_COLUMNS_QUERIED = 51,
COLUMN_QUERIED_MORE_THAN_ONCE = 52,
TYPE_MISMATCH = 53,
STORAGE_DOESNT_ALLOW_PARAMETERS = 54,
STORAGE_REQUIRES_PARAMETER = 55,
UNKNOWN_STORAGE = 56,
TABLE_ALREADY_EXISTS = 57,
TABLE_METADATA_ALREADY_EXISTS = 58,
ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER = 59,
UNKNOWN_TABLE = 60,
ONLY_FILTER_COLUMN_IN_BLOCK = 61,
SYNTAX_ERROR = 62,
UNKNOWN_AGGREGATE_FUNCTION = 63,
CANNOT_READ_AGGREGATE_FUNCTION_FROM_TEXT = 64,
CANNOT_WRITE_AGGREGATE_FUNCTION_AS_TEXT = 65,
NOT_A_COLUMN = 66,
ILLEGAL_KEY_OF_AGGREGATION = 67,
CANNOT_GET_SIZE_OF_FIELD = 68,
ARGUMENT_OUT_OF_BOUND = 69,
CANNOT_CONVERT_TYPE = 70,
CANNOT_WRITE_AFTER_END_OF_BUFFER = 71,
CANNOT_PARSE_NUMBER = 72,
UNKNOWN_FORMAT = 73,
CANNOT_READ_FROM_FILE_DESCRIPTOR = 74,
CANNOT_WRITE_TO_FILE_DESCRIPTOR = 75,
CANNOT_OPEN_FILE = 76,
CANNOT_CLOSE_FILE = 77,
UNKNOWN_TYPE_OF_QUERY = 78,
INCORRECT_FILE_NAME = 79,
INCORRECT_QUERY = 80,
UNKNOWN_DATABASE = 81,
DATABASE_ALREADY_EXISTS = 82,
DIRECTORY_DOESNT_EXIST = 83,
DIRECTORY_ALREADY_EXISTS = 84,
FORMAT_IS_NOT_SUITABLE_FOR_INPUT = 85,
RECEIVED_ERROR_FROM_REMOTE_IO_SERVER = 86,
CANNOT_SEEK_THROUGH_FILE = 87,
CANNOT_TRUNCATE_FILE = 88,
UNKNOWN_COMPRESSION_METHOD = 89,
EMPTY_LIST_OF_COLUMNS_PASSED = 90,
SIZES_OF_MARKS_FILES_ARE_INCONSISTENT = 91,
EMPTY_DATA_PASSED = 92,
UNKNOWN_AGGREGATED_DATA_VARIANT = 93,
CANNOT_MERGE_DIFFERENT_AGGREGATED_DATA_VARIANTS = 94,
CANNOT_READ_FROM_SOCKET = 95,
CANNOT_WRITE_TO_SOCKET = 96,
CANNOT_READ_ALL_DATA_FROM_CHUNKED_INPUT = 97,
CANNOT_WRITE_TO_EMPTY_BLOCK_OUTPUT_STREAM = 98,
UNKNOWN_PACKET_FROM_CLIENT = 99,
UNKNOWN_PACKET_FROM_SERVER = 100,
UNEXPECTED_PACKET_FROM_CLIENT = 101,
UNEXPECTED_PACKET_FROM_SERVER = 102,
RECEIVED_DATA_FOR_WRONG_QUERY_ID = 103,
TOO_SMALL_BUFFER_SIZE = 104,
CANNOT_READ_HISTORY = 105,
CANNOT_APPEND_HISTORY = 106,
FILE_DOESNT_EXIST = 107,
NO_DATA_TO_INSERT = 108,
CANNOT_BLOCK_SIGNAL = 109,
CANNOT_UNBLOCK_SIGNAL = 110,
CANNOT_MANIPULATE_SIGSET = 111,
CANNOT_WAIT_FOR_SIGNAL = 112,
THERE_IS_NO_SESSION = 113,
CANNOT_CLOCK_GETTIME = 114,
UNKNOWN_SETTING = 115,
THERE_IS_NO_DEFAULT_VALUE = 116,
INCORRECT_DATA = 117,
TABLE_METADATA_DOESNT_EXIST = 118,
ENGINE_REQUIRED = 119,
CANNOT_INSERT_VALUE_OF_DIFFERENT_SIZE_INTO_TUPLE = 120,
UNKNOWN_SET_DATA_VARIANT = 121,
INCOMPATIBLE_COLUMNS = 122,
UNKNOWN_TYPE_OF_AST_NODE = 123,
INCORRECT_ELEMENT_OF_SET = 124,
INCORRECT_RESULT_OF_SCALAR_SUBQUERY = 125,
CANNOT_GET_RETURN_TYPE = 126,
ILLEGAL_INDEX = 127,
TOO_LARGE_ARRAY_SIZE = 128,
FUNCTION_IS_SPECIAL = 129,
CANNOT_READ_ARRAY_FROM_TEXT = 130,
TOO_LARGE_STRING_SIZE = 131,
CANNOT_CREATE_TABLE_FROM_METADATA = 132,
AGGREGATE_FUNCTION_DOESNT_ALLOW_PARAMETERS = 133,
PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS = 134,
ZERO_ARRAY_OR_TUPLE_INDEX = 135,
UNKNOWN_ELEMENT_IN_CONFIG = 137,
EXCESSIVE_ELEMENT_IN_CONFIG = 138,
NO_ELEMENTS_IN_CONFIG = 139,
ALL_REQUESTED_COLUMNS_ARE_MISSING = 140,
SAMPLING_NOT_SUPPORTED = 141,
NOT_FOUND_NODE = 142,
FOUND_MORE_THAN_ONE_NODE = 143,
FIRST_DATE_IS_BIGGER_THAN_LAST_DATE = 144,
UNKNOWN_OVERFLOW_MODE = 145,
QUERY_SECTION_DOESNT_MAKE_SENSE = 146,
NOT_FOUND_FUNCTION_ELEMENT_FOR_AGGREGATE = 147,
NOT_FOUND_RELATION_ELEMENT_FOR_CONDITION = 148,
NOT_FOUND_RHS_ELEMENT_FOR_CONDITION = 149,
NO_ATTRIBUTES_LISTED = 150,
INDEX_OF_COLUMN_IN_SORT_CLAUSE_IS_OUT_OF_RANGE = 151,
UNKNOWN_DIRECTION_OF_SORTING = 152,
ILLEGAL_DIVISION = 153,
AGGREGATE_FUNCTION_NOT_APPLICABLE = 154,
UNKNOWN_RELATION = 155,
DICTIONARIES_WAS_NOT_LOADED = 156,
ILLEGAL_OVERFLOW_MODE = 157,
TOO_MUCH_ROWS = 158,
TIMEOUT_EXCEEDED = 159,
TOO_SLOW = 160,
TOO_MUCH_COLUMNS = 161,
TOO_DEEP_SUBQUERIES = 162,
TOO_DEEP_PIPELINE = 163,
READONLY = 164,
TOO_MUCH_TEMPORARY_COLUMNS = 165,
TOO_MUCH_TEMPORARY_NON_CONST_COLUMNS = 166,
TOO_DEEP_AST = 167,
TOO_BIG_AST = 168,
BAD_TYPE_OF_FIELD = 169,
BAD_GET = 170,
BLOCKS_HAS_DIFFERENT_STRUCTURE = 171,
CANNOT_CREATE_DIRECTORY = 172,
CANNOT_ALLOCATE_MEMORY = 173,
CYCLIC_ALIASES = 174,
CHUNK_NOT_FOUND = 176,
DUPLICATE_CHUNK_NAME = 177,
MULTIPLE_ALIASES_FOR_EXPRESSION = 178,
MULTIPLE_EXPRESSIONS_FOR_ALIAS = 179,
THERE_IS_NO_PROFILE = 180,
ILLEGAL_FINAL = 181,
ILLEGAL_PREWHERE = 182,
UNEXPECTED_EXPRESSION = 183,
ILLEGAL_AGGREGATION = 184,
UNSUPPORTED_MYISAM_BLOCK_TYPE = 185,
UNSUPPORTED_COLLATION_LOCALE = 186,
COLLATION_COMPARISON_FAILED = 187,
UNKNOWN_ACTION = 188,
TABLE_MUST_NOT_BE_CREATED_MANUALLY = 189,
SIZES_OF_ARRAYS_DOESNT_MATCH = 190,
SET_SIZE_LIMIT_EXCEEDED = 191,
UNKNOWN_USER = 192,
WRONG_PASSWORD = 193,
REQUIRED_PASSWORD = 194,
IP_ADDRESS_NOT_ALLOWED = 195,
UNKNOWN_ADDRESS_PATTERN_TYPE = 196,
SERVER_REVISION_IS_TOO_OLD = 197,
DNS_ERROR = 198,
UNKNOWN_QUOTA = 199,
QUOTA_DOESNT_ALLOW_KEYS = 200,
QUOTA_EXPIRED = 201,
TOO_MUCH_SIMULTANEOUS_QUERIES = 202,
NO_FREE_CONNECTION = 203,
CANNOT_FSYNC = 204,
NESTED_TYPE_TOO_DEEP = 205,
ALIAS_REQUIRED = 206,
AMBIGUOUS_IDENTIFIER = 207,
EMPTY_NESTED_TABLE = 208,
SOCKET_TIMEOUT = 209,
NETWORK_ERROR = 210,
EMPTY_QUERY = 211,
UNKNOWN_LOAD_BALANCING = 212,
UNKNOWN_TOTALS_MODE = 213,
CANNOT_STATVFS = 214,
NOT_AN_AGGREGATE = 215,
QUERY_WITH_SAME_ID_IS_ALREADY_RUNNING = 216,
CLIENT_HAS_CONNECTED_TO_WRONG_PORT = 217,
TABLE_IS_DROPPED = 218,
DATABASE_NOT_EMPTY = 219,
DUPLICATE_INTERSERVER_IO_ENDPOINT = 220,
NO_SUCH_INTERSERVER_IO_ENDPOINT = 221,
ADDING_REPLICA_TO_NON_EMPTY_TABLE = 222,
UNEXPECTED_AST_STRUCTURE = 223,
REPLICA_IS_ALREADY_ACTIVE = 224,
NO_ZOOKEEPER = 225,
NO_FILE_IN_DATA_PART = 226,
UNEXPECTED_FILE_IN_DATA_PART = 227,
BAD_SIZE_OF_FILE_IN_DATA_PART = 228,
QUERY_IS_TOO_LARGE = 229,
NOT_FOUND_EXPECTED_DATA_PART = 230,
TOO_MANY_UNEXPECTED_DATA_PARTS = 231,
NO_SUCH_DATA_PART = 232,
BAD_DATA_PART_NAME = 233,
NO_REPLICA_HAS_PART = 234,
DUPLICATE_DATA_PART = 235,
ABORTED = 236,
NO_REPLICA_NAME_GIVEN = 237,
FORMAT_VERSION_TOO_OLD = 238,
CANNOT_MUNMAP = 239,
CANNOT_MREMAP = 240,
MEMORY_LIMIT_EXCEEDED = 241,
TABLE_IS_READ_ONLY = 242,
NOT_ENOUGH_SPACE = 243,
UNEXPECTED_ZOOKEEPER_ERROR = 244,
INVALID_NESTED_NAME = 245,
CORRUPTED_DATA = 246,
INCORRECT_MARK = 247,
INVALID_PARTITION_NAME = 248,
NOT_ENOUGH_BLOCK_NUMBERS = 250,
NO_SUCH_REPLICA = 251,
TOO_MUCH_PARTS = 252,
REPLICA_IS_ALREADY_EXIST = 253,
NO_ACTIVE_REPLICAS = 254,
TOO_MUCH_RETRIES_TO_FETCH_PARTS = 255,
PARTITION_ALREADY_EXISTS = 256,
PARTITION_DOESNT_EXIST = 257,
UNION_ALL_RESULT_STRUCTURES_MISMATCH = 258,
UNION_ALL_COLUMN_ALIAS_MISMATCH = 259,
CLIENT_OUTPUT_FORMAT_SPECIFIED = 260,
UNKNOWN_BLOCK_INFO_FIELD = 261,
BAD_COLLATION = 262,
CANNOT_COMPILE_CODE = 263,
INCOMPATIBLE_TYPE_OF_JOIN = 264,
NO_AVAILABLE_REPLICA = 265,
MISMATCH_REPLICAS_DATA_SOURCES = 266,
STORAGE_DOESNT_SUPPORT_PARALLEL_REPLICAS = 267,
CPUID_ERROR = 268,
INFINITE_LOOP = 269,
CANNOT_COMPRESS = 270,
CANNOT_DECOMPRESS = 271,
AIO_SUBMIT_ERROR = 272,
AIO_COMPLETION_ERROR = 273,
AIO_READ_ERROR = 274,
AIO_WRITE_ERROR = 275,
INDEX_NOT_USED = 277,
LEADERSHIP_LOST = 278,
ALL_CONNECTION_TRIES_FAILED = 279,
NO_AVAILABLE_DATA = 280,
DICTIONARY_IS_EMPTY = 281,
INCORRECT_INDEX = 282,
UNKNOWN_DISTRIBUTED_PRODUCT_MODE = 283,
UNKNOWN_GLOBAL_SUBQUERIES_METHOD = 284,
TOO_LESS_LIVE_REPLICAS = 285,
UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE = 286,
UNKNOWN_FORMAT_VERSION = 287,
DISTRIBUTED_IN_JOIN_SUBQUERY_DENIED = 288,
REPLICA_IS_NOT_IN_QUORUM = 289,
LIMIT_EXCEEDED = 290,
DATABASE_ACCESS_DENIED = 291,
LEADERSHIP_CHANGED = 292,
MONGODB_INIT_FAILED = 293,
INVALID_BLOCK_EXTRA_INFO = 294,
RECEIVED_EMPTY_DATA = 295,
NO_REMOTE_SHARD_FOUND = 296,
SHARD_HAS_NO_CONNECTIONS = 297,
CANNOT_PIPE = 298,
CANNOT_FORK = 299,
CANNOT_DLSYM = 300,
CANNOT_CREATE_CHILD_PROCESS = 301,
CHILD_WAS_NOT_EXITED_NORMALLY = 302,
CANNOT_SELECT = 303,
CANNOT_WAITPID = 304,
TABLE_WAS_NOT_DROPPED = 305,
KEEPER_EXCEPTION = 999,
POCO_EXCEPTION = 1000,
STD_EXCEPTION = 1001,
UNKNOWN_EXCEPTION = 1002,
};
}
}

View File

@ -7,7 +7,6 @@
#include <boost/static_assert.hpp>
#include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/Core/Types.h>
#include <common/strong_typedef.h>
@ -15,6 +14,13 @@
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_TYPE_OF_FIELD;
extern const int BAD_GET;
extern const int NOT_IMPLEMENTED;
}
class Field;
using Array = std::vector<Field>; /// Значение типа "массив"
using TupleBackend = std::vector<Field>;
@ -26,7 +32,7 @@ using Poco::SharedPtr;
/** 32 хватает с запасом (достаточно 28), но выбрано круглое число,
* чтобы арифметика при использовании массивов из Field была проще (не содержала умножения).
*/
#define DBMS_TOTAL_FIELD_SIZE 32
#define DBMS_MIN_FIELD_SIZE 32
/** Discriminated union из нескольких типов.
@ -37,7 +43,7 @@ using Poco::SharedPtr;
* Используется для представления единичного значения одного из нескольких типов в оперативке.
* Внимание! Предпочтительно вместо единичных значений хранить кусочки столбцов. См. Column.h
*/
class __attribute__((aligned(DBMS_TOTAL_FIELD_SIZE))) Field
class Field
{
public:
struct Types
@ -262,16 +268,9 @@ public:
}
private:
/// Хватает с запасом
static const size_t storage_size = DBMS_TOTAL_FIELD_SIZE - sizeof(Types::Which);
BOOST_STATIC_ASSERT(storage_size >= sizeof(Null));
BOOST_STATIC_ASSERT(storage_size >= sizeof(UInt64));
BOOST_STATIC_ASSERT(storage_size >= sizeof(Int64));
BOOST_STATIC_ASSERT(storage_size >= sizeof(Float64));
BOOST_STATIC_ASSERT(storage_size >= sizeof(String));
BOOST_STATIC_ASSERT(storage_size >= sizeof(Array));
BOOST_STATIC_ASSERT(storage_size >= sizeof(Tuple));
static const size_t storage_size = std::max({
DBMS_MIN_FIELD_SIZE - sizeof(Types::Which),
sizeof(Null), sizeof(UInt64), sizeof(Int64), sizeof(Float64), sizeof(String), sizeof(Array), sizeof(Tuple)});
char storage[storage_size] __attribute__((aligned(8)));
Types::Which which;
@ -346,7 +345,7 @@ private:
}
};
#undef DBMS_TOTAL_FIELD_SIZE
#undef DBMS_MIN_FIELD_SIZE
template <> struct Field::TypeToEnum<Null> { static const Types::Which value = Types::Null; };

View File

@ -13,6 +13,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_CONVERT_TYPE;
}
/** StaticVisitor (его наследники) - класс с перегруженными для разных типов операторами ().
* Вызвать visitor для field можно с помощью функции apply_visitor.

View File

@ -9,7 +9,9 @@
#include <functional>
#include <ostream>
#include <emmintrin.h>
#if defined(__x86_64__)
#include <emmintrin.h>
#endif
/// Штука, чтобы не создавать строки для поиска подстроки в хэш таблице.
@ -31,6 +33,8 @@ struct StringRef
typedef std::vector<StringRef> StringRefs;
#if defined(__x86_64__)
/** Сравнение строк на равенство.
* Подход является спорным и выигрывает не во всех случаях.
* Подробнее смотрите hash_map_string_2.cpp
@ -111,6 +115,8 @@ inline bool memequalSSE2Wide(const char * p1, const char * p2, size_t size)
return true;
}
#endif
inline bool operator== (StringRef lhs, StringRef rhs)
{
@ -120,7 +126,11 @@ inline bool operator== (StringRef lhs, StringRef rhs)
if (lhs.size == 0)
return true;
#if defined(__x86_64__)
return memequalSSE2Wide(lhs.data, rhs.data, lhs.size);
#else
return 0 == memcmp(lhs.data, rhs.data, lhs.size);
#endif
}
inline bool operator!= (StringRef lhs, StringRef rhs)
@ -149,6 +159,8 @@ inline bool operator> (StringRef lhs, StringRef rhs)
* Подробнее см. hash_map_string_3.cpp
*/
#if defined(__x86_64__)
#ifdef __SSE4_1__
#include <smmintrin.h>
#else
@ -247,9 +259,6 @@ struct CRC32Hash
}
};
#if 1
struct StringRefHash : CRC32Hash {};
#else

View File

@ -1,16 +1,12 @@
#pragma once
#include <Poco/SharedPtr.h>
#include <DB/Interpreters/ExpressionActions.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>
namespace DB
{
using Poco::SharedPtr;
class ExpressionActions;
/** Выполняет над блоком вычисление некоторого выражения.
* Выражение состоит из идентификаторов столбцов из блока, констант, обычных функций.
@ -19,44 +15,18 @@ using Poco::SharedPtr;
*/
class ExpressionBlockInputStream : public IProfilingBlockInputStream
{
private:
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
public:
ExpressionBlockInputStream(BlockInputStreamPtr input_, ExpressionActionsPtr expression_)
: expression(expression_)
{
children.push_back(input_);
}
ExpressionBlockInputStream(BlockInputStreamPtr input_, ExpressionActionsPtr expression_);
String getName() const override { return "Expression"; }
String getID() const override
{
std::stringstream res;
res << "Expression(" << children.back()->getID() << ", " << expression->getID() << ")";
return res.str();
}
const Block & getTotals() override
{
if (IProfilingBlockInputStream * child = dynamic_cast<IProfilingBlockInputStream *>(&*children.back()))
{
totals = child->getTotals();
expression->executeOnTotals(totals);
}
return totals;
}
String getName() const override;
String getID() const override;
const Block & getTotals() override;
protected:
Block readImpl() override
{
Block res = children.back()->read();
if (!res)
return res;
expression->execute(res);
return res;
}
Block readImpl() override;
private:
ExpressionActionsPtr expression;

View File

@ -13,6 +13,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int CORRUPTED_DATA;
}
/** Соединяет несколько сортированных потоков в один.
*/
class MergingSortedBlockInputStream : public IProfilingBlockInputStream

View File

@ -12,10 +12,10 @@ namespace DB
* Выводится заголовок с нужной информацией.
* Затем данные выводятся в порядке строк. Каждое значение выводится так: длина в формате VarUInt, затем данные в текстовом виде.
*/
class ODBCBlockOutputStream : public IBlockOutputStream
class ODBCDriverBlockOutputStream : public IBlockOutputStream
{
public:
ODBCBlockOutputStream(WriteBuffer & out_);
ODBCDriverBlockOutputStream(WriteBuffer & out_);
void write(const Block & block) override;

View File

@ -6,6 +6,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
/// Throws exception on encountering prohibited column in block
class ProhibitColumnsBlockOutputStream : public IBlockOutputStream

View File

@ -10,6 +10,13 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNEXPECTED_PACKET_FROM_SERVER;
extern const int LOGICAL_ERROR;
}
/** Позволяет выполнить запрос INSERT на удалённом сервере и отправить данные.
*/
class RemoteBlockOutputStream : public IBlockOutputStream

View File

@ -14,6 +14,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
/** Соединяет несколько сортированных потоков в один.
* При этом, для каждой группы идущих подряд одинаковых значений первичного ключа (столбцов, по которым сортируются данные),
* схлопывает их в одну строку, суммируя все числовые столбцы кроме первичного ключа.
@ -33,22 +39,7 @@ public:
String getName() const override { return "SummingSorted"; }
String getID() const override
{
std::stringstream res;
res << "SummingSorted(inputs";
for (size_t i = 0; i < children.size(); ++i)
res << ", " << children[i]->getID();
res << ", description";
for (size_t i = 0; i < description.size(); ++i)
res << ", " << description[i].getID();
res << ")";
return res.str();
}
String getID() const override;
protected:
/// Может возвращаться на 1 больше записей, чем max_block_size.
@ -105,26 +96,6 @@ private:
/// Вставить в результат просуммированную строку для текущей группы.
void insertCurrentRow(ColumnPlainPtrs & merged_columns);
/** Реализует операцию +=.
* Возвращает false, если результат получился нулевым.
*/
class FieldVisitorSum : public StaticVisitor<bool>
{
private:
const Field & rhs;
public:
FieldVisitorSum(const Field & rhs_) : rhs(rhs_) {}
bool operator() (UInt64 & x) const { x += get<UInt64>(rhs); return x != 0; }
bool operator() (Int64 & x) const { x += get<Int64>(rhs); return x != 0; }
bool operator() (Float64 & x) const { x += get<Float64>(rhs); return x != 0; }
bool operator() (Null & x) const { throw Exception("Cannot sum Nulls", ErrorCodes::LOGICAL_ERROR); }
bool operator() (String & x) const { throw Exception("Cannot sum Strings", ErrorCodes::LOGICAL_ERROR); }
bool operator() (Array & x) const { throw Exception("Cannot sum Arrays", ErrorCodes::LOGICAL_ERROR); }
};
/** Для вложенных Map выполняется слияние по ключу с выбрасыванием строк вложенных массивов, в которых
* все элементы - нулевые.
*/

View File

@ -1,14 +1,12 @@
#pragma once
#include <DB/Interpreters/Aggregator.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>
#include <DB/Interpreters/ExpressionActions.h>
namespace DB
{
using Poco::SharedPtr;
class ExpressionActions;
/** Принимает блоки после группировки, с нефиализированными агрегатными функциями.
@ -17,26 +15,18 @@ using Poco::SharedPtr;
*/
class TotalsHavingBlockInputStream : public IProfilingBlockInputStream
{
private:
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
public:
TotalsHavingBlockInputStream(BlockInputStreamPtr input_, const Names & keys_names_,
const AggregateDescriptions & aggregates_, bool overflow_row_, ExpressionActionsPtr expression_,
const std::string & filter_column_, TotalsMode totals_mode_, double auto_include_threshold_)
: overflow_row(overflow_row_),
expression(expression_), filter_column_name(filter_column_), totals_mode(totals_mode_),
auto_include_threshold(auto_include_threshold_)
{
children.push_back(input_);
}
TotalsHavingBlockInputStream(
BlockInputStreamPtr input_,
bool overflow_row_, ExpressionActionsPtr expression_,
const std::string & filter_column_, TotalsMode totals_mode_, double auto_include_threshold_);
String getName() const override { return "TotalsHaving"; }
String getID() const override
{
std::stringstream res;
res << "TotalsHavingBlockInputStream(" << children.back()->getID()
<< "," << filter_column_name << ")";
return res.str();
}
String getID() const override;
const Block & getTotals() override;

View File

@ -12,6 +12,12 @@ namespace DB
using Poco::SharedPtr;
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace
{

View File

@ -10,6 +10,13 @@ namespace DB
using Poco::SharedPtr;
namespace ErrorCodes
{
extern const int THERE_IS_NO_DEFAULT_VALUE;
extern const int NOT_IMPLEMENTED;
}
/** Тип - состояние агрегатной функции.
* Параметры типа - это агрегатная функция, типы её аргументов и её параметры (для параметрических агрегатных функций).
*/

View File

@ -9,6 +9,13 @@
namespace DB
{
namespace ErrorCodes
{
extern const int SYNTAX_ERROR;
extern const int LOGICAL_ERROR;
extern const int EMPTY_DATA_PASSED;
}
template <typename FieldType> struct EnumName;
template <> struct EnumName<Int8> { static constexpr auto value = "Enum8"; };

View File

@ -12,6 +12,11 @@ namespace DB
using Poco::SharedPtr;
namespace ErrorCodes
{
extern const int ARGUMENT_OUT_OF_BOUND;
}
class DataTypeFixedString final : public IDataType
{

View File

@ -7,12 +7,17 @@
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
/** Для заданного значения Field возвращает минимальный тип данных, позволяющий хранить значение этого типа.
* В случае, если Field - массив, конвертирует все элементы к общему типу.
*/

View File

@ -23,6 +23,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
extern const int BAD_ARGUMENTS;
extern const int UNSUPPORTED_METHOD;
}
class CacheDictionary final : public IDictionary
{
public:

View File

@ -14,6 +14,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNSUPPORTED_METHOD;
}
const auto max_connections = 16;
/** Allows loading dictionaries from local or remote ClickHouse instance

View File

@ -26,6 +26,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
extern const int BAD_ARGUMENTS;
extern const int UNSUPPORTED_METHOD;
}
class ComplexKeyCacheDictionary final : public IDictionaryBase
{
public:

View File

@ -16,6 +16,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int BAD_ARGUMENTS;
extern const int DICTIONARY_IS_EMPTY;
}
class ComplexKeyHashedDictionary final : public IDictionaryBase
{
public:

View File

@ -15,6 +15,13 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
extern const int EXCESSIVE_ELEMENT_IN_CONFIG;
extern const int LOGICAL_ERROR;
}
namespace
{

View File

@ -1,6 +1,5 @@
#pragma once
#include <DB/Core/ErrorCodes.h>
#include <DB/DataTypes/DataTypeFactory.h>
#include <DB/IO/ReadBufferFromString.h>
#include <DB/IO/WriteBuffer.h>
@ -16,6 +15,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_TYPE;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int TYPE_MISMATCH;
}
enum class AttributeUnderlyingType
{
UInt8,

View File

@ -0,0 +1,80 @@
#pragma once
#include <DB/Dictionaries/Embedded/RegionsHierarchy.h>
#include <Poco/DirectoryIterator.h>
/** Содержит несколько иерархий регионов, загружаемых из нескольких разных файлов.
* Используется, чтобы поддержать несколько разных точек зрения о принадлежности регионов странам.
* В первую очередь, для Крыма (Российская и Украинская точки зрения).
*/
class RegionsHierarchies
{
private:
typedef std::unordered_map<std::string, RegionsHierarchy> Container;
Container data;
Logger * log = &Logger::get("RegionsHierarchies");
public:
static constexpr auto required_key = "path_to_regions_hierarchy_file";
/** path должен указывать на файл с иерархией регионов "по-умолчанию". Она будет доступна по пустому ключу.
* Кроме того, рядом ищутся файлы, к имени которых (до расширения, если есть) добавлен произвольный _suffix.
* Такие файлы загружаются, и иерархия регионов кладётся по ключу suffix.
*
* Например, если указано /opt/geo/regions_hierarchy.txt,
* то будет также загружен файл /opt/geo/regions_hierarchy_ua.txt, если такой есть - он будет доступен по ключу ua.
*/
RegionsHierarchies(const std::string & default_path = Poco::Util::Application::instance().config().getString(required_key))
{
LOG_DEBUG(log, "Adding default regions hierarchy from " << default_path);
data.emplace(std::piecewise_construct,
std::forward_as_tuple(""),
std::forward_as_tuple(default_path));
std::string basename = Poco::Path(default_path).getBaseName();
Poco::Path dir_path = Poco::Path(default_path).absolute().parent();
Poco::DirectoryIterator dir_end;
for (Poco::DirectoryIterator dir_it(dir_path); dir_it != dir_end; ++dir_it)
{
std::string other_basename = dir_it.path().getBaseName();
if (0 == other_basename.compare(0, basename.size(), basename) && other_basename.size() > basename.size() + 1)
{
if (other_basename[basename.size()] != '_')
continue;
std::string suffix = other_basename.substr(basename.size() + 1);
LOG_DEBUG(log, "Adding regions hierarchy from " << dir_it->path() << ", key: " << suffix);
data.emplace(std::piecewise_construct,
std::forward_as_tuple(suffix),
std::forward_as_tuple(dir_it->path()));
}
}
}
/** Перезагружает, при необходимости, все иерархии регионов.
*/
void reload()
{
for (auto & elem : data)
elem.second.reload();
}
const RegionsHierarchy & get(const std::string & key) const
{
auto it = data.find(key);
if (data.end() == it)
throw Poco::Exception("There is no regions hierarchy for key " + key);
return it->second;
}
};

View File

@ -0,0 +1,301 @@
#pragma once
#include <Poco/Util/Application.h>
#include <Poco/Exception.h>
#include <Poco/File.h>
#include <common/logger_useful.h>
#include <common/singleton.h>
#include <DB/IO/ReadBufferFromFile.h>
#include <DB/IO/ReadHelpers.h>
#include <boost/noncopyable.hpp>
#define REGION_TYPE_CITY 6
#define REGION_TYPE_AREA 5
#define REGION_TYPE_DISTRICT 4
#define REGION_TYPE_COUNTRY 3
#define REGION_TYPE_CONTINENT 1
/** Класс, позволяющий узнавать, принадлежит ли регион с одним RegionID региону с другим RegionID.
* Информацию об иерархии регионов загружает из текстового файла.
* Умеет, по запросу, обновлять данные.
*/
class RegionsHierarchy : private boost::noncopyable
{
private:
std::string path;
time_t file_modification_time;
Logger * log;
typedef Int32 RegionID;
typedef Int8 RegionType;
typedef Int8 RegionDepth;
typedef UInt32 RegionPopulation;
/// отношение parent; 0, если родителей нет - обычная lookup таблица.
typedef std::vector<RegionID> RegionParents;
/// тип региона
typedef std::vector<RegionType> RegionTypes;
/// глубина в дереве, начиная от страны (страна: 1, корень: 0)
typedef std::vector<RegionDepth> RegionDepths;
/// население региона. Если больше 2^32 - 1, то приравнивается к этому максимуму.
typedef std::vector<RegionPopulation> RegionPopulations;
/// регион -> родительский регион
RegionParents parents;
/// регион -> город, включающий его или 0, если такого нет
RegionParents city;
/// регион -> страна, включающая его или 0, если такого нет
RegionParents country;
/// регион -> область, включающая его или 0, если такой нет
RegionParents area;
/// регион -> округ, включающий его или 0, если такого нет
RegionParents district;
/// регион -> континет, включающий его или 0, если такого нет
RegionParents continent;
/// регион -> население или 0, если неизвестно.
RegionPopulations populations;
/// регион - глубина в дереве
RegionDepths depths;
public:
RegionsHierarchy(const std::string & path_ = Poco::Util::Application::instance().config().getString("path_to_regions_hierarchy_file"))
: path(path_), file_modification_time(0), log(&Logger::get("RegionsHierarchy"))
{
}
/// Перезагружает, при необходимости, иерархию регионов. Непотокобезопасно.
void reload()
{
time_t new_modification_time = Poco::File(path).getLastModified().epochTime();
if (new_modification_time <= file_modification_time)
return;
file_modification_time = new_modification_time;
LOG_DEBUG(log, "Reloading regions hierarchy");
const size_t initial_size = 10000;
RegionParents new_parents(initial_size);
RegionParents new_city(initial_size);
RegionParents new_country(initial_size);
RegionParents new_area(initial_size);
RegionParents new_district(initial_size);
RegionParents new_continent(initial_size);
RegionPopulations new_populations(initial_size);
RegionDepths new_depths(initial_size);
RegionTypes types(initial_size);
DB::ReadBufferFromFile in(path);
RegionID max_region_id = 0;
while (!in.eof())
{
RegionID region_id = 0;
RegionID parent_id = 0;
RegionType type = 0;
RegionPopulation population = 0;
DB::readIntText(region_id, in);
DB::assertString("\t", in);
DB::readIntText(parent_id, in);
DB::assertString("\t", in);
DB::readIntText(type, in);
/** Далее может быть перевод строки (старый вариант)
* или таб, население региона, перевод строки (новый вариант).
*/
if (!in.eof() && *in.position() == '\t')
{
++in.position();
UInt64 population_big = 0;
DB::readIntText(population_big, in);
population = population_big > std::numeric_limits<RegionPopulation>::max()
? std::numeric_limits<RegionPopulation>::max()
: population_big;
}
DB::assertString("\n", in);
if (region_id <= 0)
continue;
if (parent_id < 0)
parent_id = 0;
if (region_id > max_region_id)
{
max_region_id = region_id;
while (region_id >= static_cast<int>(new_parents.size()))
{
new_parents.resize(new_parents.size() * 2);
new_populations.resize(new_parents.size());
types.resize(new_parents.size());
}
}
new_parents[region_id] = parent_id;
new_populations[region_id] = population;
types[region_id] = type;
}
new_parents .resize(max_region_id + 1);
new_city .resize(max_region_id + 1);
new_country .resize(max_region_id + 1);
new_area .resize(max_region_id + 1);
new_district .resize(max_region_id + 1);
new_continent .resize(max_region_id + 1);
new_populations .resize(max_region_id + 1);
new_depths .resize(max_region_id + 1);
types .resize(max_region_id + 1);
/// пропишем города и страны для регионов
for (RegionID i = 0; i <= max_region_id; ++i)
{
if (types[i] == REGION_TYPE_CITY)
new_city[i] = i;
if (types[i] == REGION_TYPE_AREA)
new_area[i] = i;
if (types[i] == REGION_TYPE_DISTRICT)
new_district[i] = i;
if (types[i] == REGION_TYPE_COUNTRY)
new_country[i] = i;
if (types[i] == REGION_TYPE_CONTINENT)
{
new_continent[i] = i;
continue;
}
RegionDepth depth = 0;
RegionID current = i;
while (true)
{
++depth;
current = new_parents[current];
if (current == 0)
break;
if (current > max_region_id)
throw Poco::Exception("Logical error in regions hierarchy: region " + DB::toString(current) + " (specified as parent) doesn't exist");
if (types[current] == REGION_TYPE_CITY)
new_city[i] = current;
if (types[current] == REGION_TYPE_AREA)
new_area[i] = current;
if (types[current] == REGION_TYPE_DISTRICT)
new_district[i] = current;
if (types[current] == REGION_TYPE_COUNTRY)
new_country[i] = current;
if (types[current] == REGION_TYPE_CONTINENT)
{
new_continent[i] = current;
break;
}
}
new_depths[i] = depth;
}
parents.swap(new_parents);
country.swap(new_country);
city.swap(new_city);
area.swap(new_area);
district.swap(new_district);
continent.swap(new_continent);
populations.swap(new_populations);
depths.swap(new_depths);
}
bool in(RegionID lhs, RegionID rhs) const
{
if (static_cast<size_t>(lhs) >= parents.size())
return false;
while (lhs != 0 && lhs != rhs)
lhs = parents[lhs];
return lhs != 0;
}
RegionID toCity(RegionID region) const
{
if (static_cast<size_t>(region) >= city.size())
return 0;
return city[region];
}
RegionID toCountry(RegionID region) const
{
if (static_cast<size_t>(region) >= country.size())
return 0;
return country[region];
}
RegionID toArea(RegionID region) const
{
if (static_cast<size_t>(region) >= area.size())
return 0;
return area[region];
}
RegionID toDistrict(RegionID region) const
{
if (static_cast<size_t>(region) >= district.size())
return 0;
return district[region];
}
RegionID toContinent(RegionID region) const
{
if (static_cast<size_t>(region) >= continent.size())
return 0;
return continent[region];
}
RegionID toParent(RegionID region) const
{
if (static_cast<size_t>(region) >= parents.size())
return 0;
return parents[region];
}
RegionDepth getDepth(RegionID region) const
{
if (static_cast<size_t>(region) >= depths.size())
return 0;
return depths[region];
}
RegionPopulation getPopulation(RegionID region) const
{
if (static_cast<size_t>(region) >= populations.size())
return 0;
return populations[region];
}
};
class RegionsHierarchySingleton : public Singleton<RegionsHierarchySingleton>, public RegionsHierarchy
{
friend class Singleton<RegionsHierarchySingleton>;
protected:
RegionsHierarchySingleton()
{
}
};

View File

@ -0,0 +1,208 @@
#pragma once
#include <google/dense_hash_map>
#include <Poco/File.h>
#include <Poco/NumberParser.h>
#include <Poco/Util/Application.h>
#include <Poco/Exception.h>
#include <common/Common.h>
#include <common/logger_useful.h>
#include <DB/Core/StringRef.h>
#include <DB/IO/ReadHelpers.h>
#include <DB/IO/ReadBufferFromFile.h>
/** @brief Класс, позволяющий узнавать по id региона его текстовое название на одном из поддерживаемых языков: ru, en, ua, by, kz, tr.
*
* Информацию об именах регионов загружает из текстовых файлов с названиями следующего формата:
* regions_names_xx.txt,
* где xx - одно из двух буквенных обозначений следующих поддерживаемых языков:
* ru, en, ua, by, kz, tr.
*
* Умеет, по запросу, обновлять данные.
*/
class RegionsNames
{
public:
enum class Language
{
RU = 0,
EN,
UA,
BY,
KZ,
TR,
};
private:
static const size_t ROOT_LANGUAGE = 0;
static const size_t SUPPORTED_LANGUAGES_COUNT = 6;
static const size_t LANGUAGE_ALIASES_COUNT = 7;
static const char ** getSupportedLanguages()
{
static const char * res[] { "ru", "en", "ua", "by", "kz", "tr" };
return res;
}
struct language_alias { const char * const name; const Language lang; };
static const language_alias * getLanguageAliases()
{
static constexpr const language_alias language_aliases[] {
{ "ru", Language::RU },
{ "en", Language::EN },
{ "ua", Language::UA },
{ "uk", Language::UA },
{ "by", Language::BY },
{ "kz", Language::KZ },
{ "tr", Language::TR }
};
return language_aliases;
}
typedef int RegionID_t;
typedef std::vector<char> Chars_t;
typedef std::vector<Chars_t> CharsForLanguageID_t;
typedef std::vector<time_t> ModificationTimes_t;
typedef std::vector<StringRef> StringRefs_t; /// Lookup table RegionID -> StringRef
typedef std::vector<StringRefs_t> StringRefsForLanguageID_t;
public:
static constexpr auto required_key = "path_to_regions_names_files";
RegionsNames(const std::string & directory_ = Poco::Util::Application::instance().config().getString(required_key))
: directory(directory_)
{
}
/** @brief Перезагружает, при необходимости, имена регионов.
*/
void reload()
{
LOG_DEBUG(log, "Reloading regions names");
RegionID_t max_region_id = 0;
for (size_t language_id = 0; language_id < SUPPORTED_LANGUAGES_COUNT; ++language_id)
{
const std::string & language = getSupportedLanguages()[language_id];
std::string path = directory + "/regions_names_" + language + ".txt";
Poco::File file(path);
time_t new_modification_time = file.getLastModified().epochTime();
if (new_modification_time <= file_modification_times[language_id])
continue;
file_modification_times[language_id] = new_modification_time;
LOG_DEBUG(log, "Reloading regions names for language: " << language);
DB::ReadBufferFromFile in(path);
const size_t initial_size = 10000;
Chars_t new_chars;
StringRefs_t new_names_refs(initial_size, StringRef("", 0));
/// Выделим непрерывный кусок памяти, которого хватит для хранения всех имён.
new_chars.reserve(Poco::File(path).getSize());
while (!in.eof())
{
RegionID_t region_id;
std::string region_name;
DB::readIntText(region_id, in);
DB::assertString("\t", in);
DB::readString(region_name, in);
DB::assertString("\n", in);
if (region_id <= 0)
continue;
size_t old_size = new_chars.size();
if (new_chars.capacity() < old_size + region_name.length() + 1)
throw Poco::Exception("Logical error. Maybe size of file " + path + " is wrong.");
new_chars.resize(old_size + region_name.length() + 1);
memcpy(&new_chars[old_size], region_name.c_str(), region_name.length() + 1);
if (region_id > max_region_id)
max_region_id = region_id;
while (region_id >= static_cast<int>(new_names_refs.size()))
new_names_refs.resize(new_names_refs.size() * 2, StringRef("", 0));
new_names_refs[region_id] = StringRef(&new_chars[old_size], region_name.length());
}
chars[language_id].swap(new_chars);
names_refs[language_id].swap(new_names_refs);
}
for (size_t language_id = 0; language_id < SUPPORTED_LANGUAGES_COUNT; ++language_id)
names_refs[language_id].resize(max_region_id + 1, StringRef("", 0));
}
StringRef getRegionName(RegionID_t region_id, Language language = Language::RU) const
{
size_t language_id = static_cast<size_t>(language);
if (static_cast<size_t>(region_id) > names_refs[language_id].size())
return StringRef("", 0);
StringRef ref = names_refs[language_id][region_id];
while (ref.size == 0 && language_id != ROOT_LANGUAGE)
{
static const size_t FALLBACK[] = { 0, 0, 0, 0, 0, 1 };
language_id = FALLBACK[language_id];
ref = names_refs[language_id][region_id];
}
return ref;
}
static Language getLanguageEnum(const std::string & language)
{
if (language.size() == 2)
{
for (size_t i = 0; i < LANGUAGE_ALIASES_COUNT; ++i)
{
const auto & alias = getLanguageAliases()[i];
if (language[0] == alias.name[0] && language[1] == alias.name[1])
return alias.lang;
}
}
throw Poco::Exception("Unsupported language for region name. Supported languages are: " + dumpSupportedLanguagesNames() + ".");
}
static std::string dumpSupportedLanguagesNames()
{
std::string res = "";
for (size_t i = 0; i < LANGUAGE_ALIASES_COUNT; ++i)
{
if (i > 0)
res += ", ";
res += '\'';
res += getLanguageAliases()[i].name;
res += '\'';
}
return res;
}
private:
const std::string directory;
ModificationTimes_t file_modification_times = ModificationTimes_t(SUPPORTED_LANGUAGES_COUNT);
Logger * log = &Logger::get("RegionsNames");
/// Байты имен для каждого языка, уложенные подряд, разделенные нулями
CharsForLanguageID_t chars = CharsForLanguageID_t(SUPPORTED_LANGUAGES_COUNT);
/// Отображение для каждого языка из id региона в указатель на диапазон байт имени
StringRefsForLanguageID_t names_refs = StringRefsForLanguageID_t(SUPPORTED_LANGUAGES_COUNT);
};

View File

@ -0,0 +1,117 @@
#pragma once
#include <string.h>
#include <common/logger_useful.h>
#include <common/singleton.h>
#include <mysqlxx/PoolWithFailover.h>
/** @brief Класс, позволяющий узнавать, принадлежит ли поисковая система или операционная система
* другой поисковой или операционной системе, соответственно.
* Информацию об иерархии регионов загружает из БД.
*/
class TechDataHierarchy
{
private:
Logger * log;
UInt8 os_parent[256];
UInt8 se_parent[256];
public:
static constexpr auto required_key = "mysql_metrica";
TechDataHierarchy()
: log(&Logger::get("TechDataHierarchy"))
{
LOG_DEBUG(log, "Loading tech data hierarchy.");
memset(os_parent, 0, sizeof(os_parent));
memset(se_parent, 0, sizeof(se_parent));
mysqlxx::PoolWithFailover pool(required_key);
mysqlxx::Pool::Entry conn = pool.Get();
{
mysqlxx::Query q = conn->query("SELECT Id, COALESCE(Parent_Id, 0) FROM OS2");
LOG_TRACE(log, q.str());
mysqlxx::UseQueryResult res = q.use();
while (mysqlxx::Row row = res.fetch())
{
UInt64 child = row[0].getUInt();
UInt64 parent = row[1].getUInt();
if (child > 255 || parent > 255)
throw Poco::Exception("Too large OS id (> 255).");
os_parent[child] = parent;
}
}
{
mysqlxx::Query q = conn->query("SELECT Id, COALESCE(ParentId, 0) FROM SearchEngines");
LOG_TRACE(log, q.str());
mysqlxx::UseQueryResult res = q.use();
while (mysqlxx::Row row = res.fetch())
{
UInt64 child = row[0].getUInt();
UInt64 parent = row[1].getUInt();
if (child > 255 || parent > 255)
throw Poco::Exception("Too large search engine id (> 255).");
se_parent[child] = parent;
}
}
}
/// Отношение "принадлежит".
bool isOSIn(UInt8 lhs, UInt8 rhs) const
{
while (lhs != rhs && os_parent[lhs])
lhs = os_parent[lhs];
return lhs == rhs;
}
bool isSEIn(UInt8 lhs, UInt8 rhs) const
{
while (lhs != rhs && se_parent[lhs])
lhs = se_parent[lhs];
return lhs == rhs;
}
UInt8 OSToParent(UInt8 x) const
{
return os_parent[x];
}
UInt8 SEToParent(UInt8 x) const
{
return se_parent[x];
}
/// К самому верхнему предку.
UInt8 OSToMostAncestor(UInt8 x) const
{
while (os_parent[x])
x = os_parent[x];
return x;
}
UInt8 SEToMostAncestor(UInt8 x) const
{
while (se_parent[x])
x = se_parent[x];
return x;
}
};
class TechDataHierarchySingleton : public Singleton<TechDataHierarchySingleton>, public TechDataHierarchy {};

View File

@ -15,6 +15,15 @@
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int BAD_ARGUMENTS;
extern const int DICTIONARY_IS_EMPTY;
}
const auto initial_array_size = 1024;
const auto max_array_size = 500000;

View File

@ -14,6 +14,15 @@
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int BAD_ARGUMENTS;
extern const int DICTIONARY_IS_EMPTY;
}
class HashedDictionary final : public IDictionary
{
public:

View File

@ -12,6 +12,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNSUPPORTED_METHOD;
extern const int WRONG_PASSWORD;
extern const int MONGODB_INIT_FAILED;
}
/// Allows loading dictionaries from a MongoDB collection
class MongoDBDictionarySource final : public IDictionarySource
{

View File

@ -16,6 +16,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
}
/// Allows processing results of a MySQL query as a sequence of Blocks, simplifies chaining
class MySQLBlockInputStream final : public IProfilingBlockInputStream
{

View File

@ -11,6 +11,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNSUPPORTED_METHOD;
}
/// Allows loading dictionaries from a MySQL database
class MySQLDictionarySource final : public IDictionarySource
{

View File

@ -11,6 +11,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_DIVISION;
}
/** Арифметические функции: +, -, *, /, %,
* intDiv (целочисленное деление), унарный минус.
* Битовые функции: |, &, ^, ~.
@ -907,7 +913,10 @@ template <> struct FunctionUnaryArithmeticMonotonicity<NameBitNot>
/// Оптимизации для целочисленного деления на константу.
#define LIBDIVIDE_USE_SSE2 1
#if defined(__x86_64__)
#define LIBDIVIDE_USE_SSE2 1
#endif
#include <libdivide.h>
@ -941,6 +950,8 @@ struct DivideIntegralByConstantImpl
const A * a_pos = &a[0];
const A * a_end = a_pos + size;
ResultType * c_pos = &c[0];
#if defined(__x86_64__)
static constexpr size_t values_per_sse_register = 16 / sizeof(A);
const A * a_end_sse = a_pos + size / values_per_sse_register * values_per_sse_register;
@ -952,6 +963,7 @@ struct DivideIntegralByConstantImpl
a_pos += values_per_sse_register;
c_pos += values_per_sse_register;
}
#endif
while (a_pos < a_end)
{

View File

@ -33,6 +33,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ZERO_ARRAY_OR_TUPLE_INDEX;
extern const int SIZES_OF_ARRAYS_DOESNT_MATCH;
extern const int PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS;
}
/** Функции по работе с массивами:
*
* array(с1, с2, ...) - создать массив из констант.

View File

@ -1,5 +1,10 @@
#pragma once
#include <ext/enumerate.hpp>
#include <ext/collection_cast.hpp>
#include <ext/range.hpp>
#include <type_traits>
#include <DB/IO/WriteBufferFromVector.h>
#include <DB/IO/ReadBufferFromString.h>
#include <DB/DataTypes/DataTypeFactory.h>
@ -9,25 +14,26 @@
#include <DB/DataTypes/DataTypeDate.h>
#include <DB/DataTypes/DataTypeDateTime.h>
#include <DB/DataTypes/DataTypeEnum.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/DataTypes/DataTypeTuple.h>
#include <DB/Columns/ColumnString.h>
#include <DB/Columns/ColumnFixedString.h>
#include <DB/Columns/ColumnConst.h>
#include <DB/Functions/IFunction.h>
#include <DB/Core/FieldVisitors.h>
#include <ext/range.hpp>
#include <type_traits>
#include <DB/Interpreters/ExpressionActions.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/Columns/ColumnArray.h>
#include <DB/DataTypes/DataTypeTuple.h>
#include <ext/enumerate.hpp>
#include <ext/collection_cast.hpp>
#include <DB/Core/FieldVisitors.h>
#include <DB/Interpreters/ExpressionActions.h>
#include <DB/Functions/IFunction.h>
#include <DB/Functions/FunctionsMiscellaneous.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_PARSE_NUMBER;
}
/** Функции преобразования типов.
* toType - преобразование "естественным образом";
*/
@ -44,7 +50,8 @@ struct ConvertImpl
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
{
if (const ColumnVector<FromFieldType> * col_from = typeid_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
if (const ColumnVector<FromFieldType> * col_from
= typeid_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
{
ColumnVector<ToFieldType> * col_to = new ColumnVector<ToFieldType>;
block.getByPosition(result).column = col_to;

View File

@ -29,6 +29,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int DICTIONARIES_WAS_NOT_LOADED;
}
/** Функции, использующие словари Яндекс.Метрики
* - словари регионов, операционных систем, поисковых систем.
*

View File

@ -14,6 +14,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int SIZES_OF_ARRAYS_DOESNT_MATCH;
}
/** Функции высшего порядка для массивов:
*
* arrayMap(x1,...,xn -> expression, array1,...,arrayn) - применить выражение к каждому элементу массива (или набора параллельных массивов).

View File

@ -2,9 +2,12 @@
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/Functions/IFunction.h>
#include <vectorf128.h>
#include <vectormath_exp.h>
#include <vectormath_trig.h>
#if defined(__x86_64__)
#include <vectorf128.h>
#include <vectormath_exp.h>
#include <vectormath_trig.h>
#endif
namespace DB
@ -156,6 +159,22 @@ private:
}
};
template <typename Name, Float64(&Function)(Float64)>
struct UnaryFunctionPlain
{
static constexpr auto name = Name::name;
static constexpr auto rows_per_iteration = 1;
template <typename T>
static void execute(const T * const src, Float64 * const dst)
{
dst[0] = static_cast<Float64>(Function(static_cast<Float64>(src[0])));
}
};
#if defined(__x86_64__)
template <typename Name, Vec2d(&Function)(const Vec2d &)>
struct UnaryFunctionVectorized
{
@ -170,18 +189,11 @@ struct UnaryFunctionVectorized
}
};
template <typename Name, Float64(&Function)(Float64)>
struct UnaryFunctionPlain
{
static constexpr auto name = Name::name;
static constexpr auto rows_per_iteration = 1;
#else
template <typename T>
static void execute(const T * const src, Float64 * const dst)
{
dst[0] = static_cast<Float64>(Function(static_cast<Float64>(src[0])));
}
};
#define UnaryFunctionVectorized UnaryFunctionPlain
#endif
template <typename Impl> class FunctionMathBinaryFloat64 : public IFunction
@ -422,6 +434,22 @@ private:
}
};
template <typename Name, Float64(&Function)(Float64, Float64)>
struct BinaryFunctionPlain
{
static constexpr auto name = Name::name;
static constexpr auto rows_per_iteration = 1;
template <typename T1, typename T2>
static void execute(const T1 * const src_left, const T2 * const src_right, Float64 * const dst)
{
dst[0] = static_cast<Float64>(Function(static_cast<Float64>(src_left[0]), static_cast<Float64>(src_right[0])));
}
};
#if defined(__x86_64__)
template <typename Name, Vec2d(&Function)(const Vec2d &, const Vec2d &)>
struct BinaryFunctionVectorized
{
@ -436,6 +464,12 @@ struct BinaryFunctionVectorized
}
};
#else
#define BinaryFunctionVectorized BinaryFunctionPlain
#endif
struct EImpl
{
@ -478,7 +512,15 @@ using FunctionLog2 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Log2Name,
using FunctionExp10 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Exp10Name, exp10>>;
using FunctionLog10 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Log10Name, log10>>;
using FunctionSqrt = FunctionMathUnaryFloat64<UnaryFunctionVectorized<SqrtName, sqrt>>;
using FunctionCbrt = FunctionMathUnaryFloat64<UnaryFunctionVectorized<CbrtName, Power_rational<1, 3>::pow>>;
using FunctionCbrt = FunctionMathUnaryFloat64<UnaryFunctionVectorized<CbrtName,
#if defined(__x86_64__)
Power_rational<1, 3>::pow
#else
cbrt
#endif
>>;
using FunctionSin = FunctionMathUnaryFloat64<UnaryFunctionVectorized<SinName, sin>>;
using FunctionCos = FunctionMathUnaryFloat64<UnaryFunctionVectorized<CosName, cos>>;
using FunctionTan = FunctionMathUnaryFloat64<UnaryFunctionVectorized<TanName, tan>>;

View File

@ -32,6 +32,13 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_INDEX;
extern const int FUNCTION_IS_SPECIAL;
}
/** Вспомогательные функции:
*
* visibleWidth(x) - вычисляет приблизительную ширину при выводе значения в текстовом (tab-separated) виде на консоль.

View File

@ -10,6 +10,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_CLOCK_GETTIME;
}
/** Функции генерации псевдослучайных чисел.
* Функция может быть вызвана без аргументов или с одним аргументом.
* Аргумент игнорируется и служит лишь для того, чтобы несколько вызовов одной функции считались разными и не склеивались.

View File

@ -5,6 +5,10 @@
#include <type_traits>
#include <array>
#if defined(__x86_64__)
#include <smmintrin.h>
#endif
namespace DB
{
@ -13,6 +17,7 @@ namespace DB
* roundToExp2 - вниз до ближайшей степени двойки;
* roundDuration - вниз до ближайшего из: 0, 1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000;
* roundAge - вниз до ближайшего из: 0, 18, 25, 35, 45.
*
* round(x, N) - арифметическое округление (N = 0 по умолчанию).
* ceil(x, N) - наименьшее число, которое не меньше x (N = 0 по умолчанию).
* floor(x, N) - наибольшее число, которое не больше x (N = 0 по умолчанию).
@ -163,6 +168,12 @@ namespace DB
NullScale // возвращать нулевое значение
};
#if !defined(_MM_FROUND_NINT)
#define _MM_FROUND_NINT 0
#define _MM_FROUND_FLOOR 1
#define _MM_FROUND_CEIL 2
#endif
/** Реализация низкоуровневых функций округления для целочисленных значений.
*/
template<typename T, int rounding_mode, ScaleMode scale_mode, typename Enable = void>
@ -257,10 +268,11 @@ namespace DB
}
};
template<typename T>
#if defined(__x86_64__)
template <typename T>
class BaseFloatRoundingComputation;
template<>
template <>
class BaseFloatRoundingComputation<Float32>
{
public:
@ -298,7 +310,7 @@ namespace DB
}
};
template<>
template <>
class BaseFloatRoundingComputation<Float64>
{
public:
@ -522,6 +534,81 @@ namespace DB
_mm_storeu_pd(out, val);
}
};
#else
/// Реализация для ARM. Не векторизована. Не исправляет отрицательные нули.
template <int mode>
float roundWithMode(float x)
{
if (mode == _MM_FROUND_NINT) return roundf(x);
if (mode == _MM_FROUND_FLOOR) return floorf(x);
if (mode == _MM_FROUND_CEIL) return ceilf(x);
__builtin_unreachable();
}
template <int mode>
double roundWithMode(double x)
{
if (mode == _MM_FROUND_NINT) return round(x);
if (mode == _MM_FROUND_FLOOR) return floor(x);
if (mode == _MM_FROUND_CEIL) return ceil(x);
__builtin_unreachable();
}
template <typename T>
class BaseFloatRoundingComputation
{
public:
using Scale = T;
static const size_t data_count = 1;
static inline void prepare(size_t scale, Scale & mm_scale)
{
mm_scale = static_cast<T>(scale);
}
};
template <typename T, int rounding_mode, ScaleMode scale_mode>
class FloatRoundingComputation;
template <typename T, int rounding_mode>
class FloatRoundingComputation<T, rounding_mode, PositiveScale>
: public BaseFloatRoundingComputation<T>
{
public:
static inline void compute(const T * __restrict in, const T & scale, T * __restrict out)
{
out[0] = roundWithMode<rounding_mode>(in[0] * scale) / scale;
}
};
template <typename T, int rounding_mode>
class FloatRoundingComputation<T, rounding_mode, NegativeScale>
: public BaseFloatRoundingComputation<T>
{
public:
static inline void compute(const T * __restrict in, const T & scale, T * __restrict out)
{
out[0] = roundWithMode<rounding_mode>(in[0] / scale) * scale;
}
};
template <typename T, int rounding_mode>
class FloatRoundingComputation<T, rounding_mode, ZeroScale>
: public BaseFloatRoundingComputation<T>
{
public:
static inline void prepare(size_t scale, T & mm_scale)
{
}
static inline void compute(const T * __restrict in, const T & scale, T * __restrict out)
{
out[0] = roundWithMode<rounding_mode>(in[0]);
}
};
#endif
/** Реализация высокоуровневых функций округления.
*/
@ -906,7 +993,7 @@ namespace
/** Выбрать подходящий алгоритм обработки в зависимости от масштаба.
*/
template<typename T, template<typename> class U, int rounding_mode>
template<typename T, template <typename> class U, int rounding_mode>
struct Dispatcher
{
static inline void apply(Block & block, U<T> * col, const ColumnNumbers & arguments, size_t result)
@ -1053,9 +1140,10 @@ namespace
typedef FunctionUnaryArithmetic<RoundToExp2Impl, NameRoundToExp2> FunctionRoundToExp2;
typedef FunctionUnaryArithmetic<RoundDurationImpl, NameRoundDuration> FunctionRoundDuration;
typedef FunctionUnaryArithmetic<RoundAgeImpl, NameRoundAge> FunctionRoundAge;
typedef FunctionRounding<NameRound, _MM_FROUND_NINT> FunctionRound;
typedef FunctionRounding<NameCeil, _MM_FROUND_CEIL> FunctionCeil;
typedef FunctionRounding<NameFloor, _MM_FROUND_FLOOR> FunctionFloor;
typedef FunctionRounding<NameCeil, _MM_FROUND_CEIL> FunctionCeil;
struct PositiveMonotonicity

View File

@ -17,8 +17,10 @@
#include <DB/Functions/IFunction.h>
#include <ext/range.hpp>
#include <emmintrin.h>
#include <nmmintrin.h>
#if defined(__x86_64__)
#include <emmintrin.h>
#include <nmmintrin.h>
#endif
namespace DB
@ -233,11 +235,12 @@ struct LowerUpperImpl
private:
static void array(const UInt8 * src, const UInt8 * src_end, UInt8 * dst)
{
const auto flip_case_mask = 'A' ^ 'a';
#if defined(__x86_64__)
const auto bytes_sse = sizeof(__m128i);
const auto src_end_sse = src_end - (src_end - src) % bytes_sse;
const auto flip_case_mask = 'A' ^ 'a';
const auto v_not_case_lower_bound = _mm_set1_epi8(not_case_lower_bound - 1);
const auto v_not_case_upper_bound = _mm_set1_epi8(not_case_upper_bound + 1);
const auto v_flip_case_mask = _mm_set1_epi8(flip_case_mask);
@ -260,6 +263,7 @@ private:
/// store result back to destination
_mm_storeu_si128(reinterpret_cast<__m128i *>(dst), cased_chars);
}
#endif
for (; src < src_end; ++src, ++dst)
if (*src >= not_case_lower_bound && *src <= not_case_upper_bound)
@ -394,6 +398,7 @@ private:
static void array(const UInt8 * src, const UInt8 * src_end, UInt8 * dst)
{
#if defined(__x86_64__)
const auto bytes_sse = sizeof(__m128i);
auto src_end_sse = src + (src_end - src) / bytes_sse * bytes_sse;
@ -455,7 +460,7 @@ private:
}
}
}
#endif
/// handle remaining symbols
while (src < src_end)
toCase(src, src_end, dst);
@ -1006,7 +1011,7 @@ public:
{
if (arguments.size() < 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be at least 1.",
+ toString(arguments.size()) + ", should be at least 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
for (const auto arg_idx : ext::range(0, arguments.size()))

View File

@ -12,6 +12,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_LESS_ARGUMENTS_FOR_FUNCTION;
}
class ExpressionAction;

View File

@ -6,7 +6,6 @@
#include <DB/Common/Allocator.h>
#include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/Core/Defines.h>

View File

@ -9,6 +9,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ARGUMENT_OUT_OF_BOUND;
}
/** Буфер для чтения из сжатого файла с использованием кэша разжатых блоков.
* Кэш внешний - передаётся в качестве аргумента в конструктор.
* Позволяет увеличить производительность в случае, когда часто читаются одни и те же блоки.

View File

@ -10,7 +10,6 @@
#include <DB/Common/PODArray.h>
#include <DB/Common/ProfileEvents.h>
#include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/IO/ReadBuffer.h>
#include <DB/IO/BufferWithOwnMemory.h>
#include <DB/IO/CompressedStream.h>
@ -20,6 +19,15 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_COMPRESSION_METHOD;
extern const int TOO_LARGE_SIZE_COMPRESSED;
extern const int CHECKSUM_DOESNT_MATCH;
extern const int CANNOT_DECOMPRESS;
}
class CompressedReadBufferBase
{
protected:

View File

@ -10,6 +10,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ARGUMENT_OUT_OF_BOUND;
}
/// В отличие от CompressedReadBuffer, умеет делать seek.
class CompressedReadBufferFromFile : public CompressedReadBufferBase, public BufferWithOwnMemory<ReadBuffer>
{

View File

@ -21,6 +21,13 @@
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_COMPRESS;
extern const int UNKNOWN_COMPRESSION_METHOD;
}
class CompressedWriteBuffer : public BufferWithOwnMemory<WriteBuffer>
{
private:

View File

@ -4,13 +4,18 @@
#include <algorithm>
#include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/IO/BufferBase.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ATTEMPT_TO_READ_AFTER_EOF;
extern const int CANNOT_READ_ALL_DATA;
}
/** Простой абстрактный класс для буферизованного чтения данных (последовательности char) откуда-нибудь.
* В отличие от std::istream, предоставляет доступ к внутреннему буферу,
* а также позволяет вручную управлять позицией внутри буфера.

View File

@ -8,6 +8,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int FILE_DOESNT_EXIST;
extern const int CANNOT_OPEN_FILE;
extern const int CANNOT_CLOSE_FILE;
}
/** Принимает имя файла. Самостоятельно открывает и закрывает файл.
*/
class ReadBufferFromFile : public ReadBufferFromFileDescriptor

View File

@ -10,7 +10,6 @@
#include <DB/Common/Stopwatch.h>
#include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/IO/ReadBufferFromFileBase.h>
#include <DB/IO/ReadBuffer.h>
@ -21,6 +20,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_READ_FROM_FILE_DESCRIPTOR;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int CANNOT_SEEK_THROUGH_FILE;
extern const int CANNOT_SELECT;
}
/** Работает с готовым файловым дескриптором. Не открывает и не закрывает файл.
*/
class ReadBufferFromFileDescriptor : public ReadBufferFromFileBase

View File

@ -18,6 +18,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int RECEIVED_ERROR_FROM_REMOTE_IO_SERVER;
}
/** Делает указанный HTTP-запрос и отдает ответ.
*/
class ReadBufferFromHTTP : public ReadBuffer

View File

@ -3,7 +3,6 @@
#include <iostream>
#include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/IO/ReadBuffer.h>
#include <DB/IO/BufferWithOwnMemory.h>
@ -12,6 +11,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_READ_FROM_ISTREAM;
}
class ReadBufferFromIStream : public BufferWithOwnMemory<ReadBuffer>
{
private:

View File

@ -14,7 +14,6 @@
#include <DB/Core/Types.h>
#include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/IO/ReadBuffer.h>
#include <DB/IO/VarInt.h>
@ -26,6 +25,13 @@
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_PARSE_DATE;
extern const int CANNOT_PARSE_DATETIME;
extern const int CANNOT_READ_ARRAY_FROM_TEXT;
}
/// Функции-помошники для форматированного чтения
inline char parseEscapeSequence(char c)

View File

@ -21,6 +21,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_WRITE_TO_OSTREAM;
extern const int RECEIVED_ERROR_FROM_REMOTE_IO_SERVER;
}
/** Позволяет писать файл на удалённый сервер.
*/
class RemoteWriteBuffer : public WriteBuffer
@ -63,7 +69,7 @@ public:
{
Poco::URI::encode(path, "&#", encoded_path);
Poco::URI::encode(tmp_path, "&#", encoded_tmp_path);
std::stringstream uri;
uri << "http://" << host << ":" << port
<< "/?action=write"
@ -76,14 +82,14 @@ public:
session.setHost(host);
session.setPort(port);
session.setKeepAlive(true);
/// устанавливаем таймаут
session.setTimeout(connection_timeout, send_timeout, receive_timeout);
Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_POST, uri_str, Poco::Net::HTTPRequest::HTTP_1_1);
request.setChunkedTransferEncoding(true);
for (unsigned i = 0; i < connection_retries; ++i)
{
LOG_TRACE((&Logger::get("RemoteWriteBuffer")), "Sending write request to " << host << ":" << port << uri_str);
@ -126,7 +132,7 @@ public:
/// Для корректной работы с AsynchronousWriteBuffer, который подменяет буферы.
impl->set(buffer().begin(), buffer().size());
impl->position() = pos;
try
@ -145,7 +151,7 @@ public:
{
if (finalized)
return;
next();
checkStatus();
@ -193,7 +199,7 @@ private:
throw Exception(error_message.str(), ErrorCodes::RECEIVED_ERROR_FROM_REMOTE_IO_SERVER);
}
}
void rename()
{
std::stringstream uri;
@ -203,9 +209,9 @@ private:
<< "&to=" << encoded_path;
uri_str = Poco::URI(uri.str()).getPathAndQuery();
Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_GET, uri_str, Poco::Net::HTTPRequest::HTTP_1_1);
for (unsigned i = 0; i < connection_retries; ++i)
{
LOG_TRACE((&Logger::get("RemoteWriteBuffer")), "Sending rename request to " << host << ":" << port << uri_str);
@ -219,7 +225,7 @@ private:
{
if (i + 1 == connection_retries)
throw;
LOG_WARNING((&Logger::get("RemoteWriteBuffer")), e.what() << ", message: " << e.displayText()
<< ", URL: " << host << ":" << port << uri_str << ", try No " << i + 1 << ".");
session.reset();
@ -229,7 +235,7 @@ private:
{
if (i + 1 == connection_retries)
throw;
LOG_WARNING((&Logger::get("RemoteWriteBuffer")), "Connection timeout from " << host << ":" << port << uri_str << ", try No " << i + 1 << ".");
session.reset();
continue;

View File

@ -4,13 +4,18 @@
#include <cstring>
#include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/IO/BufferBase.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_WRITE_AFTER_END_OF_BUFFER;
}
/** Простой абстрактный класс для буферизованной записи данных (последовательности char) куда-нибудь.
* В отличие от std::ostream, предоставляет доступ к внутреннему буферу,
* а также позволяет вручную управлять позицией внутри буфера.

View File

@ -12,6 +12,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int FILE_DOESNT_EXIST;
extern const int CANNOT_OPEN_FILE;
extern const int CANNOT_CLOSE_FILE;
}
/** Принимает имя файла. Самостоятельно открывает и закрывает файл.
*/
class WriteBufferFromFile : public WriteBufferFromFileDescriptor

View File

@ -4,7 +4,6 @@
#include <errno.h>
#include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/Common/ProfileEvents.h>
#include <DB/IO/WriteBufferFromFileBase.h>
@ -16,6 +15,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_WRITE_TO_FILE_DESCRIPTOR;
extern const int CANNOT_FSYNC;
extern const int CANNOT_SEEK_THROUGH_FILE;
extern const int CANNOT_TRUNCATE_FILE;
}
/** Работает с готовым файловым дескриптором. Не открывает и не закрывает файл.
*/
class WriteBufferFromFileDescriptor : public WriteBufferFromFileBase

View File

@ -3,7 +3,6 @@
#include <Poco/Net/HTTPServerResponse.h>
#include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/IO/WriteBuffer.h>
#include <DB/IO/BufferWithOwnMemory.h>
@ -13,6 +12,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_WRITE_TO_OSTREAM;
}
/** Отличается от WriteBufferFromOStream тем, что инициализируется не std::ostream, а Poco::Net::HTTPServerResponse.
* При первом сбросе данных, получает из него std::ostream (с помощью метода send).

Some files were not shown because too many files have changed in this diff Show More