This commit is contained in:
Evgeniy Gatov 2015-09-25 00:33:26 +03:00
commit 7423bd1224
98 changed files with 2646 additions and 1228 deletions

View File

@ -1,28 +1,65 @@
#pragma once
#include <Poco/RegularExpression.h>
#include <DB/AggregateFunctions/IAggregateFunction.h>
#include <DB/DataTypes/IDataType.h>
#include <boost/iterator/transform_iterator.hpp>
namespace DB
{
/** Позволяет создать агрегатную функцию по её имени.
*/
class AggregateFunctionFactory
class AggregateFunctionFactory final
{
public:
using FunctionNames = std::vector<std::string>;
private:
/// Не std::function, так как меньше indirection и размер объекта.
using Creator = AggregateFunctionPtr(*)(const std::string & name, const DataTypes & argument_types);
public:
AggregateFunctionFactory();
AggregateFunctionPtr get(const String & name, const DataTypes & argument_types, int recursion_level = 0) const;
AggregateFunctionPtr tryGet(const String & name, const DataTypes & argument_types) const;
const FunctionNames & getFunctionNames() const;
bool isAggregateFunctionName(const String & name, int recursion_level = 0) const;
/// Зарегистрировать агрегатную функцию заданную по одному или нескольким названиям.
void registerFunction(const std::vector<std::string> & names, Creator creator);
AggregateFunctionFactory(const AggregateFunctionFactory &) = delete;
AggregateFunctionFactory & operator=(const AggregateFunctionFactory &) = delete;
private:
struct Descriptor
{
Creator creator;
bool is_alias;
};
using AggregateFunctions = std::unordered_map<std::string, Descriptor>;
public:
struct Details
{
std::string name;
bool is_alias;
};
private:
/// Вспомогательная функция для реализации итератора (см. ниже).
static Details getDetails(const AggregateFunctions::value_type & entry);
public:
/** Итератор над агрегатными функциями. Возвращает объект типа Details.
* Этот итератор нужен для таблицы system.functions.
*/
using const_iterator = boost::transform_iterator<decltype(&AggregateFunctionFactory::getDetails),
typename AggregateFunctions::const_iterator>;
public:
const_iterator begin() const;
const_iterator end() const;
private:
AggregateFunctions aggregate_functions;
};
}

View File

@ -2,6 +2,7 @@
#include <DB/DataTypes/DataTypeAggregateFunction.h>
#include <DB/AggregateFunctions/IAggregateFunction.h>
#include <DB/Columns/ColumnAggregateFunction.h>
namespace DB

View File

@ -2,6 +2,7 @@
#include <DB/AggregateFunctions/IAggregateFunction.h>
#include <DB/DataTypes/DataTypeDateTime.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/Parsers/CommonParsers.h>
#include <statdaemons/ext/range.hpp>
#include <boost/range/iterator_range_core.hpp>

View File

@ -1,8 +1,5 @@
#pragma once
#include <DB/IO/WriteHelpers.h>
#include <DB/IO/ReadHelpers.h>
#include <DB/AggregateFunctions/AggregateFunctionsMinMaxAny.h>

View File

@ -0,0 +1,103 @@
#pragma once
#include <DB/DataTypes/DataTypeDate.h>
#include <DB/DataTypes/DataTypeDateTime.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypeFixedString.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
namespace DB
{
/** Создать агрегатную функцию с числовым типом в параметре шаблона, в зависимости от типа аргумента.
*/
template <template <typename> class AggregateFunctionTemplate>
static IAggregateFunction * createWithNumericType(const IDataType & argument_type)
{
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionTemplate<UInt8>;
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return new AggregateFunctionTemplate<UInt16>;
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return new AggregateFunctionTemplate<UInt32>;
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return new AggregateFunctionTemplate<UInt64>;
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return new AggregateFunctionTemplate<Int8>;
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return new AggregateFunctionTemplate<Int16>;
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return new AggregateFunctionTemplate<Int32>;
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return new AggregateFunctionTemplate<Int64>;
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return new AggregateFunctionTemplate<Float32>;
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return new AggregateFunctionTemplate<Float64>;
else
return nullptr;
}
template <template <typename, typename> class AggregateFunctionTemplate, class Data>
static IAggregateFunction * createWithNumericType(const IDataType & argument_type)
{
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionTemplate<UInt8, Data>;
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return new AggregateFunctionTemplate<UInt16, Data>;
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return new AggregateFunctionTemplate<UInt32, Data>;
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return new AggregateFunctionTemplate<UInt64, Data>;
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return new AggregateFunctionTemplate<Int8, Data>;
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return new AggregateFunctionTemplate<Int16, Data>;
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return new AggregateFunctionTemplate<Int32, Data>;
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return new AggregateFunctionTemplate<Int64, Data>;
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return new AggregateFunctionTemplate<Float32, Data>;
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return new AggregateFunctionTemplate<Float64, Data>;
else
return nullptr;
}
template <template <typename, typename> class AggregateFunctionTemplate, template <typename> class Data>
static IAggregateFunction * createWithNumericType(const IDataType & argument_type)
{
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionTemplate<UInt8, Data<UInt8> >;
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return new AggregateFunctionTemplate<UInt16, Data<UInt16> >;
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return new AggregateFunctionTemplate<UInt32, Data<UInt32> >;
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return new AggregateFunctionTemplate<UInt64, Data<UInt64> >;
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return new AggregateFunctionTemplate<Int8, Data<Int8> >;
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return new AggregateFunctionTemplate<Int16, Data<Int16> >;
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return new AggregateFunctionTemplate<Int32, Data<Int32> >;
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return new AggregateFunctionTemplate<Int64, Data<Int64> >;
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return new AggregateFunctionTemplate<Float32, Data<Float32> >;
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return new AggregateFunctionTemplate<Float64, Data<Float64> >;
else
return nullptr;
}
/** Для шаблона с двумя аргументами.
*/
template <typename FirstType, template <typename, typename> class AggregateFunctionTemplate>
static IAggregateFunction * createWithTwoNumericTypesSecond(const IDataType & second_type)
{
if (typeid_cast<const DataTypeUInt8 *>(&second_type)) return new AggregateFunctionTemplate<FirstType, UInt8>;
else if (typeid_cast<const DataTypeUInt16 *>(&second_type)) return new AggregateFunctionTemplate<FirstType, UInt16>;
else if (typeid_cast<const DataTypeUInt32 *>(&second_type)) return new AggregateFunctionTemplate<FirstType, UInt32>;
else if (typeid_cast<const DataTypeUInt64 *>(&second_type)) return new AggregateFunctionTemplate<FirstType, UInt64>;
else if (typeid_cast<const DataTypeInt8 *>(&second_type)) return new AggregateFunctionTemplate<FirstType, Int8>;
else if (typeid_cast<const DataTypeInt16 *>(&second_type)) return new AggregateFunctionTemplate<FirstType, Int16>;
else if (typeid_cast<const DataTypeInt32 *>(&second_type)) return new AggregateFunctionTemplate<FirstType, Int32>;
else if (typeid_cast<const DataTypeInt64 *>(&second_type)) return new AggregateFunctionTemplate<FirstType, Int64>;
else if (typeid_cast<const DataTypeFloat32 *>(&second_type)) return new AggregateFunctionTemplate<FirstType, Float32>;
else if (typeid_cast<const DataTypeFloat64 *>(&second_type)) return new AggregateFunctionTemplate<FirstType, Float64>;
else
return nullptr;
}
template <template <typename, typename> class AggregateFunctionTemplate>
static IAggregateFunction * createWithTwoNumericTypes(const IDataType & first_type, const IDataType & second_type)
{
if (typeid_cast<const DataTypeUInt8 *>(&first_type)) return createWithTwoNumericTypesSecond<UInt8, AggregateFunctionTemplate>(second_type);
else if (typeid_cast<const DataTypeUInt16 *>(&first_type)) return createWithTwoNumericTypesSecond<UInt16, AggregateFunctionTemplate>(second_type);
else if (typeid_cast<const DataTypeUInt32 *>(&first_type)) return createWithTwoNumericTypesSecond<UInt32, AggregateFunctionTemplate>(second_type);
else if (typeid_cast<const DataTypeUInt64 *>(&first_type)) return createWithTwoNumericTypesSecond<UInt64, AggregateFunctionTemplate>(second_type);
else if (typeid_cast<const DataTypeInt8 *>(&first_type)) return createWithTwoNumericTypesSecond<Int8, AggregateFunctionTemplate>(second_type);
else if (typeid_cast<const DataTypeInt16 *>(&first_type)) return createWithTwoNumericTypesSecond<Int16, AggregateFunctionTemplate>(second_type);
else if (typeid_cast<const DataTypeInt32 *>(&first_type)) return createWithTwoNumericTypesSecond<Int32, AggregateFunctionTemplate>(second_type);
else if (typeid_cast<const DataTypeInt64 *>(&first_type)) return createWithTwoNumericTypesSecond<Int64, AggregateFunctionTemplate>(second_type);
else if (typeid_cast<const DataTypeFloat32 *>(&first_type)) return createWithTwoNumericTypesSecond<Float32, AggregateFunctionTemplate>(second_type);
else if (typeid_cast<const DataTypeFloat64 *>(&first_type)) return createWithTwoNumericTypesSecond<Float64, AggregateFunctionTemplate>(second_type);
else
return nullptr;
}
}

View File

@ -0,0 +1,116 @@
#pragma once
#include <DB/AggregateFunctions/AggregateFunctionsArgMinMax.h>
#include <DB/DataTypes/DataTypeDate.h>
#include <DB/DataTypes/DataTypeDateTime.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypeFixedString.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
namespace DB
{
/// min, max, any, anyLast
template <template <typename> class AggregateFunctionTemplate, template <typename> class Data>
static IAggregateFunction * createAggregateFunctionSingleValue(const String & name, const DataTypes & argument_types)
{
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const IDataType & argument_type = *argument_types[0];
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionTemplate<Data<SingleValueDataFixed<UInt8>>>;
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return new AggregateFunctionTemplate<Data<SingleValueDataFixed<UInt16>>>;
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return new AggregateFunctionTemplate<Data<SingleValueDataFixed<UInt32>>>;
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return new AggregateFunctionTemplate<Data<SingleValueDataFixed<UInt64>>>;
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return new AggregateFunctionTemplate<Data<SingleValueDataFixed<Int8>>>;
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return new AggregateFunctionTemplate<Data<SingleValueDataFixed<Int16>>>;
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return new AggregateFunctionTemplate<Data<SingleValueDataFixed<Int32>>>;
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return new AggregateFunctionTemplate<Data<SingleValueDataFixed<Int64>>>;
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return new AggregateFunctionTemplate<Data<SingleValueDataFixed<Float32>>>;
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return new AggregateFunctionTemplate<Data<SingleValueDataFixed<Float64>>>;
else if (typeid_cast<const DataTypeDate *>(&argument_type))
return new AggregateFunctionTemplate<Data<SingleValueDataFixed<DataTypeDate::FieldType>>>;
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
return new AggregateFunctionTemplate<Data<SingleValueDataFixed<DataTypeDateTime::FieldType>>>;
else if (typeid_cast<const DataTypeString*>(&argument_type))
return new AggregateFunctionTemplate<Data<SingleValueDataString>>;
else
return new AggregateFunctionTemplate<Data<SingleValueDataGeneric>>;
}
/// argMin, argMax
template <template <typename> class MinMaxData, typename ResData>
static IAggregateFunction * createAggregateFunctionArgMinMaxSecond(const String & name, const IDataType & val_type)
{
if (typeid_cast<const DataTypeUInt8 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<UInt8>>>>;
else if (typeid_cast<const DataTypeUInt16 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<UInt16>>>>;
else if (typeid_cast<const DataTypeUInt32 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<UInt32>>>>;
else if (typeid_cast<const DataTypeUInt64 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<UInt64>>>>;
else if (typeid_cast<const DataTypeInt8 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<Int8>>>>;
else if (typeid_cast<const DataTypeInt16 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<Int16>>>>;
else if (typeid_cast<const DataTypeInt32 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<Int32>>>>;
else if (typeid_cast<const DataTypeInt64 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<Int64>>>>;
else if (typeid_cast<const DataTypeFloat32 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<Float32>>>>;
else if (typeid_cast<const DataTypeFloat64 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<Float64>>>>;
else if (typeid_cast<const DataTypeDate *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<DataTypeDate::FieldType>>>>;
else if (typeid_cast<const DataTypeDateTime*>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<DataTypeDateTime::FieldType>>>>;
else if (typeid_cast<const DataTypeString*>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataString>>>;
else
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataGeneric>>>;
}
template <template <typename> class MinMaxData>
static IAggregateFunction * createAggregateFunctionArgMinMax(const String & name, const DataTypes & argument_types)
{
if (argument_types.size() != 2)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const IDataType & res_type = *argument_types[0];
const IDataType & val_type = *argument_types[1];
if (typeid_cast<const DataTypeUInt8 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<UInt8>>(name, val_type);
else if (typeid_cast<const DataTypeUInt16 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<UInt16>>(name, val_type);
else if (typeid_cast<const DataTypeUInt32 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<UInt32>>(name, val_type);
else if (typeid_cast<const DataTypeUInt64 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<UInt64>>(name, val_type);
else if (typeid_cast<const DataTypeInt8 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<Int8>>(name, val_type);
else if (typeid_cast<const DataTypeInt16 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<Int16>>(name, val_type);
else if (typeid_cast<const DataTypeInt32 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<Int32>>(name, val_type);
else if (typeid_cast<const DataTypeInt64 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<Int64>>(name, val_type);
else if (typeid_cast<const DataTypeFloat32 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<Float32>>(name, val_type);
else if (typeid_cast<const DataTypeFloat64 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<Float64>>(name, val_type);
else if (typeid_cast<const DataTypeDate *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<DataTypeDate::FieldType>>(name, val_type);
else if (typeid_cast<const DataTypeDateTime*>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<DataTypeDateTime::FieldType>>(name, val_type);
else if (typeid_cast<const DataTypeString*>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataString>(name, val_type);
else
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataGeneric>(name, val_type);
}
}

View File

@ -7,6 +7,7 @@
#include <sstream>
#include <stats/ReservoirSampler.h>
#include <Yandex/Common.h>
#include <DB/Common/HashTable/Hash.h>
#include <DB/IO/ReadBuffer.h>
#include <DB/IO/ReadHelpers.h>
#include <DB/IO/WriteHelpers.h>

View File

@ -0,0 +1,7 @@
#pragma once
/** Устанавливает имя потока (максимальная длина - 15 байт),
* которое будет видно в ps, gdb, /proc,
* для удобства наблюдений и отладки.
*/
void setThreadName(const char * name);

View File

@ -5,6 +5,7 @@
#include <Poco/Event.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>
#include <DB/Common/setThreadName.h>
namespace DB
@ -120,6 +121,7 @@ protected:
/// Вычисления, которые могут выполняться в отдельном потоке
void calculate(MemoryTracker * memory_tracker)
{
setThreadName("AsyncBlockInput");
current_memory_tracker = memory_tracker;
try

View File

@ -9,6 +9,7 @@
#include <Yandex/logger_useful.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>
#include <DB/Common/setThreadName.h>
/** Позволяет обработать множество источников блоков параллельно, используя указанное количество потоков.
@ -141,6 +142,8 @@ private:
current_memory_tracker = memory_tracker;
ExceptionPtr exception;
setThreadName("ParalInputsProc");
try
{
loop(thread_num);

View File

@ -344,16 +344,56 @@ struct LowerUpperUTF8Impl
reinterpret_cast<UInt8 *>(&res_data[0]));
}
/** Converts a single code point starting at `src` to desired case, storing result starting at `dst`.
* `src` and `dst` are incremented by corresponding sequence lengths. */
static void toCase(const UInt8 * & src, const UInt8 * const src_end, UInt8 * & dst)
{
if (src[0] <= ascii_upper_bound)
{
if (*src >= not_case_lower_bound && *src <= not_case_upper_bound)
*dst++ = *src++ ^ flip_case_mask;
else
*dst++ = *src++;
}
else if (src + 1 < src_end &&
((src[0] == 0xD0u && (src[1] >= 0x80u && src[1] <= 0xBFu)) ||
(src[0] == 0xD1u && (src[1] >= 0x80u && src[1] <= 0x9Fu))))
{
cyrillic_to_case(src, src_end, dst);
}
else if (src + 1 < src_end && src[0] == 0xC2u)
{
/// Пунктуация U+0080 - U+00BF, UTF-8: C2 80 - C2 BF
*dst++ = *src++;
*dst++ = *src++;
}
else if (src + 2 < src_end && src[0] == 0xE2u)
{
/// Символы U+2000 - U+2FFF, UTF-8: E2 80 80 - E2 BF BF
*dst++ = *src++;
*dst++ = *src++;
*dst++ = *src++;
}
else
{
static const Poco::UTF8Encoding utf8;
if (const auto chars = utf8.convert(to_case(utf8.convert(src)), dst, src_end - src))
src += chars, dst += chars;
else
++src, ++dst;
}
}
private:
static constexpr auto ascii_upper_bound = '\x7f';
static constexpr auto flip_case_mask = 'A' ^ 'a';
static void array(const UInt8 * src, const UInt8 * src_end, UInt8 * dst)
{
static const Poco::UTF8Encoding utf8;
const auto bytes_sse = sizeof(__m128i);
auto src_end_sse = src + (src_end - src) / bytes_sse * bytes_sse;
const auto flip_case_mask = 'A' ^ 'a';
const auto ascii_upper_bound = '\x7f';
/// SSE2 packed comparison operate on signed types, hence compare (c < 0) instead of (c > 0x7f)
const auto v_zero = _mm_setzero_si128();
const auto v_not_case_lower_bound = _mm_set1_epi8(not_case_lower_bound - 1);
@ -399,41 +439,7 @@ private:
const auto expected_end = src + bytes_sse;
while (src < expected_end)
{
if (src[0] <= ascii_upper_bound)
{
if (*src >= not_case_lower_bound && *src <= not_case_upper_bound)
*dst++ = *src++ ^ flip_case_mask;
else
*dst++ = *src++;
}
else if (src + 1 < src_end &&
((src[0] == 0xD0u && (src[1] >= 0x80u && src[1] <= 0xBFu)) ||
(src[0] == 0xD1u && (src[1] >= 0x80u && src[1] <= 0x9Fu))))
{
cyrillic_to_case(src, src_end, dst);
}
else if (src + 1 < src_end && src[0] == 0xC2u)
{
/// Пунктуация U+0080 - U+00BF, UTF-8: C2 80 - C2 BF
*dst++ = *src++;
*dst++ = *src++;
}
else if (src + 2 < src_end && src[0] == 0xE2u)
{
/// Символы U+2000 - U+2FFF, UTF-8: E2 80 80 - E2 BF BF
*dst++ = *src++;
*dst++ = *src++;
*dst++ = *src++;
}
else
{
if (const auto chars = utf8.convert(to_case(utf8.convert(src)), dst, src_end - src))
src += chars, dst += chars;
else
++src, ++dst;
}
}
toCase(src, src_end, dst);
/// adjust src_end_sse by pushing it forward or backward
const auto diff = src - expected_end;
@ -449,10 +455,7 @@ private:
/// handle remaining symbols
while (src < src_end)
if (const auto chars = utf8.convert(to_case(utf8.convert(src)), dst, src_end - src))
src += chars, dst += chars;
else
++src, ++dst;
toCase(src, src_end, dst);
}
};

View File

@ -15,10 +15,12 @@
#include <DB/Functions/IFunction.h>
#include <re2/re2.h>
#include <re2/stringpiece.h>
#include <Poco/UTF8Encoding.h>
#include <mutex>
#include <stack>
#include <statdaemons/ext/range.hpp>
#include <Poco/Unicode.h>
namespace DB
@ -103,6 +105,43 @@ struct PositionImpl
};
namespace
{
const UInt8 utf8_continuation_octet_mask = 0b11000000u;
const UInt8 utf8_continuation_octet = 0b10000000u;
/// return true if `octet` binary repr starts with 10 (octet is a UTF-8 sequence continuation)
bool utf8_is_continuation_octet(const UInt8 octet)
{
return (octet & utf8_continuation_octet_mask) == utf8_continuation_octet;
}
/// moves `s` forward until either first non-continuation octet or string end is met
void utf8_sync_forward(const UInt8 * & s, const UInt8 * const end = nullptr)
{
while (s < end && utf8_is_continuation_octet(*s))
++s;
}
/// returns UTF-8 code point sequence length judging by it's first octet
std::size_t utf8_seq_length(const UInt8 first_octet)
{
if (first_octet < 0x80u)
return 1;
const std::size_t bits = 8;
const auto first_zero = _bit_scan_reverse(static_cast<UInt8>(~first_octet));
return bits - 1 - first_zero;
}
}
struct PositionUTF8Impl
{
typedef UInt64 ResultType;
@ -136,7 +175,7 @@ struct PositionUTF8Impl
/// А теперь надо найти, сколько кодовых точек находится перед pos.
res[i] = 1;
for (const UInt8 * c = begin + (i != 0 ? offsets[i - 1] : 0); c < pos; ++c)
if (*c <= 0x7F || *c >= 0xC0)
if (!utf8_is_continuation_octet(*c))
++res[i];
}
else
@ -151,63 +190,85 @@ struct PositionUTF8Impl
static void constant(const std::string & data, const std::string & needle, UInt64 & res)
{
res = data.find(needle);
if (res == std::string::npos)
res = 0;
const auto pos = data.find(needle);
if (pos != std::string::npos)
{
/// А теперь надо найти, сколько кодовых точек находится перед pos.
res = 1;
for (const auto i : ext::range(0, pos))
if (!utf8_is_continuation_octet(static_cast<UInt8>(data[i])))
++res;
}
else
++res;
res = 0;
}
};
struct PositionCaseInsensitiveImpl
{
using ResultType = UInt64;
static void vector(
const ColumnString::Chars_t & data, const ColumnString::Offsets_t & offsets, const std::string & needle,
PODArray<UInt64> & res)
private:
class CaseInsensitiveSearcher
{
static constexpr auto n = sizeof(__m128i);
const int page_size = getpagesize();
/// string to be searched for
const std::string & needle;
/// lower and uppercase variants of the first character in `needle`
const auto l = std::tolower(needle.front());
const auto u = std::toupper(needle.front());
/// for detecting leftmost position of the first symbol
const auto patl = _mm_set1_epi8(l);
const auto patu = _mm_set1_epi8(u);
UInt8 l{};
UInt8 u{};
/// 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`
auto cachel = _mm_setzero_si128();
auto cacheu = _mm_setzero_si128();
int cachemask = 0;
__m128i cachel = _mm_setzero_si128(), cacheu = _mm_setzero_si128();
int cachemask{};
const auto n = sizeof(cachel);
const auto needle_begin = needle.data();
const auto needle_end = needle_begin + needle.size();
auto needle_pos = needle_begin;
for (const auto i : ext::range(0, n))
bool page_safe(const void * const ptr) const
{
cachel = _mm_srli_si128(cachel, 1);
cacheu = _mm_srli_si128(cacheu, 1);
return ((page_size - 1) & reinterpret_cast<std::uintptr_t>(ptr)) <= page_size - n;
}
cachel = _mm_insert_epi8(cachel, std::tolower(*needle_pos), n - 1);
cacheu = _mm_insert_epi8(cacheu, std::toupper(*needle_pos), n - 1);
public:
CaseInsensitiveSearcher(const std::string & needle) : needle(needle)
{
if (needle.empty())
return;
if (needle_pos != needle_end)
auto needle_pos = needle.data();
l = std::tolower(*needle_pos);
u = std::toupper(*needle_pos);
patl = _mm_set1_epi8(l);
patu = _mm_set1_epi8(u);
const auto needle_end = needle_pos + needle.size();
for (const auto i : ext::range(0, n))
{
cachemask |= 1 << i;
++needle_pos;
cachel = _mm_srli_si128(cachel, 1);
cacheu = _mm_srli_si128(cacheu, 1);
if (needle_pos != needle_end)
{
cachel = _mm_insert_epi8(cachel, std::tolower(*needle_pos), n - 1);
cacheu = _mm_insert_epi8(cacheu, std::toupper(*needle_pos), n - 1);
cachemask |= 1 << i;
++needle_pos;
}
}
}
const auto page_size = getpagesize();
const auto page_safe = [&] (const void * const ptr) {
return ((page_size - 1) & reinterpret_cast<std::uintptr_t>(ptr)) <= page_size - n;
};
const auto find_ci = [&] (const UInt8 * haystack, const UInt8 * const haystack_end) -> const UInt8 * {
if (needle_begin == needle_end)
const UInt8 * find(const UInt8 * haystack, const UInt8 * const haystack_end) const
{
if (needle.empty())
return haystack;
const auto needle_begin = reinterpret_cast<const UInt8 *>(needle.data());
const auto needle_end = needle_begin + needle.size();
while (haystack < haystack_end)
{
/// @todo supposedly for long strings spanning across multiple pages. Why don't we use this technique in other places?
@ -241,13 +302,14 @@ struct PositionCaseInsensitiveImpl
{
if (mask == cachemask)
{
auto s1 = haystack + n;
auto s2 = needle_begin + n;
auto haystack_pos = haystack + n;
auto needle_pos = needle_begin + n;
while (s1 < haystack_end && s2 < needle_end && std::tolower(*s1) == std::tolower(*s2))
++s1, ++s2;
while (haystack_pos < haystack_end && needle_pos < needle_end &&
std::tolower(*haystack_pos) == std::tolower(*needle_pos))
++haystack_pos, ++needle_pos;
if (s2 == needle_end)
if (needle_pos == needle_end)
return haystack;
}
}
@ -260,17 +322,18 @@ struct PositionCaseInsensitiveImpl
}
if (haystack == haystack_end)
return haystack;
return haystack_end;
if (*haystack == l || *haystack == u)
{
auto s1 = haystack + 1;
auto s2 = needle_begin + 1;
auto haystack_pos = haystack + 1;
auto needle_pos = needle_begin + 1;
while (s1 < haystack_end && s2 < needle_end && std::tolower(*s1) == std::tolower(*s2))
++s1, ++s2;
while (haystack_pos < haystack_end && needle_pos < needle_end &&
std::tolower(*haystack_pos) == std::tolower(*needle_pos))
++haystack_pos, ++needle_pos;
if (s2 == needle_end)
if (needle_pos == needle_end)
return haystack;
}
@ -278,7 +341,17 @@ struct PositionCaseInsensitiveImpl
}
return haystack_end;
};
}
};
public:
using ResultType = UInt64;
static void vector(
const ColumnString::Chars_t & data, const ColumnString::Offsets_t & offsets, const std::string & needle,
PODArray<UInt64> & res)
{
const CaseInsensitiveSearcher searcher{needle};
const UInt8 * begin = &data[0];
const UInt8 * pos = begin;
@ -288,7 +361,7 @@ struct PositionCaseInsensitiveImpl
size_t i = 0;
/// Искать будем следующее вхождение сразу во всех строках.
while (pos < end && end != (pos = find_ci(pos, end)))
while (pos < end && end != (pos = searcher.find(pos, end)))
{
/// Определим, к какому индексу оно относится.
while (begin + offsets[i] < pos)
@ -326,24 +399,290 @@ struct PositionCaseInsensitiveImpl
struct PositionCaseInsensitiveUTF8Impl
{
private:
class CaseInsensitiveSearcher
{
using UTF8SequenceBuffer = UInt8[6];
static constexpr auto n = sizeof(__m128i);
const int page_size = getpagesize();
/// string to be searched for
const std::string & needle;
bool first_needle_symbol_is_ascii{};
/// lower and uppercase variants of the first octet of the first character in `needle`
UInt8 l{};
UInt8 u{};
/// 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{};
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;
}
public:
CaseInsensitiveSearcher(const std::string & needle) : needle(needle)
{
if (needle.empty())
return;
static const Poco::UTF8Encoding utf8;
UTF8SequenceBuffer l_seq, u_seq;
auto needle_pos = reinterpret_cast<const UInt8 *>(needle.data());
if (*needle_pos < 0x80u)
{
first_needle_symbol_is_ascii = true;
l = std::tolower(*needle_pos);
u = std::toupper(*needle_pos);
}
else
{
const auto first_u32 = utf8.convert(needle_pos);
const auto first_l_u32 = Poco::Unicode::toLower(first_u32);
const auto first_u_u32 = Poco::Unicode::toUpper(first_u32);
/// lower and uppercase variants of the first octet of the first character in `needle`
utf8.convert(first_l_u32, l_seq, sizeof(l_seq));
l = l_seq[0];
utf8.convert(first_u_u32, u_seq, sizeof(u_seq));
u = u_seq[0];
}
/// for detecting leftmost position of the first symbol
patl = _mm_set1_epi8(l);
patu = _mm_set1_epi8(u);
/// lower and uppercase vectors of first 16 octets of `needle`
const auto needle_end = needle_pos + needle.size();
for (std::size_t i = 0; i < n;)
{
if (needle_pos == needle_end)
{
cachel = _mm_srli_si128(cachel, 1);
cacheu = _mm_srli_si128(cacheu, 1);
++i;
continue;
}
const auto src_len = utf8_seq_length(*needle_pos);
const auto c_u32 = utf8.convert(needle_pos);
const auto c_l_u32 = Poco::Unicode::toLower(c_u32);
const auto c_u_u32 = Poco::Unicode::toUpper(c_u32);
const auto dst_l_len = static_cast<UInt8>(utf8.convert(c_l_u32, l_seq, sizeof(l_seq)));
const auto dst_u_len = static_cast<UInt8>(utf8.convert(c_u_u32, u_seq, sizeof(u_seq)));
/// @note Unicode standard states it is a rare but possible occasion
if (!(dst_l_len == dst_u_len && dst_u_len == src_len))
throw Exception{
"UTF8 sequences with different lowercase and uppercase lengths are not supported",
ErrorCodes::UNSUPPORTED_PARAMETER
};
cache_actual_len += src_len;
if (cache_actual_len < n)
cache_valid_len += src_len;
for (std::size_t j = 0; j < src_len && i < n; ++j, ++i)
{
cachel = _mm_srli_si128(cachel, 1);
cacheu = _mm_srli_si128(cacheu, 1);
if (needle_pos != needle_end)
{
cachel = _mm_insert_epi8(cachel, l_seq[j], n - 1);
cacheu = _mm_insert_epi8(cacheu, u_seq[j], n - 1);
cachemask |= 1 << i;
++needle_pos;
}
}
}
}
const UInt8 * find(const UInt8 * haystack, const UInt8 * const haystack_end) const
{
if (needle.empty())
return haystack;
static const Poco::UTF8Encoding utf8;
const auto needle_begin = reinterpret_cast<const UInt8 *>(needle.data());
const auto needle_end = needle_begin + needle.size();
while (haystack < haystack_end)
{
if (haystack + n <= haystack_end && page_safe(haystack))
{
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
const auto v_against_l = _mm_cmpeq_epi8(v_haystack, patl);
const auto v_against_u = _mm_cmpeq_epi8(v_haystack, patu);
const auto v_against_l_or_u = _mm_or_si128(v_against_l, v_against_u);
const auto mask = _mm_movemask_epi8(v_against_l_or_u);
if (mask == 0)
{
haystack += n;
utf8_sync_forward(haystack, haystack_end);
continue;
}
const auto offset = _bit_scan_forward(mask);
haystack += offset;
if (haystack < haystack_end && haystack + n <= haystack_end && page_safe(haystack))
{
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
const auto v_against_l = _mm_cmpeq_epi8(v_haystack, cachel);
const auto v_against_u = _mm_cmpeq_epi8(v_haystack, cacheu);
const auto v_against_l_or_u = _mm_or_si128(v_against_l, v_against_u);
const auto mask = _mm_movemask_epi8(v_against_l_or_u);
if (0xffff == cachemask)
{
if (mask == cachemask)
{
auto haystack_pos = haystack + cache_valid_len;
auto needle_pos = needle_begin + cache_valid_len;
while (haystack_pos < haystack_end && needle_pos < needle_end &&
Poco::Unicode::toLower(utf8.convert(haystack_pos)) ==
Poco::Unicode::toLower(utf8.convert(needle_pos)))
{
/// @note assuming sequences for lowercase and uppercase have exact same length
const auto len = utf8_seq_length(*haystack_pos);
haystack_pos += len, needle_pos += len;
}
if (needle_pos == needle_end)
return haystack;
}
}
else if ((mask & cachemask) == cachemask)
return haystack;
/// first octet was ok, but not the first 16, move to start of next sequence and reapply
haystack += utf8_seq_length(*haystack);
continue;
}
}
if (haystack == haystack_end)
return haystack_end;
if (*haystack == l || *haystack == u)
{
auto haystack_pos = haystack + first_needle_symbol_is_ascii;
auto needle_pos = needle_begin + first_needle_symbol_is_ascii;
while (haystack_pos < haystack_end && needle_pos < needle_end &&
Poco::Unicode::toLower(utf8.convert(haystack_pos)) ==
Poco::Unicode::toLower(utf8.convert(needle_pos)))
{
const auto len = utf8_seq_length(*haystack_pos);
haystack_pos += len, needle_pos += len;
}
if (needle_pos == needle_end)
return haystack;
}
/// advance to the start of the next sequence
haystack += utf8_seq_length(*haystack);
}
return haystack_end;
}
};
public:
using ResultType = UInt64;
static void vector(
const ColumnString::Chars_t & data, const ColumnString::Offsets_t & offsets, const std::string & needle,
PODArray<UInt64> & res)
{
throw Exception{
"Not yet implemented",
ErrorCodes::NOT_IMPLEMENTED
};
const CaseInsensitiveSearcher searcher{needle};
const UInt8 * begin = &data[0];
const UInt8 * pos = begin;
const UInt8 * end = pos + data.size();
/// Текущий индекс в массиве строк.
size_t i = 0;
/// Искать будем следующее вхождение сразу во всех строках.
while (pos < end && end != (pos = searcher.find(pos, end)))
{
/// Определим, к какому индексу оно относится.
while (begin + offsets[i] < pos)
{
res[i] = 0;
++i;
}
/// Проверяем, что вхождение не переходит через границы строк.
if (pos + needle.size() < begin + offsets[i])
{
/// А теперь надо найти, сколько кодовых точек находится перед pos.
res[i] = 1;
for (const UInt8 * c = begin + (i != 0 ? offsets[i - 1] : 0); c < pos; ++c)
if (!utf8_is_continuation_octet(*c))
++res[i];
}
else
res[i] = 0;
pos = begin + offsets[i];
++i;
}
memset(&res[i], 0, (res.size() - i) * sizeof(res[0]));
}
static void constant(const std::string & data, const std::string & needle, UInt64 & res)
static void constant(std::string data, std::string needle, UInt64 & res)
{
throw Exception{
"Not yet implemented",
ErrorCodes::NOT_IMPLEMENTED
};
static const Poco::UTF8Encoding utf8;
auto data_pos = reinterpret_cast<UInt8 *>(&data[0]);
const auto data_end = data_pos + data.size();
while (data_pos < data_end)
{
const auto len = utf8.convert(Poco::Unicode::toLower(utf8.convert(data_pos)), data_pos, data_end - data_pos);
data_pos += len;
}
auto needle_pos = reinterpret_cast<UInt8 *>(&needle[0]);
const auto needle_end = needle_pos + needle.size();
while (needle_pos < needle_end)
{
const auto len = utf8.convert(Poco::Unicode::toLower(utf8.convert(needle_pos)), needle_pos, needle_end - needle_pos);
needle_pos += len;
}
const auto pos = data.find(needle);
if (pos != std::string::npos)
{
/// А теперь надо найти, сколько кодовых точек находится перед pos.
res = 1;
for (const auto i : ext::range(0, pos))
if (!utf8_is_continuation_octet(static_cast<UInt8>(data[i])))
++res;
}
else
res = 0;
}
};
@ -1394,7 +1733,7 @@ public:
struct NamePosition { static constexpr auto name = "position"; };
struct NamePositionUTF8 { static constexpr auto name = "positionUTF8"; };
struct NamePositionCaseInsensitive { static constexpr auto name = "positionCaseInsensitive"; };
struct NamePositionCaseInsenseitiveUTF8 { static constexpr auto name = "positionCaseInsensitiveUTF8"; };
struct NamePositionCaseInsensitiveUTF8 { static constexpr auto name = "positionCaseInsensitiveUTF8"; };
struct NameMatch { static constexpr auto name = "match"; };
struct NameLike { static constexpr auto name = "like"; };
struct NameNotLike { static constexpr auto name = "notLike"; };
@ -1407,7 +1746,7 @@ struct NameReplaceRegexpAll { static constexpr auto name = "replaceRegexpAll"
typedef FunctionsStringSearch<PositionImpl, NamePosition> FunctionPosition;
typedef FunctionsStringSearch<PositionUTF8Impl, NamePositionUTF8> FunctionPositionUTF8;
typedef FunctionsStringSearch<PositionCaseInsensitiveImpl, NamePositionCaseInsensitive> FunctionPositionCaseInsensitive;
typedef FunctionsStringSearch<PositionCaseInsensitiveUTF8Impl, NamePositionCaseInsenseitiveUTF8> FunctionPositionCaseInsensitiveUTF8;
typedef FunctionsStringSearch<PositionCaseInsensitiveUTF8Impl, NamePositionCaseInsensitiveUTF8> FunctionPositionCaseInsensitiveUTF8;
typedef FunctionsStringSearch<MatchImpl<false>, NameMatch> FunctionMatch;
typedef FunctionsStringSearch<MatchImpl<true>, NameLike> FunctionLike;
typedef FunctionsStringSearch<MatchImpl<true, true>, NameNotLike> FunctionNotLike;

View File

@ -190,6 +190,8 @@ public:
/// Как другие серверы могут обратиться к этому для скачивания реплицируемых данных.
void setInterserverIOAddress(const String & host, UInt16 port);
std::pair<String, UInt16> getInterserverIOAddress() const;
/// Порт, который сервер слушает для выполнения SQL-запросов.
UInt16 getTCPPort() const;
/// Получить запрос на CREATE таблицы.
ASTPtr getCreateQuery(const String & database_name, const String & table_name) const;

View File

@ -6,6 +6,7 @@
#include <statdaemons/RegionsHierarchies.h>
#include <statdaemons/TechDataHierarchy.h>
#include <statdaemons/RegionsNames.h>
#include <DB/Common/setThreadName.h>
namespace DB
@ -123,6 +124,8 @@ private:
/// Обновляет каждые reload_period секунд.
void reloadPeriodically()
{
setThreadName("DictReload");
while (true)
{
if (destroy.tryWait(reload_period * 1000))

View File

@ -3,6 +3,7 @@
#include <DB/Dictionaries/IDictionary.h>
#include <DB/Core/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/Common/setThreadName.h>
#include <Yandex/MultiVersion.h>
#include <Yandex/logger_useful.h>
#include <Poco/Event.h>
@ -86,6 +87,8 @@ private:
void reloadPeriodically()
{
setThreadName("ExterDictReload");
while (true)
{
if (destroy.tryWait(check_period_sec * 1000))

View File

@ -8,11 +8,16 @@
#include <Poco/Net/DNS.h>
#include <Poco/Util/Application.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Poco/String.h>
#include <DB/Core/Types.h>
#include <DB/Core/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/IO/ReadHelpers.h>
#include <DB/IO/HexWriteBuffer.h>
#include <DB/IO/WriteBufferFromString.h>
#include <openssl/sha.h>
#include <Yandex/logger_useful.h>
@ -64,7 +69,7 @@ public:
construct(Poco::Net::IPAddress(addr), prefix_bits_);
}
}
bool contains(const Poco::Net::IPAddress & addr) const
{
return prefixBitsEquals(reinterpret_cast<const char *>(toIPv6(addr).addr()), reinterpret_cast<const char *>(mask_address.addr()), prefix_bits);
@ -168,7 +173,7 @@ private:
public:
HostRegexpPattern(const String & host_regexp_) : host_regexp(host_regexp_) {}
bool contains(const Poco::Net::IPAddress & addr) const
{
Poco::Net::SocketAddress sock_addr(addr, 0);
@ -243,7 +248,7 @@ public:
pattern = new HostExactPattern(value);
else
throw Exception("Unknown address pattern type: " + *it, ErrorCodes::UNKNOWN_ADDRESS_PATTERN_TYPE);
patterns.push_back(pattern);
}
}
@ -256,8 +261,9 @@ struct User
{
String name;
/// Требуемый пароль. Хранится в открытом виде.
/// Требуемый пароль. Может храниться либо в открытом виде, либо в виде SHA256.
String password;
String password_sha256_hex;
String profile;
String quota;
@ -267,7 +273,26 @@ struct User
User(const String & name_, const String & config_elem, Poco::Util::AbstractConfiguration & config)
: name(name_)
{
password = config.getString(config_elem + ".password");
bool has_password = config.has(config_elem + ".password");
bool has_password_sha256_hex = config.has(config_elem + ".password_sha256_hex");
if (has_password && has_password_sha256_hex)
throw Exception("Both fields 'password' and 'password_sha256_hex' are specified for user " + name + ". Must be only one of them.", ErrorCodes::BAD_ARGUMENTS);
if (!has_password && !has_password_sha256_hex)
throw Exception("Either 'password' or 'password_sha256_hex' must be specified for user " + name + ".", ErrorCodes::BAD_ARGUMENTS);
if (has_password)
password = config.getString(config_elem + ".password");
if (has_password_sha256_hex)
{
password_sha256_hex = Poco::toLower(config.getString(config_elem + ".password_sha256_hex"));
if (password_sha256_hex.size() != 64)
throw Exception("password_sha256_hex for user " + name + " has length " + toString(password_sha256_hex.size()) + " but must be exactly 64 symbols.", ErrorCodes::BAD_ARGUMENTS);
}
profile = config.getString(config_elem + ".profile");
quota = config.getString(config_elem + ".quota");
@ -285,7 +310,7 @@ class Users
private:
typedef std::map<String, User> Container;
Container cont;
public:
void loadFromConfig(Poco::Util::AbstractConfiguration & config)
{
@ -308,12 +333,38 @@ public:
if (!it->second.addresses.contains(address))
throw Exception("User " + name + " is not allowed to connect from address " + address.toString(), ErrorCodes::IP_ADDRESS_NOT_ALLOWED);
if (password != it->second.password)
auto on_wrong_password = [&]()
{
if (password.empty())
throw Exception("Password required for user " + name, ErrorCodes::REQUIRED_PASSWORD);
else
throw Exception("Wrong password for user " + name, ErrorCodes::WRONG_PASSWORD);
};
if (!it->second.password_sha256_hex.empty())
{
unsigned char hash[32];
SHA256_CTX ctx;
SHA256_Init(&ctx);
SHA256_Update(&ctx, reinterpret_cast<const unsigned char *>(password.data()), password.size());
SHA256_Final(hash, &ctx);
String hash_hex;
{
WriteBufferFromString buf(hash_hex);
HexWriteBuffer hex_buf(buf);
hex_buf.write(reinterpret_cast<const char *>(hash), sizeof(hash));
}
Poco::toLowerInPlace(hash_hex);
if (hash_hex != it->second.password_sha256_hex)
on_wrong_password();
}
else if (password != it->second.password)
{
on_wrong_password();
}
return it->second;

View File

@ -2,6 +2,7 @@
#include <DB/DataStreams/RemoteBlockOutputStream.h>
#include <DB/Common/escapeForFileName.h>
#include <DB/Common/setThreadName.h>
#include <DB/Storages/StorageDistributed.h>
#include <DB/IO/ReadBufferFromFile.h>
@ -86,6 +87,8 @@ public:
private:
void run()
{
setThreadName("DistrDirMonitor");
std::unique_lock<std::mutex> lock{mutex};
const auto quit_requested = [this] { return quit; };

View File

@ -8,11 +8,8 @@
#include <Poco/Mutex.h>
#include <Poco/RWLock.h>
#include <Poco/Event.h>
#include <Poco/SharedPtr.h>
#include <DB/Core/Types.h>
#include <DB/Core/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/IO/WriteHelpers.h>
#include <Yandex/logger_useful.h>
namespace DB
{
@ -63,22 +60,7 @@ public:
{
public:
/// Разбудить какой-нибудь поток.
void wake()
{
if (removed)
return;
std::unique_lock<std::mutex> lock(pool.mutex);
pool.tasks.splice(pool.tasks.begin(), pool.tasks, iterator);
/// Если эта задача в прошлый раз ничего не сделала, и ей было назначено спать, то отменим время сна.
time_t current_time = time(0);
if (next_time_to_execute > current_time)
next_time_to_execute = current_time;
/// Если все потоки сейчас выполняют работу, этот вызов никого не разбудит.
pool.wake_event.notify_one();
}
void wake();
private:
friend class BackgroundProcessingPool;
@ -99,77 +81,27 @@ public:
typedef std::shared_ptr<TaskInfo> TaskHandle;
BackgroundProcessingPool(int size_) : size(size_)
{
LOG_INFO(&Logger::get("BackgroundProcessingPool"), "Create BackgroundProcessingPool with " << size << " threads");
threads.resize(size);
for (auto & thread : threads)
thread = std::thread([this] { threadFunction(); });
}
BackgroundProcessingPool(int size_);
size_t getNumberOfThreads() const
{
return size;
}
int getCounter(const String & name)
{
std::unique_lock<std::mutex> lock(mutex);
return counters[name];
}
int getCounter(const String & name);
TaskHandle addTask(const Task & task)
{
TaskHandle res(new TaskInfo(*this, task));
TaskHandle addTask(const Task & task);
void removeTask(const TaskHandle & task);
{
std::unique_lock<std::mutex> lock(mutex);
res->iterator = tasks.insert(tasks.begin(), res);
}
wake_event.notify_all();
return res;
}
void removeTask(const TaskHandle & task)
{
task->removed = true;
/// Дождёмся завершения всех выполнений этой задачи.
{
Poco::ScopedWriteRWLock wlock(task->rwlock);
}
{
std::unique_lock<std::mutex> lock(mutex);
tasks.erase(task->iterator);
}
}
~BackgroundProcessingPool()
{
try
{
shutdown = true;
wake_event.notify_all();
for (std::thread & thread : threads)
thread.join();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
~BackgroundProcessingPool();
private:
typedef std::list<TaskHandle> Tasks;
typedef std::vector<std::thread> Threads;
const size_t size;
enum { sleep_seconds = 10 };
static constexpr double sleep_seconds = 10;
static constexpr double sleep_seconds_random_part = 1.0;
Tasks tasks; /// Задачи в порядке, в котором мы планируем их выполнять.
Counters counters;
@ -181,97 +113,7 @@ private:
std::condition_variable wake_event;
void threadFunction()
{
while (!shutdown)
{
Counters counters_diff;
bool has_exception = false;
try
{
TaskHandle task;
time_t min_time = std::numeric_limits<time_t>::max();
{
std::unique_lock<std::mutex> lock(mutex);
if (!tasks.empty())
{
/// O(n), n - число задач. По сути, количество таблиц. Обычно их мало.
for (const auto & handle : tasks)
{
time_t next_time_to_execute = handle->next_time_to_execute;
if (next_time_to_execute < min_time)
{
min_time = next_time_to_execute;
task = handle;
}
}
if (task) /// Переложим в конец очереди (уменьшим приоритет среди задач с одинаковым next_time_to_execute).
tasks.splice(tasks.end(), tasks, task->iterator);
}
}
if (shutdown)
break;
if (!task)
{
std::unique_lock<std::mutex> lock(mutex);
wake_event.wait_for(lock, std::chrono::duration<double>(sleep_seconds));
continue;
}
if (task->removed)
continue;
/// Лучшей задачи не нашлось, а эта задача в прошлый раз ничего не сделала, и поэтому ей назначено некоторое время спать.
time_t current_time = time(0);
if (min_time > current_time)
{
std::unique_lock<std::mutex> lock(mutex);
wake_event.wait_for(lock, std::chrono::duration<double>(min_time - current_time));
}
Poco::ScopedReadRWLock rlock(task->rwlock);
if (task->removed)
continue;
Context context(*this, counters_diff);
bool done_work = task->function(context);
/// Если задача сделала полезную работу, то она сможет выполняться в следующий раз хоть сразу.
/// Если нет - добавляем задержку перед повторным исполнением.
task->next_time_to_execute = time(0) + (done_work ? 0 : sleep_seconds);
}
catch (...)
{
has_exception = true;
tryLogCurrentException(__PRETTY_FUNCTION__);
}
/// Вычтем все счётчики обратно.
if (!counters_diff.empty())
{
std::unique_lock<std::mutex> lock(mutex);
for (const auto & it : counters_diff)
counters[it.first] -= it.second;
}
if (shutdown)
break;
if (has_exception)
{
std::unique_lock<std::mutex> lock(mutex);
wake_event.wait_for(lock, std::chrono::duration<double>(sleep_seconds));
}
}
}
void threadFunction();
};
typedef Poco::SharedPtr<BackgroundProcessingPool> BackgroundProcessingPoolPtr;

View File

@ -131,10 +131,8 @@ public:
}
/// Контрольная сумма от множества контрольных сумм .bin файлов.
String summaryDataChecksum() const
void summaryDataChecksum(SipHash & hash) const
{
SipHash hash;
/// Пользуемся тем, что итерирование в детерминированном (лексикографическом) порядке.
for (const auto & it : files)
{
@ -148,10 +146,6 @@ public:
hash.update(reinterpret_cast<const char *>(&sum.uncompressed_size), sizeof(sum.uncompressed_size));
hash.update(reinterpret_cast<const char *>(&sum.uncompressed_hash), sizeof(sum.uncompressed_hash));
}
UInt64 lo, hi;
hash.get128(lo, hi);
return DB::toString(lo) + "_" + DB::toString(hi);
}
String toString() const

View File

@ -82,12 +82,24 @@ public:
MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block, part_number);
String part_name = ActiveDataPartSet::getPartName(part->left_date, part->right_date, part->left, part->right, part->level);
/// Хэш от данных.
SipHash hash;
part->checksums.summaryDataChecksum(hash);
union
{
char bytes[16];
UInt64 lo, hi;
} hash_value;
hash.get128(hash_value.bytes);
String checksum(hash_value.bytes, 16);
/// Если в запросе не указан ID, возьмем в качестве ID хеш от данных. То есть, не вставляем одинаковые данные дважды.
/// NOTE: Если такая дедупликация не нужна, можно вместо этого оставлять block_id пустым.
/// Можно для этого сделать настройку или синтаксис в запросе (например, ID=null).
if (block_id.empty())
{
block_id = part->checksums.summaryDataChecksum();
block_id = toString(hash_value.lo) + "_" + toString(hash_value.hi);
if (block_id.empty())
throw Exception("Logical error: block_id is empty.", ErrorCodes::LOGICAL_ERROR);
@ -105,30 +117,25 @@ public:
/// Информация о блоке.
zkutil::Ops ops;
auto acl = zookeeper->getDefaultACL();
ops.push_back(
new zkutil::Op::Create(
storage.zookeeper_path + "/blocks/" + block_id,
"",
zookeeper->getDefaultACL(),
acl,
zkutil::CreateMode::Persistent));
ops.push_back(
new zkutil::Op::Create(
storage.zookeeper_path + "/blocks/" + block_id + "/columns",
part->columns.toString(),
zookeeper->getDefaultACL(),
zkutil::CreateMode::Persistent));
ops.push_back(
new zkutil::Op::Create(
storage.zookeeper_path + "/blocks/" + block_id + "/checksums",
part->checksums.toString(),
zookeeper->getDefaultACL(),
storage.zookeeper_path + "/blocks/" + block_id + "/checksum",
checksum,
acl,
zkutil::CreateMode::Persistent));
ops.push_back(
new zkutil::Op::Create(
storage.zookeeper_path + "/blocks/" + block_id + "/number",
toString(part_number),
zookeeper->getDefaultACL(),
acl,
zkutil::CreateMode::Persistent));
/// Информация о куске, в данных реплики.
@ -138,7 +145,7 @@ public:
ops.push_back(new zkutil::Op::Create(
storage.zookeeper_path + "/log/log-",
log_entry.toString(),
zookeeper->getDefaultACL(),
acl,
zkutil::CreateMode::PersistentSequential));
/// Удаление информации о том, что номер блока используется для записи.
@ -174,7 +181,7 @@ public:
new zkutil::Op::Create(
quorum_status_path,
quorum_entry.toString(),
zookeeper->getDefaultACL(),
acl,
zkutil::CreateMode::Persistent));
}
@ -192,16 +199,20 @@ public:
else if (code == ZNODEEXISTS)
{
/// Если блок с таким ID уже есть в таблице, откатим его вставку.
String expected_checksums_str;
String expected_checksum;
if (!block_id.empty() && zookeeper->tryGet(
storage.zookeeper_path + "/blocks/" + block_id + "/checksums", expected_checksums_str))
storage.zookeeper_path + "/blocks/" + block_id + "/checksum", expected_checksum))
{
LOG_INFO(log, "Block with ID " << block_id << " already exists; ignoring it (removing part " << part->name << ")");
auto expected_checksums = MergeTreeData::DataPart::Checksums::parse(expected_checksums_str);
/// Если данные отличались от тех, что были вставлены ранее с тем же ID, бросим исключение.
expected_checksums.checkEqual(part->checksums, true);
if (expected_checksum != checksum)
{
if (!insert_id.empty())
throw Exception("Attempt to insert block with same ID but different checksum", ErrorCodes::CHECKSUM_DOESNT_MATCH);
else
throw Exception("Logical error: got ZNODEEXISTS while inserting data, block ID is derived from checksum but checksum doesn't match", ErrorCodes::LOGICAL_ERROR);
}
transaction.rollback();
}

View File

@ -3,6 +3,7 @@
#include <DB/Core/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/Core/Types.h>
#include <DB/IO/WriteHelpers.h>
#include <mutex>
#include <condition_variable>
@ -19,7 +20,10 @@ class WriteBuffer;
class StorageReplicatedMergeTree;
/// Добавляет кусок в множество future_parts; в деструкторе убирает.
/** Добавляет кусок в множество future_parts; в деструкторе убирает.
* future_parts - множество кусков, которые будут созданы после выполнения
* выполняющихся в данный момент элементов очереди.
*/
struct FuturePartTagger
{
String part;
@ -32,22 +36,34 @@ struct FuturePartTagger
typedef Poco::SharedPtr<FuturePartTagger> FuturePartTaggerPtr;
/// Запись о том, что нужно сделать.
struct ReplicatedMergeTreeLogEntry
/// Запись о том, что нужно сделать. Только данные (их можно копировать).
struct ReplicatedMergeTreeLogEntryData
{
typedef Poco::SharedPtr<ReplicatedMergeTreeLogEntry> Ptr;
enum Type
{
EMPTY, /// Не используется.
GET_PART, /// Получить кусок с другой реплики.
MERGE_PARTS, /// Слить куски.
DROP_RANGE, /// Удалить куски в указанном месяце в указанном диапазоне номеров.
ATTACH_PART, /// Перенести кусок из директории detached или unreplicated.
};
String typeToString() const
{
switch (type)
{
case ReplicatedMergeTreeLogEntryData::GET_PART: return "GET_PART";
case ReplicatedMergeTreeLogEntryData::MERGE_PARTS: return "MERGE_PARTS";
case ReplicatedMergeTreeLogEntryData::DROP_RANGE: return "DROP_RANGE";
case ReplicatedMergeTreeLogEntryData::ATTACH_PART: return "ATTACH_PART";
default:
throw Exception("Unknown log entry type: " + DB::toString(type), ErrorCodes::LOGICAL_ERROR);
}
}
String znode_name;
Type type;
Type type = EMPTY;
String source_replica; /// Пустая строка значит, что эта запись была добавлена сразу в очередь, а не скопирована из лога.
/// Имя куска, получающегося в результате.
@ -62,9 +78,7 @@ struct ReplicatedMergeTreeLogEntry
/// Для ATTACH_PART имя куска в директории detached или unreplicated.
String source_part_name;
/// Нужно переносить из директории unreplicated, а не detached.
bool attach_unreplicated;
FuturePartTaggerPtr future_part_tagger;
bool attach_unreplicated = false;
/// Доступ под queue_mutex.
bool currently_executing = false; /// Выполняется ли действие сейчас.
@ -73,16 +87,24 @@ struct ReplicatedMergeTreeLogEntry
size_t num_tries = 0; /// Количество попыток выполнить действие (с момента старта сервера; включая выполняющееся).
ExceptionPtr exception; /// Последний эксепшен, в случае безуспешной попытки выполнить действие.
time_t last_attempt_time = 0; /// Время начала последней попытки выполнить действие.
String last_action; /// Что делается сейчас или делалось в последний раз.
size_t num_postponed = 0; /// Количество раз, когда действие было отложено.
String postpone_reason; /// Причина, по которой действие было отложено, если оно отложено.
std::condition_variable execution_complete; /// Пробуждается когда currently_executing становится false.
time_t last_postpone_time = 0; /// Время последнего раза, когда действие было отложено.
/// Время создания или время копирования из общего лога в очередь конкретной реплики.
time_t create_time = 0;
/// Величина кворума (для GET_PART) - ненулевое значение при включенной кворумной записи.
size_t quorum = 0;
};
struct ReplicatedMergeTreeLogEntry : ReplicatedMergeTreeLogEntryData
{
typedef Poco::SharedPtr<ReplicatedMergeTreeLogEntry> Ptr;
FuturePartTaggerPtr future_part_tagger;
std::condition_variable execution_complete; /// Пробуждается когда currently_executing становится false.
void addResultToVirtualParts(StorageReplicatedMergeTree & storage);
void tagPartAsFuture(StorageReplicatedMergeTree & storage);

View File

@ -177,6 +177,9 @@ public:
/// Получить статус таблицы. Если with_zk_fields = false - не заполнять поля, требующие запросов в ZK.
void getStatus(Status & res, bool with_zk_fields = true);
using LogEntriesData = std::vector<ReplicatedMergeTreeLogEntryData>;
void getQueue(LogEntriesData & res, String & replica_name);
private:
void dropUnreplicatedPartition(const Field & partition, bool detach, const Settings & settings);
@ -383,7 +386,7 @@ private:
/** Можно ли сейчас попробовать выполнить это действие. Если нет, нужно оставить его в очереди и попробовать выполнить другое.
* Вызывается под queue_mutex.
*/
bool shouldExecuteLogEntry(const LogEntry & entry);
bool shouldExecuteLogEntry(const LogEntry & entry, String & out_postpone_reason);
/** Выполнить действие из очереди. Бросает исключение, если что-то не так.
* Возвращает, получилось ли выполнить. Если не получилось, запись нужно положить в конец очереди.

View File

@ -0,0 +1,43 @@
#pragma once
#include <Poco/SharedPtr.h>
#include <DB/Storages/IStorage.h>
#include <DB/Interpreters/Context.h>
namespace DB
{
using Poco::SharedPtr;
/** Реализует системную таблицу replication_queue, которая позволяет посмотреть очереди репликации для реплицируемых таблиц.
*/
class StorageSystemReplicationQueue : public IStorage
{
public:
static StoragePtr create(const std::string & name_);
std::string getName() const override { return "SystemReplicationQueue"; }
std::string getTableName() const override { return name; }
const NamesAndTypesList & getColumnsListImpl() const override { return columns; }
BlockInputStreams read(
const Names & column_names,
ASTPtr query,
const Context & context,
const Settings & settings,
QueryProcessingStage::Enum & processed_stage,
size_t max_block_size = DEFAULT_BLOCK_SIZE,
unsigned threads = 1) override;
private:
const std::string name;
NamesAndTypesList columns;
StorageSystemReplicationQueue(const std::string & name_);
};
}

View File

@ -0,0 +1,11 @@
#include <DB/AggregateFunctions/AggregateFunctionArray.h>
namespace DB
{
AggregateFunctionPtr createAggregateFunctionArray(AggregateFunctionPtr & nested)
{
return new AggregateFunctionArray(nested);
}
}

View File

@ -0,0 +1,31 @@
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
#include <DB/AggregateFunctions/AggregateFunctionAvg.h>
#include <DB/AggregateFunctions/Helpers.h>
namespace DB
{
namespace
{
AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionAvg>(*argument_types[0]);
if (!res)
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
}
}
void registerAggregateFunctionAvg(AggregateFunctionFactory & factory)
{
factory.registerFunction({"avg"}, createAggregateFunctionAvg);
}
}

View File

@ -0,0 +1,22 @@
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
#include <DB/AggregateFunctions/AggregateFunctionCount.h>
namespace DB
{
namespace
{
AggregateFunctionPtr createAggregateFunctionCount(const std::string & name, const DataTypes & argument_types)
{
return new AggregateFunctionCount;
}
}
void registerAggregateFunctionCount(AggregateFunctionFactory & factory)
{
factory.registerFunction({"count"}, createAggregateFunctionCount);
}
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,23 @@
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
#include <DB/AggregateFunctions/AggregateFunctionGroupArray.h>
#include <DB/AggregateFunctions/Helpers.h>
namespace DB
{
namespace
{
AggregateFunctionPtr createAggregateFunctionGroupArray(const std::string & name, const DataTypes & argument_types)
{
return new AggregateFunctionGroupArray;
}
}
void registerAggregateFunctionGroupArray(AggregateFunctionFactory & factory)
{
factory.registerFunction({"groupArray"}, createAggregateFunctionGroupArray);
}
}

View File

@ -0,0 +1,40 @@
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
#include <DB/AggregateFunctions/AggregateFunctionGroupUniqArray.h>
#include <DB/AggregateFunctions/Helpers.h>
namespace DB
{
namespace
{
AggregateFunctionPtr createAggregateFunctionGroupUniqArray(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const DataTypeArray * arr = typeid_cast<const DataTypeArray *>(&*argument_types[0]);
AggregateFunctionPtr res;
if (!arr)
res = createWithNumericType<AggregateFunctionGroupUniqArray>(*argument_types[0]);
else
res = createWithNumericType<AggregateFunctionGroupUniqArrays>(*arr->getNestedType());
if (!res)
throw Exception("Illegal type " + argument_types[0]->getName() +
" of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
}
}
void registerAggregateFunctionGroupUniqArray(AggregateFunctionFactory & factory)
{
factory.registerFunction({"groupUniqArray"}, createAggregateFunctionGroupUniqArray);
}
}

View File

@ -0,0 +1,11 @@
#include <DB/AggregateFunctions/AggregateFunctionIf.h>
namespace DB
{
AggregateFunctionPtr createAggregateFunctionIf(AggregateFunctionPtr & nested)
{
return new AggregateFunctionIf(nested);
}
}

View File

@ -0,0 +1,11 @@
#include <DB/AggregateFunctions/AggregateFunctionMerge.h>
namespace DB
{
AggregateFunctionPtr createAggregateFunctionMerge(AggregateFunctionPtr & nested)
{
return new AggregateFunctionMerge(nested);
}
}

View File

@ -0,0 +1,11 @@
#include <DB/AggregateFunctions/AggregateFunctionState.h>
namespace DB
{
AggregateFunctionPtr createAggregateFunctionState(AggregateFunctionPtr & nested)
{
return new AggregateFunctionState(nested);
}
}

View File

@ -0,0 +1,31 @@
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
#include <DB/AggregateFunctions/AggregateFunctionSum.h>
#include <DB/AggregateFunctions/Helpers.h>
namespace DB
{
namespace
{
AggregateFunctionPtr createAggregateFunctionSum(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionSum>(*argument_types[0]);
if (!res)
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
}
}
void registerAggregateFunctionSum(AggregateFunctionFactory & factory)
{
factory.registerFunction({"sum"}, createAggregateFunctionSum);
}
}

View File

@ -0,0 +1,39 @@
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
#include <DB/AggregateFunctions/Helpers.h>
#include <DB/AggregateFunctions/AggregateFunctionUniqUpTo.h>
namespace DB
{
namespace
{
AggregateFunctionPtr createAggregateFunctionUniqUpTo(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const IDataType & argument_type = *argument_types[0];
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniqUpTo>(*argument_types[0]);
if (res)
return res;
else if (typeid_cast<const DataTypeDate *>(&argument_type))
return new AggregateFunctionUniqUpTo<DataTypeDate::FieldType>;
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
return new AggregateFunctionUniqUpTo<DataTypeDateTime::FieldType>;
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
return new AggregateFunctionUniqUpTo<String>;
else
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
}
void registerAggregateFunctionUniqUpTo(AggregateFunctionFactory & factory)
{
factory.registerFunction({"uniqUpTo"}, createAggregateFunctionUniqUpTo);
}
}

View File

@ -0,0 +1,52 @@
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
#include <DB/AggregateFunctions/HelpersMinMaxAny.h>
namespace DB
{
namespace
{
AggregateFunctionPtr createAggregateFunctionAny(const std::string & name, const DataTypes & argument_types)
{
return createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionAnyData>(name, argument_types);
}
AggregateFunctionPtr createAggregateFunctionAnyLast(const std::string & name, const DataTypes & argument_types)
{
return createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionAnyLastData>(name, argument_types);
}
AggregateFunctionPtr createAggregateFunctionMin(const std::string & name, const DataTypes & argument_types)
{
return createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionMinData>(name, argument_types);
}
AggregateFunctionPtr createAggregateFunctionMax(const std::string & name, const DataTypes & argument_types)
{
return createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionMaxData>(name, argument_types);
}
AggregateFunctionPtr createAggregateFunctionArgMin(const std::string & name, const DataTypes & argument_types)
{
return createAggregateFunctionArgMinMax<AggregateFunctionMinData>(name, argument_types);
}
AggregateFunctionPtr createAggregateFunctionArgMax(const std::string & name, const DataTypes & argument_types)
{
return createAggregateFunctionArgMinMax<AggregateFunctionMaxData>(name, argument_types);
}
}
void registerAggregateFunctionsMinMaxAny(AggregateFunctionFactory & factory)
{
factory.registerFunction({"any"}, createAggregateFunctionAny);
factory.registerFunction({"anyLast"}, createAggregateFunctionAnyLast);
factory.registerFunction({"min"}, createAggregateFunctionMin);
factory.registerFunction({"max"}, createAggregateFunctionMax);
factory.registerFunction({"argMin"}, createAggregateFunctionArgMin);
factory.registerFunction({"argMax"}, createAggregateFunctionArgMax);
}
}

View File

@ -0,0 +1,66 @@
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
#include <DB/AggregateFunctions/Helpers.h>
#include <DB/AggregateFunctions/AggregateFunctionQuantile.h>
namespace DB
{
namespace
{
AggregateFunctionPtr createAggregateFunctionQuantile(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const IDataType & argument_type = *argument_types[0];
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionQuantile<UInt8>;
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return new AggregateFunctionQuantile<UInt16>;
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return new AggregateFunctionQuantile<UInt32>;
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return new AggregateFunctionQuantile<UInt64>;
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return new AggregateFunctionQuantile<Int8>;
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return new AggregateFunctionQuantile<Int16>;
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return new AggregateFunctionQuantile<Int32>;
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return new AggregateFunctionQuantile<Int64>;
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return new AggregateFunctionQuantile<Float32>;
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return new AggregateFunctionQuantile<Float64>;
else if (typeid_cast<const DataTypeDate *>(&argument_type)) return new AggregateFunctionQuantile<DataTypeDate::FieldType, false>;
else if (typeid_cast<const DataTypeDateTime*>(&argument_type)) return new AggregateFunctionQuantile<DataTypeDateTime::FieldType, false>;
else
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
AggregateFunctionPtr createAggregateFunctionQuantiles(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const IDataType & argument_type = *argument_types[0];
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionQuantiles<UInt8>;
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return new AggregateFunctionQuantiles<UInt16>;
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return new AggregateFunctionQuantiles<UInt32>;
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return new AggregateFunctionQuantiles<UInt64>;
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return new AggregateFunctionQuantiles<Int8>;
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return new AggregateFunctionQuantiles<Int16>;
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return new AggregateFunctionQuantiles<Int32>;
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return new AggregateFunctionQuantiles<Int64>;
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return new AggregateFunctionQuantiles<Float32>;
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return new AggregateFunctionQuantiles<Float64>;
else if (typeid_cast<const DataTypeDate *>(&argument_type)) return new AggregateFunctionQuantiles<DataTypeDate::FieldType, false>;
else if (typeid_cast<const DataTypeDateTime*>(&argument_type)) return new AggregateFunctionQuantiles<DataTypeDateTime::FieldType, false>;
else
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
}
void registerAggregateFunctionsQuantile(AggregateFunctionFactory & factory)
{
factory.registerFunction({"quantile", "median"}, createAggregateFunctionQuantile);
factory.registerFunction({"quantiles"}, createAggregateFunctionQuantiles);
}
}

View File

@ -0,0 +1,92 @@
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
#include <DB/AggregateFunctions/Helpers.h>
#include <DB/AggregateFunctions/AggregateFunctionQuantileDeterministic.h>
namespace DB
{
namespace
{
AggregateFunctionPtr createAggregateFunctionQuantileDeterministic(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 2)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const auto determinator_type = argument_types[1].get();
if (!typeid_cast<const DataTypeInt32 *>(determinator_type) &&
!typeid_cast<const DataTypeUInt32 *>(determinator_type) &&
!typeid_cast<const DataTypeInt64 *>(determinator_type) &&
!typeid_cast<const DataTypeUInt64 *>(determinator_type))
{
throw Exception{
"Illegal type " + determinator_type->getName() + " of second argument for aggregate function " + name +
", Int32, UInt32, Int64 or UInt64 required",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
}
const IDataType & argument_type = *argument_types[0];
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<UInt8>;
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<UInt16>;
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<UInt32>;
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<UInt64>;
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<Int8>;
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<Int16>;
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<Int32>;
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<Int64>;
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<Float32>;
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<Float64>;
else if (typeid_cast<const DataTypeDate *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<DataTypeDate::FieldType, false>;
else if (typeid_cast<const DataTypeDateTime*>(&argument_type)) return new AggregateFunctionQuantileDeterministic<DataTypeDateTime::FieldType, false>;
else
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
AggregateFunctionPtr createAggregateFunctionQuantilesDeterministic(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 2)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const auto determinator_type = argument_types[1].get();
if (!typeid_cast<const DataTypeInt32 *>(determinator_type) &&
!typeid_cast<const DataTypeUInt32 *>(determinator_type) &&
!typeid_cast<const DataTypeInt64 *>(determinator_type) &&
!typeid_cast<const DataTypeUInt64 *>(determinator_type))
{
throw Exception{
"Illegal type " + determinator_type->getName() + " of second argument for aggregate function " + name +
", Int32, UInt32, Int64 or UInt64 required",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
}
const IDataType & argument_type = *argument_types[0];
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<UInt8>;
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<UInt16>;
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<UInt32>;
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<UInt64>;
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<Int8>;
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<Int16>;
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<Int32>;
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<Int64>;
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<Float32>;
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<Float64>;
else if (typeid_cast<const DataTypeDate *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<DataTypeDate::FieldType, false>;
else if (typeid_cast<const DataTypeDateTime*>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<DataTypeDateTime::FieldType, false>;
else
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
}
void registerAggregateFunctionsQuantileDeterministic(AggregateFunctionFactory & factory)
{
factory.registerFunction({"quantileDeterministic", "medianDeterministic"}, createAggregateFunctionQuantileDeterministic);
factory.registerFunction({"quantilesDeterministic"}, createAggregateFunctionQuantilesDeterministic);
}
}

View File

@ -0,0 +1,76 @@
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
#include <DB/AggregateFunctions/Helpers.h>
#include <DB/AggregateFunctions/AggregateFunctionQuantileTiming.h>
namespace DB
{
namespace
{
AggregateFunctionPtr createAggregateFunctionQuantileTiming(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionQuantileTiming>(*argument_types[0]);
if (!res)
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
}
AggregateFunctionPtr createAggregateFunctionQuantilesTiming(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionQuantilesTiming>(*argument_types[0]);
if (!res)
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
}
AggregateFunctionPtr createAggregateFunctionQuantileTimingWeighted(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 2)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithTwoNumericTypes<AggregateFunctionQuantileTimingWeighted>(*argument_types[0], *argument_types[1]);
if (!res)
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()
+ " of arguments for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
}
AggregateFunctionPtr createAggregateFunctionQuantilesTimingWeighted(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 2)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithTwoNumericTypes<AggregateFunctionQuantilesTimingWeighted>(*argument_types[0], *argument_types[1]);
if (!res)
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()
+ " of arguments for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
}
}
void registerAggregateFunctionsQuantileTiming(AggregateFunctionFactory & factory)
{
factory.registerFunction({"quantileTiming", "medianTiming"}, createAggregateFunctionQuantileTiming);
factory.registerFunction({"quantilesTiming"}, createAggregateFunctionQuantilesTiming);
factory.registerFunction({"quantileTimingWeighted", "medianTimingWeighted"}, createAggregateFunctionQuantileTimingWeighted);
factory.registerFunction({"quantilesTimingWeighted"}, createAggregateFunctionQuantilesTimingWeighted);
}
}

View File

@ -0,0 +1,34 @@
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
#include <DB/AggregateFunctions/AggregateFunctionSequenceMatch.h>
namespace DB
{
namespace
{
AggregateFunctionPtr createAggregateFunctionSequenceCount(const std::string & name, const DataTypes & argument_types)
{
if (!AggregateFunctionSequenceCount::sufficientArgs(argument_types.size()))
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return new AggregateFunctionSequenceCount;
}
AggregateFunctionPtr createAggregateFunctionSequenceMatch(const std::string & name, const DataTypes & argument_types)
{
if (!AggregateFunctionSequenceMatch::sufficientArgs(argument_types.size()))
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return new AggregateFunctionSequenceMatch;
}
}
void registerAggregateFunctionsSequenceMatch(AggregateFunctionFactory & factory)
{
factory.registerFunction({"sequenceMatch"}, createAggregateFunctionSequenceMatch);
factory.registerFunction({"sequenceCount"}, createAggregateFunctionSequenceCount);
}
}

View File

@ -0,0 +1,116 @@
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
#include <DB/AggregateFunctions/Helpers.h>
#include <DB/AggregateFunctions/AggregateFunctionsStatistics.h>
namespace DB
{
namespace
{
AggregateFunctionPtr createAggregateFunctionVarPop(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionVarPop>(*argument_types[0]);
if (!res)
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
}
AggregateFunctionPtr createAggregateFunctionVarSamp(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionVarSamp>(*argument_types[0]);
if (!res)
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
}
AggregateFunctionPtr createAggregateFunctionStdDevPop(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionStdDevPop>(*argument_types[0]);
if (!res)
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
}
AggregateFunctionPtr createAggregateFunctionStdDevSamp(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionStdDevSamp>(*argument_types[0]);
if (!res)
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
}
AggregateFunctionPtr createAggregateFunctionCovarPop(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 2)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithTwoNumericTypes<AggregateFunctionCovarPop>(*argument_types[0], *argument_types[1]);
if (!res)
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()
+ " of arguments for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
}
AggregateFunctionPtr createAggregateFunctionCovarSamp(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 2)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithTwoNumericTypes<AggregateFunctionCovarSamp>(*argument_types[0], *argument_types[1]);
if (!res)
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()
+ " of arguments for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
}
AggregateFunctionPtr createAggregateFunctionCorr(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 2)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithTwoNumericTypes<AggregateFunctionCorr>(*argument_types[0], *argument_types[1]);
if (!res)
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()
+ " of arguments for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
}
}
void registerAggregateFunctionsStatistics(AggregateFunctionFactory & factory)
{
factory.registerFunction({"varSamp"}, createAggregateFunctionVarSamp);
factory.registerFunction({"varPop"}, createAggregateFunctionVarPop);
factory.registerFunction({"stddevSamp"}, createAggregateFunctionStdDevSamp);
factory.registerFunction({"stddevPop"}, createAggregateFunctionStdDevPop);
factory.registerFunction({"covarSamp"}, createAggregateFunctionCovarSamp);
factory.registerFunction({"covarPop"}, createAggregateFunctionCovarPop);
factory.registerFunction({"corr"}, createAggregateFunctionCorr);
}
}

View File

@ -0,0 +1,191 @@
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
#include <DB/AggregateFunctions/AggregateFunctionUniq.h>
#include <DB/AggregateFunctions/Helpers.h>
namespace DB
{
namespace
{
AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const IDataType & argument_type = *argument_types[0];
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq, AggregateFunctionUniqUniquesHashSetData>(*argument_types[0]);
if (res)
return res;
else if (typeid_cast<const DataTypeDate *>(&argument_type))
return new AggregateFunctionUniq<DataTypeDate::FieldType, AggregateFunctionUniqUniquesHashSetData>;
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
return new AggregateFunctionUniq<DataTypeDateTime::FieldType, AggregateFunctionUniqUniquesHashSetData>;
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
return new AggregateFunctionUniq<String, AggregateFunctionUniqUniquesHashSetData>;
else
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
AggregateFunctionPtr createAggregateFunctionUniqExact(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const IDataType & argument_type = *argument_types[0];
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq, AggregateFunctionUniqUniquesHashSetData>(*argument_types[0]);
if (res)
return res;
else if (typeid_cast<const DataTypeDate *>(&argument_type))
return new AggregateFunctionUniq<DataTypeDate::FieldType, AggregateFunctionUniqUniquesHashSetData>;
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
return new AggregateFunctionUniq<DataTypeDateTime::FieldType, AggregateFunctionUniqUniquesHashSetData>;
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
return new AggregateFunctionUniq<String, AggregateFunctionUniqUniquesHashSetData>;
else
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
AggregateFunctionPtr createAggregateFunctionUniqHLL12(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const IDataType & argument_type = *argument_types[0];
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq, AggregateFunctionUniqHLL12Data>(*argument_types[0]);
if (res)
return res;
else if (typeid_cast<const DataTypeDate *>(&argument_type))
return new AggregateFunctionUniq<DataTypeDate::FieldType, AggregateFunctionUniqHLL12Data<DataTypeDate::FieldType>>;
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
return new AggregateFunctionUniq<DataTypeDateTime::FieldType, AggregateFunctionUniqHLL12Data<DataTypeDateTime::FieldType>>;
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
return new AggregateFunctionUniq<String, AggregateFunctionUniqHLL12Data<String>>;
else
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
AggregateFunctionPtr createAggregateFunctionUniqCombined(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const IDataType & argument_type = *argument_types[0];
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq, AggregateFunctionUniqCombinedData>(*argument_types[0]);
if (res)
return res;
else if (typeid_cast<const DataTypeDate *>(&argument_type))
return new AggregateFunctionUniq<DataTypeDate::FieldType, AggregateFunctionUniqCombinedData<DataTypeDate::FieldType>>;
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
return new AggregateFunctionUniq<DataTypeDateTime::FieldType, AggregateFunctionUniqCombinedData<DataTypeDateTime::FieldType>>;
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
return new AggregateFunctionUniq<String, AggregateFunctionUniqCombinedData<String>>;
else
throw Exception("Illegal type " + argument_types[0]->getName() +
" of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
AggregateFunctionPtr createAggregateFunctionUniqCombinedRaw(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const IDataType & argument_type = *argument_types[0];
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq, AggregateFunctionUniqCombinedRawData>(*argument_types[0]);
if (res)
return res;
else if (typeid_cast<const DataTypeDate *>(&argument_type))
return new AggregateFunctionUniq<DataTypeDate::FieldType, AggregateFunctionUniqCombinedRawData<DataTypeDate::FieldType>>;
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
return new AggregateFunctionUniq<DataTypeDateTime::FieldType, AggregateFunctionUniqCombinedRawData<DataTypeDateTime::FieldType>>;
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
return new AggregateFunctionUniq<String, AggregateFunctionUniqCombinedRawData<String>>;
else
throw Exception("Illegal type " + argument_types[0]->getName() +
" of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
AggregateFunctionPtr createAggregateFunctionUniqCombinedLinearCounting(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const IDataType & argument_type = *argument_types[0];
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq,
AggregateFunctionUniqCombinedLinearCountingData>(*argument_types[0]);
if (res)
return res;
else if (typeid_cast<const DataTypeDate *>(&argument_type))
return new AggregateFunctionUniq<DataTypeDate::FieldType,
AggregateFunctionUniqCombinedLinearCountingData<DataTypeDate::FieldType>>;
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
return new AggregateFunctionUniq<DataTypeDateTime::FieldType,
AggregateFunctionUniqCombinedLinearCountingData<DataTypeDateTime::FieldType>>;
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
return new AggregateFunctionUniq<String, AggregateFunctionUniqCombinedLinearCountingData<String>>;
else
throw Exception("Illegal type " + argument_types[0]->getName() +
" of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
AggregateFunctionPtr createAggregateFunctionUniqCombinedBiasCorrected(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const IDataType & argument_type = *argument_types[0];
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq,
AggregateFunctionUniqCombinedBiasCorrectedData>(*argument_types[0]);
if (res)
return res;
else if (typeid_cast<const DataTypeDate *>(&argument_type))
return new AggregateFunctionUniq<DataTypeDate::FieldType,
AggregateFunctionUniqCombinedBiasCorrectedData<DataTypeDate::FieldType>>;
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
return new AggregateFunctionUniq<DataTypeDateTime::FieldType,
AggregateFunctionUniqCombinedBiasCorrectedData<DataTypeDateTime::FieldType>>;
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
return new AggregateFunctionUniq<String, AggregateFunctionUniqCombinedBiasCorrectedData<String>>;
else
throw Exception("Illegal type " + argument_types[0]->getName() +
" of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
}
void registerAggregateFunctionsUniq(AggregateFunctionFactory & factory)
{
factory.registerFunction({"uniq"}, createAggregateFunctionUniq);
factory.registerFunction({"uniqHLL12"}, createAggregateFunctionUniqHLL12);
factory.registerFunction({"uniqExact"}, createAggregateFunctionUniqExact);
factory.registerFunction({"uniqCombinedRaw"}, createAggregateFunctionUniqCombinedRaw);
factory.registerFunction({"uniqCombinedLinearCounting"}, createAggregateFunctionUniqCombinedLinearCounting);
factory.registerFunction({"uniqCombinedBiasCorrected"}, createAggregateFunctionUniqCombinedBiasCorrected);
factory.registerFunction({"uniqCombined"}, createAggregateFunctionUniqCombined);
}
}

View File

@ -0,0 +1,10 @@
#include <sys/prctl.h>
#include <DB/Core/Exception.h>
#include <DB/Common/setThreadName.h>
void setThreadName(const char * name)
{
if (0 != prctl(PR_SET_NAME, name, 0, 0, 0))
DB::throwFromErrno("Cannot set thread name with prctl(PR_SET_NAME...)");
}

View File

@ -6,7 +6,7 @@
#include <DB/IO/WriteBufferFromOStream.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Storages/System/StorageSystemNumbers.h>
#include <DB/DataStreams/LimitBlockInputStream.h>
#include <DB/DataStreams/ExpressionBlockInputStream.h>

View File

@ -6,7 +6,7 @@
#include <DB/IO/WriteBufferFromOStream.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Storages/System/StorageSystemNumbers.h>
#include <DB/DataStreams/LimitBlockInputStream.h>
#include <DB/DataStreams/ExpressionBlockInputStream.h>

View File

@ -7,7 +7,7 @@
#include <DB/IO/WriteBufferFromOStream.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Storages/System/StorageSystemNumbers.h>
#include <DB/DataStreams/LimitBlockInputStream.h>
#include <DB/DataStreams/ExpressionBlockInputStream.h>

View File

@ -6,7 +6,7 @@
#include <DB/IO/WriteBufferFromFileDescriptor.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Storages/System/StorageSystemNumbers.h>
#include <DB/DataStreams/LimitBlockInputStream.h>
#include <DB/DataStreams/UnionBlockInputStream.h>

View File

@ -6,7 +6,7 @@
#include <DB/IO/WriteBufferFromFileDescriptor.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Storages/System/StorageSystemNumbers.h>
#include <DB/DataStreams/LimitBlockInputStream.h>
#include <DB/DataStreams/UnionBlockInputStream.h>
@ -32,7 +32,7 @@ int main(int argc, char ** argv)
context.setPath("./");
DB::loadMetadata(context);
DB::Names column_names;
column_names.push_back("WatchID");
@ -43,7 +43,7 @@ int main(int argc, char ** argv)
for (size_t i = 0, size = streams.size(); i < size; ++i)
streams[i] = new DB::AsynchronousBlockInputStream(streams[i]);
DB::BlockInputStreamPtr stream = new DB::UnionBlockInputStream(streams, nullptr, settings.max_threads);
stream = new DB::LimitBlockInputStream(stream, 10, 0);

View File

@ -13,8 +13,7 @@ void registerFunctionsStringSearch(FunctionFactory & factory)
factory.registerFunction<FunctionPosition>();
factory.registerFunction<FunctionPositionUTF8>();
factory.registerFunction<FunctionPositionCaseInsensitive>();
/// @todo implement
// factory.registerFunction<FunctionPositionCaseInsensitiveUTF8>();
factory.registerFunction<FunctionPositionCaseInsensitiveUTF8>();
factory.registerFunction<FunctionMatch>();
factory.registerFunction<FunctionLike>();
factory.registerFunction<FunctionNotLike>();

View File

@ -826,6 +826,11 @@ std::pair<String, UInt16> Context::getInterserverIOAddress() const
return { shared->interserver_io_host, shared->interserver_io_port };
}
UInt16 Context::getTCPPort() const
{
return Poco::Util::Application::instance().config().getInt("tcp_port");
}
void Context::initClusters()
{

View File

@ -18,7 +18,7 @@
#include <DB/Storages/StorageFactory.h>
#include <DB/Storages/StorageLog.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Storages/System/StorageSystemNumbers.h>
#include <DB/Parsers/formatAST.h>
#include <DB/Parsers/ASTIdentifier.h>

View File

@ -15,6 +15,7 @@
#include <DB/Interpreters/InterpreterCreateQuery.h>
#include <DB/Interpreters/InterpreterRenameQuery.h>
#include <DB/Interpreters/QueryLog.h>
#include <DB/Common/setThreadName.h>
namespace DB
@ -116,6 +117,8 @@ QueryLog::~QueryLog()
void QueryLog::threadFunction()
{
setThreadName("QueryLogFlush");
Stopwatch time_after_last_write;
bool first = true;

View File

@ -190,7 +190,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
setClientInfo(elem, context);
bool log_queries = settings.log_queries;
bool log_queries = settings.log_queries && !internal;
/// Логгируем в таблицу начало выполнения запроса, если нужно.
if (log_queries)

View File

@ -13,6 +13,7 @@
#include <DB/IO/ReadBufferFromFile.h>
#include <DB/IO/WriteBufferFromString.h>
#include <DB/IO/copyData.h>
#include <DB/Common/escapeForFileName.h>
#include <statdaemons/Stopwatch.h>
@ -75,9 +76,10 @@ void loadMetadata(Context & context)
if (it.name().at(0) == '.')
continue;
LOG_INFO(log, "Loading database " << it.name());
String database = unescapeForFileName(it.name());
executeCreateQuery("ATTACH DATABASE " + it.name(), context, it.name(), it->path());
LOG_INFO(log, "Loading database " << database);
executeCreateQuery("ATTACH DATABASE " + backQuoteIfNeed(database), context, database, it->path());
/// Цикл по таблицам
typedef std::vector<std::string> Tables;
@ -135,7 +137,7 @@ void loadMetadata(Context & context)
try
{
executeCreateQuery(s, context, it.name(), tables[j]);
executeCreateQuery(s, context, database, tables[j]);
}
catch (const Exception & e)
{

View File

@ -9,8 +9,8 @@
#include <DB/IO/WriteBufferFromOStream.h>
#include <DB/Storages/StorageLog.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Storages/StorageSystemOne.h>
#include <DB/Storages/System/StorageSystemNumbers.h>
#include <DB/Storages/System/StorageSystemOne.h>
#include <DB/Storages/StorageFactory.h>
#include <DB/Interpreters/loadMetadata.h>

View File

@ -19,23 +19,25 @@
#include <DB/Common/Macros.h>
#include <DB/Common/getFQDNOrHostName.h>
#include <DB/Common/setThreadName.h>
#include <DB/Interpreters/loadMetadata.h>
#include <DB/Interpreters/ProcessList.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Storages/StorageSystemTables.h>
#include <DB/Storages/StorageSystemParts.h>
#include <DB/Storages/StorageSystemDatabases.h>
#include <DB/Storages/StorageSystemProcesses.h>
#include <DB/Storages/StorageSystemEvents.h>
#include <DB/Storages/StorageSystemOne.h>
#include <DB/Storages/StorageSystemMerges.h>
#include <DB/Storages/StorageSystemSettings.h>
#include <DB/Storages/StorageSystemZooKeeper.h>
#include <DB/Storages/StorageSystemReplicas.h>
#include <DB/Storages/StorageSystemDictionaries.h>
#include <DB/Storages/StorageSystemColumns.h>
#include <DB/Storages/StorageSystemFunctions.h>
#include <DB/Storages/StorageSystemClusters.h>
#include <DB/Storages/System/StorageSystemNumbers.h>
#include <DB/Storages/System/StorageSystemTables.h>
#include <DB/Storages/System/StorageSystemParts.h>
#include <DB/Storages/System/StorageSystemDatabases.h>
#include <DB/Storages/System/StorageSystemProcesses.h>
#include <DB/Storages/System/StorageSystemEvents.h>
#include <DB/Storages/System/StorageSystemOne.h>
#include <DB/Storages/System/StorageSystemMerges.h>
#include <DB/Storages/System/StorageSystemSettings.h>
#include <DB/Storages/System/StorageSystemZooKeeper.h>
#include <DB/Storages/System/StorageSystemReplicas.h>
#include <DB/Storages/System/StorageSystemReplicationQueue.h>
#include <DB/Storages/System/StorageSystemDictionaries.h>
#include <DB/Storages/System/StorageSystemColumns.h>
#include <DB/Storages/System/StorageSystemFunctions.h>
#include <DB/Storages/System/StorageSystemClusters.h>
#include <DB/IO/copyData.h>
#include <DB/IO/LimitReadBuffer.h>
@ -81,6 +83,8 @@ public:
private:
void run()
{
setThreadName("ProfileEventsTx");
const auto get_next_minute = [] {
return std::chrono::time_point_cast<std::chrono::minutes, std::chrono::system_clock>(
std::chrono::system_clock::now() + std::chrono::minutes(1)
@ -259,6 +263,8 @@ UsersConfigReloader::~UsersConfigReloader()
void UsersConfigReloader::run()
{
setThreadName("UserConfReload");
while (!quit)
{
std::this_thread::sleep_for(std::chrono::seconds(2));
@ -548,6 +554,7 @@ int Server::main(const std::vector<std::string> & args)
global_context->addTable("system", "events", StorageSystemEvents::create("events"));
global_context->addTable("system", "merges", StorageSystemMerges::create("merges"));
global_context->addTable("system", "replicas", StorageSystemReplicas::create("replicas"));
global_context->addTable("system", "replication_queue", StorageSystemReplicationQueue::create("replication_queue"));
global_context->addTable("system", "dictionaries", StorageSystemDictionaries::create("dictionaries"));
global_context->addTable("system", "columns", StorageSystemColumns::create("columns"));
global_context->addTable("system", "functions", StorageSystemFunctions::create("functions"));

View File

@ -0,0 +1,196 @@
#include <DB/Core/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/Common/setThreadName.h>
#include <DB/IO/WriteHelpers.h>
#include <Yandex/logger_useful.h>
#include <DB/Storages/MergeTree/BackgroundProcessingPool.h>
namespace DB
{
constexpr double BackgroundProcessingPool::sleep_seconds;
constexpr double BackgroundProcessingPool::sleep_seconds_random_part;
void BackgroundProcessingPool::TaskInfo::wake()
{
if (removed)
return;
std::unique_lock<std::mutex> lock(pool.mutex);
pool.tasks.splice(pool.tasks.begin(), pool.tasks, iterator);
/// Если эта задача в прошлый раз ничего не сделала, и ей было назначено спать, то отменим время сна.
time_t current_time = time(0);
if (next_time_to_execute > current_time)
next_time_to_execute = current_time;
/// Если все потоки сейчас выполняют работу, этот вызов никого не разбудит.
pool.wake_event.notify_one();
}
BackgroundProcessingPool::BackgroundProcessingPool(int size_) : size(size_)
{
LOG_INFO(&Logger::get("BackgroundProcessingPool"), "Create BackgroundProcessingPool with " << size << " threads");
threads.resize(size);
for (auto & thread : threads)
thread = std::thread([this] { threadFunction(); });
}
int BackgroundProcessingPool::getCounter(const String & name)
{
std::unique_lock<std::mutex> lock(mutex);
return counters[name];
}
BackgroundProcessingPool::TaskHandle BackgroundProcessingPool::addTask(const Task & task)
{
TaskHandle res(new TaskInfo(*this, task));
{
std::unique_lock<std::mutex> lock(mutex);
res->iterator = tasks.insert(tasks.begin(), res);
}
wake_event.notify_all();
return res;
}
void BackgroundProcessingPool::removeTask(const TaskHandle & task)
{
task->removed = true;
/// Дождёмся завершения всех выполнений этой задачи.
{
Poco::ScopedWriteRWLock wlock(task->rwlock);
}
{
std::unique_lock<std::mutex> lock(mutex);
tasks.erase(task->iterator);
}
}
BackgroundProcessingPool::~BackgroundProcessingPool()
{
try
{
shutdown = true;
wake_event.notify_all();
for (std::thread & thread : threads)
thread.join();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
void BackgroundProcessingPool::threadFunction()
{
setThreadName("BackgrProcPool");
std::mt19937 rng(reinterpret_cast<intptr_t>(&rng));
std::this_thread::sleep_for(std::chrono::duration<double>(std::uniform_real_distribution<double>(0, sleep_seconds_random_part)(rng)));
while (!shutdown)
{
Counters counters_diff;
bool has_exception = false;
try
{
TaskHandle task;
time_t min_time = std::numeric_limits<time_t>::max();
{
std::unique_lock<std::mutex> lock(mutex);
if (!tasks.empty())
{
/// O(n), n - число задач. По сути, количество таблиц. Обычно их мало.
for (const auto & handle : tasks)
{
time_t next_time_to_execute = handle->next_time_to_execute;
if (next_time_to_execute < min_time)
{
min_time = next_time_to_execute;
task = handle;
}
}
if (task) /// Переложим в конец очереди (уменьшим приоритет среди задач с одинаковым next_time_to_execute).
tasks.splice(tasks.end(), tasks, task->iterator);
}
}
if (shutdown)
break;
if (!task)
{
std::unique_lock<std::mutex> lock(mutex);
wake_event.wait_for(lock,
std::chrono::duration<double>(sleep_seconds
+ std::uniform_real_distribution<double>(0, sleep_seconds_random_part)(rng)));
continue;
}
if (task->removed)
continue;
/// Лучшей задачи не нашлось, а эта задача в прошлый раз ничего не сделала, и поэтому ей назначено некоторое время спать.
time_t current_time = time(0);
if (min_time > current_time)
{
std::unique_lock<std::mutex> lock(mutex);
wake_event.wait_for(lock, std::chrono::duration<double>(
min_time - current_time + std::uniform_real_distribution<double>(0, sleep_seconds_random_part)(rng)));
}
Poco::ScopedReadRWLock rlock(task->rwlock);
if (task->removed)
continue;
Context context(*this, counters_diff);
bool done_work = task->function(context);
/// Если задача сделала полезную работу, то она сможет выполняться в следующий раз хоть сразу.
/// Если нет - добавляем задержку перед повторным исполнением.
task->next_time_to_execute = time(0) + (done_work ? 0 : sleep_seconds);
}
catch (...)
{
has_exception = true;
tryLogCurrentException(__PRETTY_FUNCTION__);
}
/// Вычтем все счётчики обратно.
if (!counters_diff.empty())
{
std::unique_lock<std::mutex> lock(mutex);
for (const auto & it : counters_diff)
counters[it.first] -= it.second;
}
if (shutdown)
break;
if (has_exception)
{
std::unique_lock<std::mutex> lock(mutex);
wake_event.wait_for(lock, std::chrono::duration<double>(sleep_seconds));
}
}
}
}

View File

@ -1,5 +1,6 @@
#include <DB/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h>
#include <DB/Storages/StorageReplicatedMergeTree.h>
#include <DB/Common/setThreadName.h>
namespace DB
@ -13,6 +14,8 @@ ReplicatedMergeTreeCleanupThread::ReplicatedMergeTreeCleanupThread(StorageReplic
void ReplicatedMergeTreeCleanupThread::run()
{
setThreadName("ReplMTCleanup");
const auto CLEANUP_SLEEP_MS = 30 * 1000;
while (!storage.shutdown_called)
@ -176,9 +179,12 @@ void ReplicatedMergeTreeCleanupThread::clearOldBlocks()
std::sort(timed_blocks.begin(), timed_blocks.end(), std::greater<std::pair<Int64, String>>());
for (size_t i = storage.data.settings.replicated_deduplication_window; i < timed_blocks.size(); ++i)
{
/// Устаревшие ноды. Этот код можно будет убрать через пол года.
zookeeper->tryRemove(storage.zookeeper_path + "/blocks/" + timed_blocks[i].second + "/columns");
zookeeper->tryRemove(storage.zookeeper_path + "/blocks/" + timed_blocks[i].second + "/checksums");
ops.push_back(new zkutil::Op::Remove(storage.zookeeper_path + "/blocks/" + timed_blocks[i].second + "/number", -1));
ops.push_back(new zkutil::Op::Remove(storage.zookeeper_path + "/blocks/" + timed_blocks[i].second + "/columns", -1));
ops.push_back(new zkutil::Op::Remove(storage.zookeeper_path + "/blocks/" + timed_blocks[i].second + "/checksums", -1));
ops.push_back(new zkutil::Op::Remove(storage.zookeeper_path + "/blocks/" + timed_blocks[i].second + "/checksum", -1));
ops.push_back(new zkutil::Op::Remove(storage.zookeeper_path + "/blocks/" + timed_blocks[i].second, -1));
if (ops.size() > 400 || i + 1 == timed_blocks.size())

View File

@ -78,6 +78,9 @@ void ReplicatedMergeTreeLogEntry::writeText(WriteBuffer & out) const
out << "detached\n";
out << source_part_name << "\ninto\n" << new_part_name;
break;
default:
throw Exception("Unknown log entry type: " + DB::toString(type), ErrorCodes::LOGICAL_ERROR);
}
out << '\n';

View File

@ -2,6 +2,7 @@
#include <DB/Storages/StorageReplicatedMergeTree.h>
#include <DB/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h>
#include <DB/Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h>
#include <DB/Common/setThreadName.h>
namespace DB
@ -32,6 +33,8 @@ void ReplicatedMergeTreeRestartingThread::run()
constexpr auto retry_delay_ms = 10 * 1000;
constexpr auto check_delay_ms = 60 * 1000;
setThreadName("ReplMTRestart");
try
{
bool first_time = true;
@ -226,7 +229,8 @@ void ReplicatedMergeTreeRestartingThread::activateReplica()
WriteBufferFromString address_buf(address);
address_buf
<< "host: " << host_port.first << '\n'
<< "port: " << host_port.second << '\n';
<< "port: " << host_port.second << '\n'
<< "tcp_port: " << storage.context.getTCPPort() << '\n';
}
String is_active_path = storage.replica_path + "/is_active";

View File

@ -4,6 +4,7 @@
#include <DB/DataStreams/IProfilingBlockInputStream.h>
#include <DB/Storages/StorageBuffer.h>
#include <DB/Parsers/ASTInsertQuery.h>
#include <DB/Common/setThreadName.h>
#include <Poco/Ext/ThreadNumber.h>
#include <statdaemons/ext/range.hpp>
@ -449,6 +450,8 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl
void StorageBuffer::flushThread()
{
setThreadName("BufferFlush");
do
{
try

View File

@ -17,8 +17,8 @@
#include <DB/Storages/StorageMerge.h>
#include <DB/Storages/StorageMergeTree.h>
#include <DB/Storages/StorageDistributed.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Storages/StorageSystemOne.h>
#include <DB/Storages/System/StorageSystemNumbers.h>
#include <DB/Storages/System/StorageSystemOne.h>
#include <DB/Storages/StorageFactory.h>
#include <DB/Storages/StorageView.h>
#include <DB/Storages/StorageMaterializedView.h>

View File

@ -255,6 +255,7 @@ bool StorageMergeTree::mergeTask(BackgroundProcessingPool::Context & background_
{
if (shutdown_called)
return false;
try
{
size_t aio_threshold = context.getSettings().min_bytes_to_use_direct_io;

View File

@ -10,12 +10,14 @@
#include <DB/Parsers/formatAST.h>
#include <DB/IO/WriteBufferFromOStream.h>
#include <DB/IO/ReadBufferFromString.h>
#include <DB/IO/Operators.h>
#include <DB/Interpreters/InterpreterAlterQuery.h>
#include <DB/Common/VirtualColumnUtils.h>
#include <DB/Parsers/ASTInsertQuery.h>
#include <DB/DataStreams/AddingConstColumnBlockInputStream.h>
#include <DB/Common/Macros.h>
#include <DB/Common/formatReadable.h>
#include <DB/Common/setThreadName.h>
#include <Poco/DirectoryIterator.h>
#include <time.h>
@ -141,6 +143,9 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
if (!attach)
{
if (!data.getDataParts().empty())
throw Exception("Data directory for table already containing data parts - probably it was unclean DROP table or manual intervention. You must either clear directory by hand or use ATTACH TABLE instead of CREATE TABLE if you need to use that parts.", ErrorCodes::INCORRECT_DATA);
createTableIfNotExists();
checkTableStructure(false, false);
@ -240,38 +245,43 @@ void StorageReplicatedMergeTree::createTableIfNotExists()
zookeeper->createAncestors(zookeeper_path);
/// Запишем метаданные таблицы, чтобы реплики могли сверять с ними параметры таблицы.
std::stringstream metadata;
metadata << "metadata format version: 1" << std::endl;
metadata << "date column: " << data.date_column_name << std::endl;
metadata << "sampling expression: " << formattedAST(data.sampling_expression) << std::endl;
metadata << "index granularity: " << data.index_granularity << std::endl;
metadata << "mode: " << static_cast<int>(data.mode) << std::endl;
metadata << "sign column: " << data.sign_column << std::endl;
metadata << "primary key: " << formattedAST(data.primary_expr_ast) << std::endl;
std::string metadata;
{
WriteBufferFromString out(metadata);
out << "metadata format version: 1" << "\n"
<< "date column: " << data.date_column_name << "\n"
<< "sampling expression: " << formattedAST(data.sampling_expression) << "\n"
<< "index granularity: " << data.index_granularity << "\n"
<< "mode: " << static_cast<int>(data.mode) << "\n"
<< "sign column: " << data.sign_column << "\n"
<< "primary key: " << formattedAST(data.primary_expr_ast) << "\n";
}
auto acl = zookeeper->getDefaultACL();
zkutil::Ops ops;
ops.push_back(new zkutil::Op::Create(zookeeper_path, "",
zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/metadata", metadata.str(),
zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
acl, zkutil::CreateMode::Persistent));
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/metadata", metadata,
acl, zkutil::CreateMode::Persistent));
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/columns", ColumnsDescription<false>{
data.getColumnsListNonMaterialized(), data.materialized_columns,
data.alias_columns, data.column_defaults}.toString(),
zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
acl, zkutil::CreateMode::Persistent));
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/log", "",
zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
acl, zkutil::CreateMode::Persistent));
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/blocks", "",
zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
acl, zkutil::CreateMode::Persistent));
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/block_numbers", "",
zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
acl, zkutil::CreateMode::Persistent));
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/nonincrement_block_numbers", "",
zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
acl, zkutil::CreateMode::Persistent));
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/leader_election", "",
zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
acl, zkutil::CreateMode::Persistent));
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/temp", "",
zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
acl, zkutil::CreateMode::Persistent));
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/replicas", "",
zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
acl, zkutil::CreateMode::Persistent));
auto code = zookeeper->tryMulti(ops);
if (code != ZOK && code != ZNODEEXISTS)
@ -370,13 +380,14 @@ void StorageReplicatedMergeTree::createReplica()
LOG_DEBUG(log, "Creating replica " << replica_path);
/// Создадим пустую реплику. Ноду columns создадим в конце - будем использовать ее в качестве признака, что создание реплики завершено.
auto acl = zookeeper->getDefaultACL();
zkutil::Ops ops;
ops.push_back(new zkutil::Op::Create(replica_path, "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
ops.push_back(new zkutil::Op::Create(replica_path + "/host", "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
ops.push_back(new zkutil::Op::Create(replica_path + "/log_pointer", "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
ops.push_back(new zkutil::Op::Create(replica_path + "/queue", "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
ops.push_back(new zkutil::Op::Create(replica_path + "/parts", "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
ops.push_back(new zkutil::Op::Create(replica_path + "/flags", "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
ops.push_back(new zkutil::Op::Create(replica_path, "", acl, zkutil::CreateMode::Persistent));
ops.push_back(new zkutil::Op::Create(replica_path + "/host", "", acl, zkutil::CreateMode::Persistent));
ops.push_back(new zkutil::Op::Create(replica_path + "/log_pointer", "", acl, zkutil::CreateMode::Persistent));
ops.push_back(new zkutil::Op::Create(replica_path + "/queue", "", acl, zkutil::CreateMode::Persistent));
ops.push_back(new zkutil::Op::Create(replica_path + "/parts", "", acl, zkutil::CreateMode::Persistent));
ops.push_back(new zkutil::Op::Create(replica_path + "/flags", "", acl, zkutil::CreateMode::Persistent));
try
{
@ -554,36 +565,48 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
for (const String & name : parts_to_fetch)
expected_parts.erase(name);
String sanity_report =
"There are " + toString(unexpected_parts.size()) + " unexpected parts, "
+ toString(parts_to_add.size()) + " unexpectedly merged parts, "
+ toString(expected_parts.size()) + " missing obsolete parts, "
+ toString(parts_to_fetch.size()) + " missing parts";
/** Для проверки адекватности, для кусков, которые есть в ФС, но нет в ZK, будем учитывать только не самые новые куски.
* Потому что неожиданные новые куски обычно возникают лишь оттого, что они не успели записаться в ZK при грубом перезапуске сервера.
* Также это возникает от дедуплицированных кусков, которые не успели удалиться.
*/
size_t unexpected_parts_nonnew = 0;
for (const auto & part : unexpected_parts)
if (part->level > 0 || part->right < RESERVED_BLOCK_NUMBERS)
++unexpected_parts_nonnew;
String sanity_report = "There are "
+ toString(unexpected_parts.size()) + " unexpected parts ("
+ toString(unexpected_parts_nonnew) + " of them is not just-written), "
+ toString(parts_to_add.size()) + " unexpectedly merged parts, "
+ toString(expected_parts.size()) + " missing obsolete parts, "
+ toString(parts_to_fetch.size()) + " missing parts";
/** Можно автоматически синхронизировать данные,
* если количество ошибок каждого из четырёх типов не больше соответствующих порогов,
* или если отношение общего количества ошибок к общему количеству кусков (минимальному - в локальной файловой системе или в ZK)
* если количество ошибок каждого из четырёх типов не больше соответствующих порогов,
* или если отношение общего количества ошибок к общему количеству кусков (минимальному - в локальной файловой системе или в ZK)
* не больше некоторого отношения (например 5%).
*
* Большое количество несовпадений в данных на файловой системе и ожидаемых данных
* может свидетельствовать об ошибке конфигурации (сервер случайно подключили как реплику не от того шарда).
* В этом случае, защитный механизм не даёт стартовать серверу.
*/
size_t min_parts_local_or_expected = std::min(expected_parts_vec.size(), parts.size());
size_t total_difference = parts_to_add.size() + unexpected_parts_nonnew + expected_parts.size() + parts_to_fetch.size();
bool insane =
(parts_to_add.size() > data.settings.replicated_max_unexpectedly_merged_parts
|| unexpected_parts.size() > data.settings.replicated_max_unexpected_parts
|| unexpected_parts_nonnew > data.settings.replicated_max_unexpected_parts
|| expected_parts.size() > data.settings.replicated_max_missing_obsolete_parts
|| parts_to_fetch.size() > data.settings.replicated_max_missing_active_parts)
&& ((parts_to_add.size() + unexpected_parts.size() + expected_parts.size() + parts_to_fetch.size())
> min_parts_local_or_expected * data.settings.replicated_max_ratio_of_wrong_parts);
&& (total_difference > min_parts_local_or_expected * data.settings.replicated_max_ratio_of_wrong_parts);
if (insane)
{
if (skip_sanity_checks)
LOG_WARNING(log, sanity_report);
else
throw Exception("The local set of parts of table " + getTableName() + " doesn't look like the set of parts in ZooKeeper. "
+ sanity_report, ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS);
}
if (insane && !skip_sanity_checks)
throw Exception("The local set of parts of table " + getTableName() + " doesn't look like the set of parts in ZooKeeper. "
+ sanity_report, ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS);
if (total_difference > 0)
LOG_WARNING(log, sanity_report);
/// Добавим в ZK информацию о кусках, покрывающих недостающие куски.
for (const MergeTreeData::DataPartPtr & part : parts_to_add)
@ -689,23 +712,25 @@ void StorageReplicatedMergeTree::checkPartAndAddToZooKeeper(const MergeTreeData:
return;
}
auto acl = zookeeper->getDefaultACL();
ops.push_back(new zkutil::Op::Check(
zookeeper_path + "/columns",
expected_columns_version));
ops.push_back(new zkutil::Op::Create(
replica_path + "/parts/" + part_name,
"",
zookeeper->getDefaultACL(),
acl,
zkutil::CreateMode::Persistent));
ops.push_back(new zkutil::Op::Create(
replica_path + "/parts/" + part_name + "/columns",
part->columns.toString(),
zookeeper->getDefaultACL(),
acl,
zkutil::CreateMode::Persistent));
ops.push_back(new zkutil::Op::Create(
replica_path + "/parts/" + part_name + "/checksums",
part->checksums.toString(),
zookeeper->getDefaultACL(),
acl,
zkutil::CreateMode::Persistent));
}
@ -796,7 +821,7 @@ void StorageReplicatedMergeTree::pullLogsToQueue(zkutil::EventPtr next_update_ev
}
bool StorageReplicatedMergeTree::shouldExecuteLogEntry(const LogEntry & entry)
bool StorageReplicatedMergeTree::shouldExecuteLogEntry(const LogEntry & entry, String & out_postpone_reason)
{
/// queue_mutex уже захвачен. Функция вызывается только из queueTask.
@ -805,8 +830,10 @@ bool StorageReplicatedMergeTree::shouldExecuteLogEntry(const LogEntry & entry)
/// Проверим, не создаётся ли сейчас этот же кусок другим действием.
if (future_parts.count(entry.new_part_name))
{
LOG_DEBUG(log, "Not executing log entry for part " << entry.new_part_name
<< " because another log entry for the same part is being processed. This shouldn't happen often.");
String reason = "Not executing log entry for part " + entry.new_part_name
+ " because another log entry for the same part is being processed. This shouldn't happen often.";
LOG_DEBUG(log, reason);
out_postpone_reason = reason;
return false;
/** Когда соответствующее действие завершится, то shouldExecuteLogEntry, в следующий раз, пройдёт успешно,
@ -828,8 +855,10 @@ bool StorageReplicatedMergeTree::shouldExecuteLogEntry(const LogEntry & entry)
if (future_part.contains(result_part))
{
LOG_DEBUG(log, "Not executing log entry for part " << entry.new_part_name
<< " because another log entry for covering part " << future_part_name << " is being processed.");
String reason = "Not executing log entry for part " + entry.new_part_name
+ " because another log entry for covering part " + future_part_name + " is being processed.";
LOG_DEBUG(log, reason);
out_postpone_reason = reason;
return false;
}
}
@ -846,15 +875,19 @@ bool StorageReplicatedMergeTree::shouldExecuteLogEntry(const LogEntry & entry)
{
if (future_parts.count(name))
{
LOG_TRACE(log, "Not merging into part " << entry.new_part_name
<< " because part " << name << " is not ready yet (log entry for that part is being processed).");
String reason = "Not merging into part " + entry.new_part_name
+ " because part " + name + " is not ready yet (log entry for that part is being processed).";
LOG_TRACE(log, reason);
out_postpone_reason = reason;
return false;
}
}
if (merger.isCancelled())
{
LOG_DEBUG(log, "Not executing log entry for part " << entry.new_part_name << " because merges are cancelled now.");
String reason = "Not executing log entry for part " + entry.new_part_name + " because merges are cancelled now.";
LOG_DEBUG(log, reason);
out_postpone_reason = reason;
return false;
}
}
@ -1091,16 +1124,18 @@ bool StorageReplicatedMergeTree::executeLogEntry(const LogEntry & entry, Backgro
throw Exception("Logical error: log entry with quorum for part covering more than one block number",
ErrorCodes::LOGICAL_ERROR);
auto acl = zookeeper->getDefaultACL();
ops.push_back(new zkutil::Op::Create(
zookeeper_path + "/nonincrement_block_numbers/" + partition_str + "/block-" + padIndex(part_info.left),
"",
zookeeper->getDefaultACL(),
acl,
zkutil::CreateMode::Persistent));
ops.push_back(new zkutil::Op::Create(
zookeeper_path + "/quorum/failed_parts/" + entry.new_part_name,
"",
zookeeper->getDefaultACL(),
acl,
zkutil::CreateMode::Persistent));
auto code = zookeeper->tryMulti(ops);
@ -1330,6 +1365,8 @@ bool StorageReplicatedMergeTree::executeAttachPart(const StorageReplicatedMergeT
void StorageReplicatedMergeTree::queueUpdatingThread()
{
setThreadName("ReplMTQueueUpd");
while (!shutdown_called)
{
try
@ -1370,7 +1407,10 @@ bool StorageReplicatedMergeTree::queueTask(BackgroundProcessingPool::Context & p
{
for (LogEntries::iterator it = queue.begin(); it != queue.end(); ++it)
{
if (!(*it)->currently_executing && shouldExecuteLogEntry(**it))
if ((*it)->currently_executing)
continue;
if (shouldExecuteLogEntry(**it, (*it)->postpone_reason))
{
entry = *it;
entry->tagPartAsFuture(*this);
@ -1380,6 +1420,11 @@ bool StorageReplicatedMergeTree::queueTask(BackgroundProcessingPool::Context & p
entry->last_attempt_time = time(0);
break;
}
else
{
++entry->num_postponed;
entry->last_postpone_time = time(0);
}
}
}
}
@ -1469,6 +1514,8 @@ bool StorageReplicatedMergeTree::queueTask(BackgroundProcessingPool::Context & p
void StorageReplicatedMergeTree::mergeSelectingThread()
{
setThreadName("ReplMTMergeSel");
bool need_pull = true;
/** Может много времени тратиться на определение, можно ли мерджить два рядом стоящих куска.
@ -1688,6 +1735,8 @@ void StorageReplicatedMergeTree::mergeSelectingThread()
void StorageReplicatedMergeTree::alterThread()
{
setThreadName("ReplMTAlter");
bool force_recheck_parts = true;
while (!shutdown_called)
@ -2175,6 +2224,8 @@ void StorageReplicatedMergeTree::checkPart(const String & part_name)
void StorageReplicatedMergeTree::partCheckThread()
{
setThreadName("ReplMTPartCheck");
while (!shutdown_called)
{
try
@ -2352,12 +2403,8 @@ void StorageReplicatedMergeTree::fetchPart(const String & part_name, const Strin
String host_port_str = zookeeper->get(replica_path + "/host");
ReadBufferFromString buf(host_port_str);
assertString("host: ", buf);
readString(host, buf);
assertString("\nport: ", buf);
readText(port, buf);
assertString("\n", buf);
assertEOF(buf);
buf >> "host: " >> host >> "\n"
>> "port: " >> port >> "\n";
MergeTreeData::MutableDataPartPtr part = fetcher.fetchPart(part_name, replica_path, host, port, to_detached);
@ -2777,7 +2824,7 @@ void StorageReplicatedMergeTree::dropPartition(const Field & field, bool detach,
/// Такого никогда не должно происходить.
if (right == 0)
return;
throw Exception("Logical error: just allocated block number is zero", ErrorCodes::LOGICAL_ERROR);
--right;
String fake_part_name = getFakePartNameForDrop(month_name, 0, right);
@ -3221,6 +3268,18 @@ void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields)
}
void StorageReplicatedMergeTree::getQueue(LogEntriesData & res, String & replica_name_)
{
res.clear();
replica_name_ = replica_name;
std::lock_guard<std::mutex> lock(queue_mutex);
res.reserve(queue.size());
for (const auto & entry : queue)
res.emplace_back(*entry);
}
void StorageReplicatedMergeTree::fetchPartition(const Field & partition, const String & from_, const Settings & settings)
{
auto zookeeper = getZooKeeper();

View File

@ -1,4 +1,4 @@
#include <DB/Storages/StorageSystemClusters.h>
#include <DB/Storages/System/StorageSystemClusters.h>
#include <DB/Interpreters/Cluster.h>
#include <DB/Columns/ColumnString.h>
#include <DB/Columns/ColumnsNumber.h>

View File

@ -1,4 +1,4 @@
#include <DB/Storages/StorageSystemColumns.h>
#include <DB/Storages/System/StorageSystemColumns.h>
#include <DB/Storages/MergeTree/MergeTreeData.h>
#include <DB/Storages/StorageMergeTree.h>
#include <DB/Storages/StorageReplicatedMergeTree.h>

View File

@ -1,7 +1,7 @@
#include <DB/Columns/ColumnString.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/Storages/StorageSystemDatabases.h>
#include <DB/Storages/System/StorageSystemDatabases.h>
namespace DB

View File

@ -1,4 +1,4 @@
#include <DB/Storages/StorageSystemDictionaries.h>
#include <DB/Storages/System/StorageSystemDictionaries.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeArray.h>

View File

@ -3,7 +3,7 @@
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/Storages/StorageSystemEvents.h>
#include <DB/Storages/System/StorageSystemEvents.h>
namespace DB

View File

@ -1,4 +1,4 @@
#include <DB/Storages/StorageSystemFunctions.h>
#include <DB/Storages/System/StorageSystemFunctions.h>
#include <DB/Functions/FunctionFactory.h>
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
#include <DB/Columns/ColumnString.h>
@ -46,14 +46,17 @@ BlockInputStreams StorageSystemFunctions::read(
column_is_aggregate.column->insert(UInt64(0));
}
const auto & aggregate_functions = context.getAggregateFunctionFactory().getFunctionNames();
for (const auto & it : aggregate_functions)
const auto & aggregate_function_factory = context.getAggregateFunctionFactory();
for (const auto & details : aggregate_function_factory)
{
column_name.column->insert(it);
column_is_aggregate.column->insert(UInt64(1));
if (!details.is_alias)
{
column_name.column->insert(details.name);
column_is_aggregate.column->insert(UInt64(1));
}
}
return BlockInputStreams{ 1, new OneBlockInputStream{{ column_name, column_is_aggregate }} };
}
}
}

View File

@ -1,4 +1,4 @@
#include <DB/Storages/StorageSystemMerges.h>
#include <DB/Storages/System/StorageSystemMerges.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/Columns/ColumnString.h>

View File

@ -5,7 +5,7 @@
#include <DB/Columns/ColumnsNumber.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Storages/System/StorageSystemNumbers.h>
namespace DB

View File

@ -4,7 +4,7 @@
#include <DB/Columns/ColumnsNumber.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/Storages/StorageSystemOne.h>
#include <DB/Storages/System/StorageSystemOne.h>
namespace DB

View File

@ -4,7 +4,7 @@
#include <DB/DataTypes/DataTypeDateTime.h>
#include <DB/DataTypes/DataTypeDate.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/Storages/StorageSystemParts.h>
#include <DB/Storages/System/StorageSystemParts.h>
#include <DB/Storages/StorageMergeTree.h>
#include <DB/Storages/StorageReplicatedMergeTree.h>
#include <DB/Common/VirtualColumnUtils.h>

View File

@ -3,7 +3,7 @@
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/Interpreters/ProcessList.h>
#include <DB/Storages/StorageSystemProcesses.h>
#include <DB/Storages/System/StorageSystemProcesses.h>
namespace DB

View File

@ -3,7 +3,7 @@
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeDateTime.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/Storages/StorageSystemReplicas.h>
#include <DB/Storages/System/StorageSystemReplicas.h>
#include <DB/Storages/StorageReplicatedMergeTree.h>
#include <DB/Common/VirtualColumnUtils.h>

View File

@ -0,0 +1,193 @@
#include <DB/Columns/ColumnString.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeDateTime.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/Storages/System/StorageSystemReplicationQueue.h>
#include <DB/Storages/StorageReplicatedMergeTree.h>
#include <DB/Common/VirtualColumnUtils.h>
namespace DB
{
StorageSystemReplicationQueue::StorageSystemReplicationQueue(const std::string & name_)
: name(name_)
, columns{
/// Свойства таблицы.
{ "database", new DataTypeString },
{ "table", new DataTypeString },
{ "replica_name", new DataTypeString },
/// Неизменяемые свойства элемента.
{ "position", new DataTypeUInt32 },
{ "node_name", new DataTypeString },
{ "type", new DataTypeString },
{ "create_time", new DataTypeDateTime},
{ "required_quorum", new DataTypeUInt32 },
{ "source_replica", new DataTypeString },
{ "new_part_name", new DataTypeString },
{ "parts_to_merge", new DataTypeArray(new DataTypeString) },
{ "is_detach", new DataTypeUInt8 },
{ "is_attach_unreplicated", new DataTypeUInt8 },
{ "attach_source_part_name",new DataTypeString },
/// Статус обработки элемента.
{ "is_currently_executing", new DataTypeUInt8 },
{ "num_tries", new DataTypeUInt32 },
{ "last_exception", new DataTypeString },
{ "last_attempt_time", new DataTypeDateTime},
{ "num_postponed", new DataTypeUInt32 },
{ "postpone_reason", new DataTypeString },
{ "last_postpone_time", new DataTypeDateTime},
}
{
}
StoragePtr StorageSystemReplicationQueue::create(const std::string & name_)
{
return (new StorageSystemReplicationQueue(name_))->thisPtr();
}
BlockInputStreams StorageSystemReplicationQueue::read(
const Names & column_names,
ASTPtr query,
const Context & context,
const Settings & settings,
QueryProcessingStage::Enum & processed_stage,
const size_t max_block_size,
const unsigned threads)
{
check(column_names);
processed_stage = QueryProcessingStage::FetchColumns;
/// Собираем набор реплицируемых таблиц.
Databases replicated_tables;
{
Poco::ScopedLock<Poco::Mutex> lock(context.getMutex());
for (const auto & db : context.getDatabases())
for (const auto & table : db.second)
if (typeid_cast<const StorageReplicatedMergeTree *>(table.second.get()))
replicated_tables[db.first][table.first] = table.second;
}
ColumnWithTypeAndName col_database_to_filter { new ColumnString, new DataTypeString, "database" };
ColumnWithTypeAndName col_table_to_filter { new ColumnString, new DataTypeString, "table" };
for (auto & db : replicated_tables)
{
for (auto & table : db.second)
{
col_database_to_filter.column->insert(db.first);
col_table_to_filter.column->insert(table.first);
}
}
/// Определяем, какие нужны таблицы, по условиям в запросе.
{
Block filtered_block { col_database_to_filter, col_table_to_filter };
VirtualColumnUtils::filterBlockWithQuery(query, filtered_block, context);
if (!filtered_block.rows())
return BlockInputStreams();
col_database_to_filter = filtered_block.getByName("database");
col_table_to_filter = filtered_block.getByName("table");
}
ColumnWithTypeAndName col_database { new ColumnString, new DataTypeString, "database" };
ColumnWithTypeAndName col_table { new ColumnString, new DataTypeString, "table" };
ColumnWithTypeAndName col_replica_name { new ColumnString, new DataTypeString, "replica_name" };
ColumnWithTypeAndName col_position { new ColumnUInt32, new DataTypeUInt32, "position" };
ColumnWithTypeAndName col_node_name { new ColumnString, new DataTypeString, "node_name" };
ColumnWithTypeAndName col_type { new ColumnString, new DataTypeString, "type" };
ColumnWithTypeAndName col_create_time { new ColumnUInt32, new DataTypeDateTime, "create_time" };
ColumnWithTypeAndName col_required_quorum { new ColumnUInt32, new DataTypeUInt32, "required_quorum" };
ColumnWithTypeAndName col_source_replica { new ColumnString, new DataTypeString, "source_replica" };
ColumnWithTypeAndName col_new_part_name { new ColumnString, new DataTypeString, "new_part_name" };
ColumnWithTypeAndName col_parts_to_merge { new ColumnArray(new ColumnString), new DataTypeArray(new DataTypeString), "parts_to_merge" };
ColumnWithTypeAndName col_is_detach { new ColumnUInt8, new DataTypeUInt8, "is_detach" };
ColumnWithTypeAndName col_is_attach_unreplicated { new ColumnUInt8, new DataTypeUInt8, "is_attach_unreplicated" };
ColumnWithTypeAndName col_attach_source_part_name { new ColumnString, new DataTypeString, "attach_source_part_name" };
ColumnWithTypeAndName col_is_currently_executing { new ColumnUInt8, new DataTypeUInt8, "is_currently_executing" };
ColumnWithTypeAndName col_num_tries { new ColumnUInt32, new DataTypeUInt32, "num_tries" };
ColumnWithTypeAndName col_last_exception { new ColumnString, new DataTypeString, "last_exception" };
ColumnWithTypeAndName col_last_attempt_time { new ColumnUInt32, new DataTypeDateTime, "last_attempt_time" };
ColumnWithTypeAndName col_num_postponed { new ColumnUInt32, new DataTypeUInt32, "num_postponed" };
ColumnWithTypeAndName col_postpone_reason { new ColumnString, new DataTypeString, "postpone_reason" };
ColumnWithTypeAndName col_last_postpone_time { new ColumnUInt32, new DataTypeDateTime, "last_postpone_time" };
StorageReplicatedMergeTree::LogEntriesData queue;
String replica_name;
for (size_t i = 0, tables_size = col_database_to_filter.column->size(); i < tables_size; ++i)
{
String database = (*col_database_to_filter.column)[i].safeGet<const String &>();
String table = (*col_table_to_filter.column)[i].safeGet<const String &>();
typeid_cast<StorageReplicatedMergeTree &>(*replicated_tables[database][table]).getQueue(queue, replica_name);
for (size_t j = 0, queue_size = queue.size(); j < queue_size; ++j)
{
const auto & entry = queue[j];
Array parts_to_merge;
parts_to_merge.reserve(entry.parts_to_merge.size());
for (const auto & name : entry.parts_to_merge)
parts_to_merge.push_back(name);
col_database .column->insert(database);
col_table .column->insert(table);
col_replica_name .column->insert(replica_name);
col_position .column->insert(UInt64(j));
col_node_name .column->insert(entry.znode_name);
col_type .column->insert(entry.typeToString());
col_create_time .column->insert(UInt64(entry.create_time));
col_required_quorum .column->insert(UInt64(entry.quorum));
col_source_replica .column->insert(entry.source_replica);
col_new_part_name .column->insert(entry.new_part_name);
col_parts_to_merge .column->insert(parts_to_merge);
col_is_detach .column->insert(UInt64(entry.detach));
col_is_attach_unreplicated .column->insert(UInt64(entry.attach_unreplicated));
col_attach_source_part_name .column->insert(entry.source_part_name);
col_is_currently_executing .column->insert(UInt64(entry.currently_executing));
col_num_tries .column->insert(UInt64(entry.num_tries));
col_last_exception .column->insert(entry.exception ? entry.exception->displayText() : "");
col_last_attempt_time .column->insert(UInt64(entry.last_attempt_time));
col_num_postponed .column->insert(UInt64(entry.num_postponed));
col_postpone_reason .column->insert(entry.postpone_reason);
col_last_postpone_time .column->insert(UInt64(entry.last_postpone_time));
}
}
Block block{
col_database,
col_table,
col_replica_name,
col_position,
col_node_name,
col_type,
col_create_time,
col_required_quorum,
col_source_replica,
col_new_part_name,
col_parts_to_merge,
col_is_detach,
col_is_attach_unreplicated,
col_attach_source_part_name,
col_is_currently_executing,
col_num_tries,
col_last_exception,
col_last_attempt_time,
col_num_postponed,
col_postpone_reason,
col_last_postpone_time,
};
return BlockInputStreams(1, new OneBlockInputStream(block));
}
}

View File

@ -2,7 +2,7 @@
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/Storages/StorageSystemSettings.h>
#include <DB/Storages/System/StorageSystemSettings.h>
namespace DB

View File

@ -1,7 +1,7 @@
#include <DB/Columns/ColumnString.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/Storages/StorageSystemTables.h>
#include <DB/Storages/System/StorageSystemTables.h>
#include <DB/Common/VirtualColumnUtils.h>

View File

@ -3,7 +3,7 @@
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeDateTime.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/Storages/StorageSystemZooKeeper.h>
#include <DB/Storages/System/StorageSystemZooKeeper.h>
#include <DB/Parsers/ASTSelectQuery.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/ASTLiteral.h>

View File

@ -3,7 +3,7 @@
#include <Poco/SharedPtr.h>
#include <DB/IO/WriteBufferFromOStream.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Storages/System/StorageSystemNumbers.h>
#include <DB/DataStreams/LimitBlockInputStream.h>
#include <DB/DataStreams/TabSeparatedRowOutputStream.h>
#include <DB/DataStreams/BlockOutputStreamFromRowOutputStream.h>
@ -31,11 +31,11 @@ int main(int argc, char ** argv)
DB::WriteBufferFromOStream out_buf(std::cout);
DB::QueryProcessingStage::Enum stage;
DB::LimitBlockInputStream input(table->read(column_names, 0, DB::Context{}, DB::Settings(), stage, 10)[0], 10, 96);
DB::RowOutputStreamPtr output_ = new DB::TabSeparatedRowOutputStream(out_buf, sample);
DB::BlockOutputStreamFromRowOutputStream output(output_);
DB::copyData(input, output);
}
catch (const DB::Exception & e)

View File

@ -264,6 +264,10 @@ public:
GetChildrenFuture asyncGetChildren(const std::string & path);
using RemoveFuture = Future<void, int>;
RemoveFuture asyncRemove(const std::string & path);
static std::string error2string(int32_t code);
/// максимальный размер данных в узле в байтах

View File

@ -760,4 +760,32 @@ ZooKeeper::GetChildrenFuture ZooKeeper::asyncGetChildren(const std::string & pat
return future;
}
ZooKeeper::RemoveFuture ZooKeeper::asyncRemove(const std::string & path)
{
RemoveFuture future {
[path] (int rc)
{
if (rc != ZOK)
throw KeeperException(rc, path);
}};
int32_t code = zoo_adelete(
impl, path.c_str(), -1,
[] (int rc, const void * data)
{
RemoveFuture::TaskPtr owned_task =
std::move(const_cast<RemoveFuture::TaskPtr &>(*static_cast<const RemoveFuture::TaskPtr *>(data)));
(*owned_task)(rc);
},
future.task.get());
ProfileEvents::increment(ProfileEvents::ZooKeeperRemove);
ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions);
if (code != ZOK)
throw KeeperException(code, path);
return future;
}
}

View File

@ -0,0 +1,65 @@
#include <zkutil/ZooKeeper.h>
#include <DB/IO/ReadHelpers.h>
#include <DB/IO/ReadBufferFromFileDescriptor.h>
#include <boost/program_options.hpp>
int main(int argc, char ** argv)
try
{
boost::program_options::options_description desc("Allowed options");
desc.add_options()
("help,h", "produce help message")
("address,a", boost::program_options::value<std::string>()->required(),
"addresses of ZooKeeper instances, comma separated. Example: example01e.yandex.ru:2181")
;
boost::program_options::variables_map options;
boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options);
if (options.count("help"))
{
std::cout << "Remove nodes in ZooKeeper by list passed in stdin." << std::endl;
std::cout << "Usage: " << argv[0] << " [options] < list_of_nodes_on_each_line" << std::endl;
std::cout << desc << std::endl;
return 1;
}
zkutil::ZooKeeper zookeeper(options.at("address").as<std::string>());
DB::ReadBufferFromFileDescriptor in(STDIN_FILENO);
std::list<zkutil::ZooKeeper::RemoveFuture> futures;
std::cerr << "Requested: ";
while (!in.eof())
{
std::string path;
DB::readEscapedString(path, in);
DB::assertString("\n", in);
futures.push_back(zookeeper.asyncRemove(path));
std::cerr << ".";
}
std::cerr << "\n";
std::cerr << "Done: ";
for (auto & future : futures)
{
try
{
future.get();
}
catch (...)
{
std::cerr << DB::getCurrentExceptionMessage(true) << '\n';
}
std::cerr << ".";
}
std::cerr << "\n";
return 0;
}
catch (const Poco::Exception & e)
{
std::cerr << DB::getCurrentExceptionMessage(true) << '\n';
throw;
}