mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge
This commit is contained in:
commit
52af7bf718
@ -1,28 +1,65 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Poco/RegularExpression.h>
|
|
||||||
|
|
||||||
#include <DB/AggregateFunctions/IAggregateFunction.h>
|
#include <DB/AggregateFunctions/IAggregateFunction.h>
|
||||||
|
#include <DB/DataTypes/IDataType.h>
|
||||||
|
#include <boost/iterator/transform_iterator.hpp>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
|
||||||
/** Позволяет создать агрегатную функцию по её имени.
|
/** Позволяет создать агрегатную функцию по её имени.
|
||||||
*/
|
*/
|
||||||
class AggregateFunctionFactory
|
class AggregateFunctionFactory final
|
||||||
{
|
{
|
||||||
public:
|
private:
|
||||||
using FunctionNames = std::vector<std::string>;
|
/// Не std::function, так как меньше indirection и размер объекта.
|
||||||
|
using Creator = AggregateFunctionPtr(*)(const std::string & name, const DataTypes & argument_types);
|
||||||
|
|
||||||
public:
|
public:
|
||||||
AggregateFunctionFactory();
|
AggregateFunctionFactory();
|
||||||
AggregateFunctionPtr get(const String & name, const DataTypes & argument_types, int recursion_level = 0) const;
|
AggregateFunctionPtr get(const String & name, const DataTypes & argument_types, int recursion_level = 0) const;
|
||||||
AggregateFunctionPtr tryGet(const String & name, const DataTypes & argument_types) 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;
|
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;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -2,6 +2,7 @@
|
|||||||
|
|
||||||
#include <DB/DataTypes/DataTypeAggregateFunction.h>
|
#include <DB/DataTypes/DataTypeAggregateFunction.h>
|
||||||
#include <DB/AggregateFunctions/IAggregateFunction.h>
|
#include <DB/AggregateFunctions/IAggregateFunction.h>
|
||||||
|
#include <DB/Columns/ColumnAggregateFunction.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
@ -15,7 +15,7 @@
|
|||||||
#include <DB/Columns/ColumnArray.h>
|
#include <DB/Columns/ColumnArray.h>
|
||||||
|
|
||||||
#include <stats/IntHash.h>
|
#include <stats/IntHash.h>
|
||||||
#include <statdaemons/ext/range.hpp>
|
#include <ext/range.hpp>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
@ -2,8 +2,9 @@
|
|||||||
|
|
||||||
#include <DB/AggregateFunctions/IAggregateFunction.h>
|
#include <DB/AggregateFunctions/IAggregateFunction.h>
|
||||||
#include <DB/DataTypes/DataTypeDateTime.h>
|
#include <DB/DataTypes/DataTypeDateTime.h>
|
||||||
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||||
#include <DB/Parsers/CommonParsers.h>
|
#include <DB/Parsers/CommonParsers.h>
|
||||||
#include <statdaemons/ext/range.hpp>
|
#include <ext/range.hpp>
|
||||||
#include <boost/range/iterator_range_core.hpp>
|
#include <boost/range/iterator_range_core.hpp>
|
||||||
#include <DB/Parsers/ExpressionElementParsers.h>
|
#include <DB/Parsers/ExpressionElementParsers.h>
|
||||||
#include <DB/Parsers/ASTLiteral.h>
|
#include <DB/Parsers/ASTLiteral.h>
|
||||||
|
@ -1,8 +1,5 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <DB/IO/WriteHelpers.h>
|
|
||||||
#include <DB/IO/ReadHelpers.h>
|
|
||||||
|
|
||||||
#include <DB/AggregateFunctions/AggregateFunctionsMinMaxAny.h>
|
#include <DB/AggregateFunctions/AggregateFunctionsMinMaxAny.h>
|
||||||
|
|
||||||
|
|
||||||
@ -14,6 +11,9 @@ namespace DB
|
|||||||
template <typename ResultData, typename ValueData>
|
template <typename ResultData, typename ValueData>
|
||||||
struct AggregateFunctionsArgMinMaxData
|
struct AggregateFunctionsArgMinMaxData
|
||||||
{
|
{
|
||||||
|
using ResultData_t = ResultData;
|
||||||
|
using ValueData_t = ValueData;
|
||||||
|
|
||||||
ResultData result; // аргумент, при котором достигается минимальное/максимальное значение value.
|
ResultData result; // аргумент, при котором достигается минимальное/максимальное значение value.
|
||||||
ValueData value; // значение, для которого считается минимум/максимум.
|
ValueData value; // значение, для которого считается минимум/максимум.
|
||||||
};
|
};
|
||||||
@ -27,7 +27,7 @@ private:
|
|||||||
DataTypePtr type_val;
|
DataTypePtr type_val;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
String getName() const { return (0 == strcmp(decltype(Data::value)::name(), "min")) ? "argMin" : "argMax"; }
|
String getName() const { return (0 == strcmp(Data::ValueData_t::name(), "min")) ? "argMin" : "argMax"; }
|
||||||
|
|
||||||
DataTypePtr getReturnType() const
|
DataTypePtr getReturnType() const
|
||||||
{
|
{
|
||||||
|
103
dbms/include/DB/AggregateFunctions/Helpers.h
Normal file
103
dbms/include/DB/AggregateFunctions/Helpers.h
Normal 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;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
116
dbms/include/DB/AggregateFunctions/HelpersMinMaxAny.h
Normal file
116
dbms/include/DB/AggregateFunctions/HelpersMinMaxAny.h
Normal 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);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -6,7 +6,8 @@
|
|||||||
#include <climits>
|
#include <climits>
|
||||||
#include <sstream>
|
#include <sstream>
|
||||||
#include <stats/ReservoirSampler.h>
|
#include <stats/ReservoirSampler.h>
|
||||||
#include <Yandex/Common.h>
|
#include <common/Common.h>
|
||||||
|
#include <DB/Common/HashTable/Hash.h>
|
||||||
#include <DB/IO/ReadBuffer.h>
|
#include <DB/IO/ReadBuffer.h>
|
||||||
#include <DB/IO/ReadHelpers.h>
|
#include <DB/IO/ReadHelpers.h>
|
||||||
#include <DB/IO/WriteHelpers.h>
|
#include <DB/IO/WriteHelpers.h>
|
||||||
@ -32,8 +33,8 @@ enum class ReservoirSamplerDeterministicOnEmpty
|
|||||||
RETURN_NAN_OR_ZERO,
|
RETURN_NAN_OR_ZERO,
|
||||||
};
|
};
|
||||||
|
|
||||||
template<typename T,
|
template <typename T,
|
||||||
ReservoirSamplerDeterministicOnEmpty OnEmpty = ReservoirSamplerDeterministicOnEmpty::THROW>
|
ReservoirSamplerDeterministicOnEmpty OnEmpty = ReservoirSamplerDeterministicOnEmpty::THROW>
|
||||||
class ReservoirSamplerDeterministic
|
class ReservoirSamplerDeterministic
|
||||||
{
|
{
|
||||||
bool good(const UInt32 hash)
|
bool good(const UInt32 hash)
|
||||||
@ -65,40 +66,6 @@ public:
|
|||||||
++total_values;
|
++total_values;
|
||||||
}
|
}
|
||||||
|
|
||||||
void insertImpl(const T & v, const UInt32 hash)
|
|
||||||
{
|
|
||||||
while (samples.size() + 1 >= sample_count)
|
|
||||||
{
|
|
||||||
if (++skip_degree > detail::MAX_SKIP_DEGREE)
|
|
||||||
throw DB::Exception{"skip_degree exceeds maximum value", DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED};
|
|
||||||
thinOut();
|
|
||||||
}
|
|
||||||
|
|
||||||
samples.emplace_back(v, hash);
|
|
||||||
}
|
|
||||||
|
|
||||||
void thinOut()
|
|
||||||
{
|
|
||||||
auto size = samples.size();
|
|
||||||
for (size_t i = 0; i < size;)
|
|
||||||
{
|
|
||||||
if (!good(samples[i].second))
|
|
||||||
{
|
|
||||||
/// swap current element with the last one
|
|
||||||
std::swap(samples[size - 1], samples[i]);
|
|
||||||
--size;
|
|
||||||
}
|
|
||||||
else
|
|
||||||
++i;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (size != samples.size())
|
|
||||||
{
|
|
||||||
samples.resize(size);
|
|
||||||
sorted = false;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
size_t size() const
|
size_t size() const
|
||||||
{
|
{
|
||||||
return total_values;
|
return total_values;
|
||||||
@ -119,6 +86,7 @@ public:
|
|||||||
|
|
||||||
/** Если T не числовой тип, использование этого метода вызывает ошибку компиляции,
|
/** Если T не числовой тип, использование этого метода вызывает ошибку компиляции,
|
||||||
* но использование класса ошибки не вызывает. SFINAE.
|
* но использование класса ошибки не вызывает. SFINAE.
|
||||||
|
* Не SFINAE. Функции члены шаблонов типов просто не проверяются, пока не используются.
|
||||||
*/
|
*/
|
||||||
double quantileInterpolated(double level)
|
double quantileInterpolated(double level)
|
||||||
{
|
{
|
||||||
@ -153,9 +121,9 @@ public:
|
|||||||
thinOut();
|
thinOut();
|
||||||
}
|
}
|
||||||
|
|
||||||
for (size_t i = 0; i < b.samples.size(); ++i)
|
for (const auto & sample : b.samples)
|
||||||
if (good(b.samples[i].second))
|
if (good(sample.second))
|
||||||
insertImpl(b.samples[i].first, b.samples[i].second);
|
insertImpl(sample.first, sample.second);
|
||||||
|
|
||||||
total_values += b.total_values;
|
total_values += b.total_values;
|
||||||
}
|
}
|
||||||
@ -167,7 +135,7 @@ public:
|
|||||||
samples.resize(std::min(total_values, sample_count));
|
samples.resize(std::min(total_values, sample_count));
|
||||||
|
|
||||||
for (size_t i = 0; i < samples.size(); ++i)
|
for (size_t i = 0; i < samples.size(); ++i)
|
||||||
DB::readBinary(samples[i].first, buf);
|
DB::readPODBinary(samples[i], buf);
|
||||||
|
|
||||||
sorted = false;
|
sorted = false;
|
||||||
}
|
}
|
||||||
@ -178,7 +146,7 @@ public:
|
|||||||
DB::writeIntBinary<size_t>(total_values, buf);
|
DB::writeIntBinary<size_t>(total_values, buf);
|
||||||
|
|
||||||
for (size_t i = 0; i < std::min(sample_count, total_values); ++i)
|
for (size_t i = 0; i < std::min(sample_count, total_values); ++i)
|
||||||
DB::writeBinary(samples[i].first, buf);
|
DB::writePODBinary(samples[i], buf);
|
||||||
}
|
}
|
||||||
|
|
||||||
private:
|
private:
|
||||||
@ -191,6 +159,41 @@ private:
|
|||||||
std::vector<std::pair<T, UInt32>> samples;
|
std::vector<std::pair<T, UInt32>> samples;
|
||||||
UInt8 skip_degree{};
|
UInt8 skip_degree{};
|
||||||
|
|
||||||
|
void insertImpl(const T & v, const UInt32 hash)
|
||||||
|
{
|
||||||
|
/// @todo why + 1? I don't quite recall
|
||||||
|
while (samples.size() + 1 >= sample_count)
|
||||||
|
{
|
||||||
|
if (++skip_degree > detail::MAX_SKIP_DEGREE)
|
||||||
|
throw DB::Exception{"skip_degree exceeds maximum value", DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED};
|
||||||
|
thinOut();
|
||||||
|
}
|
||||||
|
|
||||||
|
samples.emplace_back(v, hash);
|
||||||
|
}
|
||||||
|
|
||||||
|
void thinOut()
|
||||||
|
{
|
||||||
|
auto size = samples.size();
|
||||||
|
for (size_t i = 0; i < size;)
|
||||||
|
{
|
||||||
|
if (!good(samples[i].second))
|
||||||
|
{
|
||||||
|
/// swap current element with the last one
|
||||||
|
std::swap(samples[size - 1], samples[i]);
|
||||||
|
--size;
|
||||||
|
}
|
||||||
|
else
|
||||||
|
++i;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (size != samples.size())
|
||||||
|
{
|
||||||
|
samples.resize(size);
|
||||||
|
sorted = false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
void sortIfNeeded()
|
void sortIfNeeded()
|
||||||
{
|
{
|
||||||
if (sorted)
|
if (sorted)
|
||||||
|
@ -1,6 +1,6 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Yandex/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
#include <Poco/Net/StreamSocket.h>
|
#include <Poco/Net/StreamSocket.h>
|
||||||
|
|
||||||
|
@ -1,6 +1,6 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <statdaemons/PoolBase.h>
|
#include <DB/Common/PoolBase.h>
|
||||||
|
|
||||||
#include <DB/Client/Connection.h>
|
#include <DB/Client/Connection.h>
|
||||||
|
|
||||||
|
@ -3,7 +3,7 @@
|
|||||||
#include <Poco/Net/NetException.h>
|
#include <Poco/Net/NetException.h>
|
||||||
#include <Poco/Net/DNS.h>
|
#include <Poco/Net/DNS.h>
|
||||||
|
|
||||||
#include <statdaemons/PoolWithFailoverBase.h>
|
#include <DB/Common/PoolWithFailoverBase.h>
|
||||||
|
|
||||||
#include <DB/Common/getFQDNOrHostName.h>
|
#include <DB/Common/getFQDNOrHostName.h>
|
||||||
#include <DB/Client/ConnectionPool.h>
|
#include <DB/Client/ConnectionPool.h>
|
||||||
|
@ -70,6 +70,9 @@ private:
|
|||||||
/// Внутренняя версия функции receivePacket без блокировки.
|
/// Внутренняя версия функции receivePacket без блокировки.
|
||||||
Connection::Packet receivePacketUnlocked();
|
Connection::Packet receivePacketUnlocked();
|
||||||
|
|
||||||
|
/// Внутренняя версия функции dumpAddresses без блокировки.
|
||||||
|
std::string dumpAddressesUnlocked() const;
|
||||||
|
|
||||||
/// Получить реплику, на которой можно прочитать данные.
|
/// Получить реплику, на которой можно прочитать данные.
|
||||||
ReplicaMap::iterator getReplicaForReading();
|
ReplicaMap::iterator getReplicaForReading();
|
||||||
|
|
||||||
|
@ -100,9 +100,9 @@ public:
|
|||||||
arenas.push_back(arena_);
|
arenas.push_back(arena_);
|
||||||
}
|
}
|
||||||
|
|
||||||
ColumnPtr convertToValues()
|
ColumnPtr convertToValues() const
|
||||||
{
|
{
|
||||||
IAggregateFunction * function = holder->func;
|
const IAggregateFunction * function = holder->func;
|
||||||
ColumnPtr res = function->getReturnType()->createColumn();
|
ColumnPtr res = function->getReturnType()->createColumn();
|
||||||
IColumn & column = *res;
|
IColumn & column = *res;
|
||||||
res->reserve(getData().size());
|
res->reserve(getData().size());
|
||||||
@ -182,6 +182,16 @@ public:
|
|||||||
throw Exception("Method insertDefault is not supported for ColumnAggregateFunction.", ErrorCodes::NOT_IMPLEMENTED);
|
throw Exception("Method insertDefault is not supported for ColumnAggregateFunction.", ErrorCodes::NOT_IMPLEMENTED);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override
|
||||||
|
{
|
||||||
|
throw Exception("Method serializeValueIntoArena is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
}
|
||||||
|
|
||||||
|
const char * deserializeAndInsertFromArena(const char * pos) override
|
||||||
|
{
|
||||||
|
throw Exception("Method deserializeAndInsertFromArena is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
}
|
||||||
|
|
||||||
size_t byteSize() const override
|
size_t byteSize() const override
|
||||||
{
|
{
|
||||||
return getData().size() * sizeof(getData()[0]);
|
return getData().size() * sizeof(getData()[0]);
|
||||||
|
@ -4,8 +4,9 @@
|
|||||||
|
|
||||||
#include <Poco/SharedPtr.h>
|
#include <Poco/SharedPtr.h>
|
||||||
|
|
||||||
#include <DB/Core/Exception.h>
|
#include <DB/Common/Exception.h>
|
||||||
#include <DB/Core/ErrorCodes.h>
|
#include <DB/Core/ErrorCodes.h>
|
||||||
|
#include <DB/Common/Arena.h>
|
||||||
|
|
||||||
#include <DB/Columns/IColumn.h>
|
#include <DB/Columns/IColumn.h>
|
||||||
#include <DB/Columns/ColumnsNumber.h>
|
#include <DB/Columns/ColumnsNumber.h>
|
||||||
@ -119,6 +120,33 @@ public:
|
|||||||
getOffsets().push_back((getOffsets().size() == 0 ? 0 : getOffsets().back()) + elems);
|
getOffsets().push_back((getOffsets().size() == 0 ? 0 : getOffsets().back()) + elems);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override
|
||||||
|
{
|
||||||
|
size_t array_size = sizeAt(n);
|
||||||
|
size_t offset = offsetAt(n);
|
||||||
|
|
||||||
|
char * pos = arena.allocContinue(sizeof(array_size), begin);
|
||||||
|
memcpy(pos, &array_size, sizeof(array_size));
|
||||||
|
|
||||||
|
size_t values_size = 0;
|
||||||
|
for (size_t i = 0; i < array_size; ++i)
|
||||||
|
values_size += data->serializeValueIntoArena(offset + i, arena, begin).size;
|
||||||
|
|
||||||
|
return StringRef(begin, sizeof(array_size) + values_size);
|
||||||
|
}
|
||||||
|
|
||||||
|
const char * deserializeAndInsertFromArena(const char * pos) override
|
||||||
|
{
|
||||||
|
size_t array_size = *reinterpret_cast<const size_t *>(pos);
|
||||||
|
pos += sizeof(array_size);
|
||||||
|
|
||||||
|
for (size_t i = 0; i < array_size; ++i)
|
||||||
|
pos = data->deserializeAndInsertFromArena(pos);
|
||||||
|
|
||||||
|
getOffsets().push_back((getOffsets().size() == 0 ? 0 : getOffsets().back()) + array_size);
|
||||||
|
return pos;
|
||||||
|
}
|
||||||
|
|
||||||
ColumnPtr cut(size_t start, size_t length) const override;
|
ColumnPtr cut(size_t start, size_t length) const override;
|
||||||
|
|
||||||
void insert(const Field & x) override
|
void insert(const Field & x) override
|
||||||
|
@ -3,7 +3,7 @@
|
|||||||
#include <Poco/SharedPtr.h>
|
#include <Poco/SharedPtr.h>
|
||||||
|
|
||||||
#include <DB/Core/Field.h>
|
#include <DB/Core/Field.h>
|
||||||
#include <DB/Core/Exception.h>
|
#include <DB/Common/Exception.h>
|
||||||
#include <DB/Core/ErrorCodes.h>
|
#include <DB/Core/ErrorCodes.h>
|
||||||
#include <DB/Columns/ColumnVector.h>
|
#include <DB/Columns/ColumnVector.h>
|
||||||
#include <DB/Columns/IColumn.h>
|
#include <DB/Columns/IColumn.h>
|
||||||
@ -89,6 +89,16 @@ public:
|
|||||||
|
|
||||||
void insertDefault() override { ++s; }
|
void insertDefault() override { ++s; }
|
||||||
|
|
||||||
|
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override
|
||||||
|
{
|
||||||
|
throw Exception("Method serializeValueIntoArena is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
}
|
||||||
|
|
||||||
|
const char * deserializeAndInsertFromArena(const char * pos) override
|
||||||
|
{
|
||||||
|
throw Exception("Method deserializeAndInsertFromArena is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
}
|
||||||
|
|
||||||
ColumnPtr filter(const Filter & filt) const override
|
ColumnPtr filter(const Filter & filt) const override
|
||||||
{
|
{
|
||||||
if (s != filt.size())
|
if (s != filt.size())
|
||||||
|
@ -3,6 +3,7 @@
|
|||||||
#include <string.h> // memcpy
|
#include <string.h> // memcpy
|
||||||
|
|
||||||
#include <DB/Common/PODArray.h>
|
#include <DB/Common/PODArray.h>
|
||||||
|
#include <DB/Common/Arena.h>
|
||||||
#include <DB/Columns/IColumn.h>
|
#include <DB/Columns/IColumn.h>
|
||||||
|
|
||||||
|
|
||||||
@ -111,6 +112,21 @@ public:
|
|||||||
chars.resize_fill(chars.size() + n);
|
chars.resize_fill(chars.size() + n);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
StringRef serializeValueIntoArena(size_t index, Arena & arena, char const *& begin) const override
|
||||||
|
{
|
||||||
|
auto pos = arena.allocContinue(n, begin);
|
||||||
|
memcpy(pos, &chars[n * index], n);
|
||||||
|
return StringRef(pos, n);
|
||||||
|
}
|
||||||
|
|
||||||
|
const char * deserializeAndInsertFromArena(const char * pos) override
|
||||||
|
{
|
||||||
|
size_t old_size = chars.size();
|
||||||
|
chars.resize(old_size + n);
|
||||||
|
memcpy(&chars[old_size], pos, n);
|
||||||
|
return pos + n;
|
||||||
|
}
|
||||||
|
|
||||||
int compareAt(size_t p1, size_t p2, const IColumn & rhs_, int nan_direction_hint) const override
|
int compareAt(size_t p1, size_t p2, const IColumn & rhs_, int nan_direction_hint) const override
|
||||||
{
|
{
|
||||||
const ColumnFixedString & rhs = static_cast<const ColumnFixedString &>(rhs_);
|
const ColumnFixedString & rhs = static_cast<const ColumnFixedString &>(rhs_);
|
||||||
|
@ -1,314 +0,0 @@
|
|||||||
#pragma once
|
|
||||||
|
|
||||||
#include <string.h> // memcpy
|
|
||||||
|
|
||||||
#include <Poco/SharedPtr.h>
|
|
||||||
|
|
||||||
#include <DB/Core/Defines.h>
|
|
||||||
#include <DB/Core/Exception.h>
|
|
||||||
#include <DB/Core/ErrorCodes.h>
|
|
||||||
|
|
||||||
#include <DB/Columns/IColumn.h>
|
|
||||||
#include <DB/Columns/ColumnsNumber.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
using Poco::SharedPtr;
|
|
||||||
|
|
||||||
/** Cтолбeц значений типа вложенная таблица.
|
|
||||||
* В памяти это выглядит, как столбцы вложенных типов одинковой длины, равной сумме размеров всех массивов с общим именем,
|
|
||||||
* и как общий для всех столбцов массив смещений, который позволяет достать каждый элемент.
|
|
||||||
*
|
|
||||||
* Не предназначен для возвращения результа в запросах SELECT. Предполагается, что для SELECT'а будут отдаваться
|
|
||||||
* столбцы вида ColumnArray, ссылающиеся на один массив Offset'ов и соответствующий массив с данными.
|
|
||||||
*
|
|
||||||
* Используется для сериализации вложенной таблицы.
|
|
||||||
*/
|
|
||||||
class ColumnNested final : public IColumn
|
|
||||||
{
|
|
||||||
public:
|
|
||||||
/** По индексу i находится смещение до начала i + 1 -го элемента. */
|
|
||||||
typedef ColumnVector<Offset_t> ColumnOffsets_t;
|
|
||||||
|
|
||||||
/** Создать пустой столбец вложенных таблиц, с типом значений, как в столбце nested_column */
|
|
||||||
explicit ColumnNested(Columns nested_columns, ColumnPtr offsets_column = nullptr)
|
|
||||||
: data(nested_columns), offsets(offsets_column)
|
|
||||||
{
|
|
||||||
if (!offsets_column)
|
|
||||||
{
|
|
||||||
offsets = new ColumnOffsets_t;
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
if (!typeid_cast<ColumnOffsets_t *>(&*offsets_column))
|
|
||||||
throw Exception("offsets_column must be a ColumnVector<UInt64>", ErrorCodes::ILLEGAL_COLUMN);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
std::string getName() const override
|
|
||||||
{
|
|
||||||
std::string res;
|
|
||||||
{
|
|
||||||
WriteBufferFromString out(res);
|
|
||||||
|
|
||||||
for (Columns::const_iterator it = data.begin(); it != data.end(); ++it)
|
|
||||||
{
|
|
||||||
if (it != data.begin())
|
|
||||||
writeCString(", ", out);
|
|
||||||
writeString((*it)->getName(), out);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return "ColumnNested(" + res + ")";
|
|
||||||
}
|
|
||||||
|
|
||||||
ColumnPtr cloneEmpty() const override
|
|
||||||
{
|
|
||||||
Columns res(data.size());
|
|
||||||
for (size_t i = 0; i < data.size(); ++i)
|
|
||||||
res[i] = data[i]->cloneEmpty();
|
|
||||||
return new ColumnNested(res);
|
|
||||||
}
|
|
||||||
|
|
||||||
size_t size() const override
|
|
||||||
{
|
|
||||||
return getOffsets().size();
|
|
||||||
}
|
|
||||||
|
|
||||||
Field operator[](size_t n) const override
|
|
||||||
{
|
|
||||||
throw Exception("Method operator[] is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
|
||||||
}
|
|
||||||
|
|
||||||
void get(size_t n, Field & res) const override
|
|
||||||
{
|
|
||||||
throw Exception("Method get is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
|
||||||
}
|
|
||||||
|
|
||||||
StringRef getDataAt(size_t n) const override
|
|
||||||
{
|
|
||||||
throw Exception("Method getDataAt is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
|
||||||
}
|
|
||||||
|
|
||||||
void insertData(const char * pos, size_t length) override
|
|
||||||
{
|
|
||||||
throw Exception("Method insertData is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
|
||||||
}
|
|
||||||
|
|
||||||
ColumnPtr cut(size_t start, size_t length) const override
|
|
||||||
{
|
|
||||||
if (length == 0)
|
|
||||||
return new ColumnNested(data);
|
|
||||||
|
|
||||||
if (start + length > getOffsets().size())
|
|
||||||
throw Exception("Parameter out of bound in ColumnNested::cut() method.",
|
|
||||||
ErrorCodes::PARAMETER_OUT_OF_BOUND);
|
|
||||||
|
|
||||||
size_t nested_offset = offsetAt(start);
|
|
||||||
size_t nested_length = getOffsets()[start + length - 1] - nested_offset;
|
|
||||||
|
|
||||||
ColumnNested * res_ = new ColumnNested(data);
|
|
||||||
ColumnPtr res = res_;
|
|
||||||
|
|
||||||
for (size_t i = 0; i < data.size(); ++i)
|
|
||||||
res_->data[i] = data[i]->cut(nested_offset, nested_length);
|
|
||||||
|
|
||||||
Offsets_t & res_offsets = res_->getOffsets();
|
|
||||||
|
|
||||||
if (start == 0)
|
|
||||||
{
|
|
||||||
res_offsets.assign(getOffsets().begin(), getOffsets().begin() + length);
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
res_offsets.resize(length);
|
|
||||||
|
|
||||||
for (size_t i = 0; i < length; ++i)
|
|
||||||
res_offsets[i] = getOffsets()[start + i] - nested_offset;
|
|
||||||
}
|
|
||||||
|
|
||||||
return res;
|
|
||||||
}
|
|
||||||
|
|
||||||
void insert(const Field & x) override
|
|
||||||
{
|
|
||||||
throw Exception("Method insert is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
|
||||||
}
|
|
||||||
|
|
||||||
void insertFrom(const IColumn & src_, size_t n) override
|
|
||||||
{
|
|
||||||
const ColumnNested & src = static_cast<const ColumnNested &>(src_);
|
|
||||||
|
|
||||||
if (data.size() != src.getData().size())
|
|
||||||
throw Exception("Number of columns in nested tables do not match.", ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH);
|
|
||||||
|
|
||||||
size_t size = src.sizeAt(n);
|
|
||||||
size_t offset = src.offsetAt(n);
|
|
||||||
|
|
||||||
for (size_t i = 0; i < data.size(); ++i)
|
|
||||||
{
|
|
||||||
if (data[i]->getName() != src.getData()[i]->getName())
|
|
||||||
throw Exception("Types of columns in nested tables do not match.", ErrorCodes::TYPE_MISMATCH);
|
|
||||||
|
|
||||||
for (size_t j = 0; j < size; ++j)
|
|
||||||
data[i]->insertFrom(*src.getData()[i], offset + j);
|
|
||||||
}
|
|
||||||
|
|
||||||
getOffsets().push_back((getOffsets().size() == 0 ? 0 : getOffsets().back()) + size);
|
|
||||||
}
|
|
||||||
|
|
||||||
void insertDefault() override
|
|
||||||
{
|
|
||||||
for (size_t i = 0; i < data.size(); ++i)
|
|
||||||
data[i]->insertDefault();
|
|
||||||
getOffsets().push_back(getOffsets().size() == 0 ? 1 : (getOffsets().back() + 1));
|
|
||||||
}
|
|
||||||
|
|
||||||
ColumnPtr filter(const Filter & filt) const override
|
|
||||||
{
|
|
||||||
size_t size = getOffsets().size();
|
|
||||||
if (size != filt.size())
|
|
||||||
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
|
||||||
|
|
||||||
if (size == 0)
|
|
||||||
return new ColumnNested(data);
|
|
||||||
|
|
||||||
/// Не слишком оптимально. Можно сделать специализацию для массивов известных типов.
|
|
||||||
Filter nested_filt(getOffsets().back());
|
|
||||||
for (size_t i = 0; i < size; ++i)
|
|
||||||
{
|
|
||||||
if (filt[i])
|
|
||||||
memset(&nested_filt[offsetAt(i)], 1, sizeAt(i));
|
|
||||||
else
|
|
||||||
memset(&nested_filt[offsetAt(i)], 0, sizeAt(i));
|
|
||||||
}
|
|
||||||
|
|
||||||
ColumnNested * res_ = new ColumnNested(data);
|
|
||||||
ColumnPtr res = res_;
|
|
||||||
for (size_t i = 0; i < data.size(); ++i)
|
|
||||||
res_->data[i] = data[i]->filter(nested_filt);
|
|
||||||
|
|
||||||
Offsets_t & res_offsets = res_->getOffsets();
|
|
||||||
res_offsets.reserve(size);
|
|
||||||
|
|
||||||
size_t current_offset = 0;
|
|
||||||
for (size_t i = 0; i < size; ++i)
|
|
||||||
{
|
|
||||||
if (filt[i])
|
|
||||||
{
|
|
||||||
current_offset += sizeAt(i);
|
|
||||||
res_offsets.push_back(current_offset);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return res;
|
|
||||||
}
|
|
||||||
|
|
||||||
ColumnPtr replicate(const Offsets_t & offsets) const override
|
|
||||||
{
|
|
||||||
throw Exception("Replication of ColumnNested is not implemented.", ErrorCodes::NOT_IMPLEMENTED);
|
|
||||||
}
|
|
||||||
|
|
||||||
ColumnPtr permute(const Permutation & perm, size_t limit) const override
|
|
||||||
{
|
|
||||||
size_t size = getOffsets().size();
|
|
||||||
if (size != perm.size())
|
|
||||||
throw Exception("Size of permutation doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
|
||||||
|
|
||||||
if (limit == 0)
|
|
||||||
limit = size;
|
|
||||||
else
|
|
||||||
limit = std::min(size, limit);
|
|
||||||
|
|
||||||
if (perm.size() < limit)
|
|
||||||
throw Exception("Size of permutation is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
|
||||||
|
|
||||||
if (limit == 0)
|
|
||||||
return new ColumnNested(data);
|
|
||||||
|
|
||||||
Permutation nested_perm(getOffsets().back());
|
|
||||||
|
|
||||||
Columns cloned_columns(data.size());
|
|
||||||
for (size_t i = 0; i < data.size(); ++i)
|
|
||||||
cloned_columns[i] = data[i]->cloneEmpty();
|
|
||||||
|
|
||||||
ColumnNested * res_ = new ColumnNested(cloned_columns);
|
|
||||||
ColumnPtr res = res_;
|
|
||||||
|
|
||||||
Offsets_t & res_offsets = res_->getOffsets();
|
|
||||||
res_offsets.resize(limit);
|
|
||||||
size_t current_offset = 0;
|
|
||||||
|
|
||||||
for (size_t i = 0; i < limit; ++i)
|
|
||||||
{
|
|
||||||
for (size_t j = 0; j < sizeAt(perm[i]); ++j)
|
|
||||||
nested_perm[current_offset + j] = offsetAt(perm[i]) + j;
|
|
||||||
current_offset += sizeAt(perm[i]);
|
|
||||||
res_offsets[i] = current_offset;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (current_offset != 0)
|
|
||||||
for (size_t i = 0; i < data.size(); ++i)
|
|
||||||
res_->data[i] = data[i]->permute(nested_perm, current_offset);
|
|
||||||
|
|
||||||
return res;
|
|
||||||
}
|
|
||||||
|
|
||||||
int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override
|
|
||||||
{
|
|
||||||
throw Exception("Method compareAt is not supported for ColumnNested.", ErrorCodes::NOT_IMPLEMENTED);
|
|
||||||
}
|
|
||||||
|
|
||||||
void getPermutation(bool reverse, size_t limit, Permutation & res) const override
|
|
||||||
{
|
|
||||||
throw Exception("Method getPermutation is not supported for ColumnNested.", ErrorCodes::NOT_IMPLEMENTED);
|
|
||||||
}
|
|
||||||
|
|
||||||
void reserve(size_t n) override
|
|
||||||
{
|
|
||||||
getOffsets().reserve(n);
|
|
||||||
for (Columns::iterator it = data.begin(); it != data.end(); ++it)
|
|
||||||
(*it)->reserve(n);
|
|
||||||
}
|
|
||||||
|
|
||||||
size_t byteSize() const override
|
|
||||||
{
|
|
||||||
size_t size = getOffsets().size() * sizeof(getOffsets()[0]);
|
|
||||||
for (Columns::const_iterator it = data.begin(); it != data.end(); ++it)
|
|
||||||
size += (*it)->byteSize();
|
|
||||||
return size;
|
|
||||||
}
|
|
||||||
|
|
||||||
void getExtremes(Field & min, Field & max) const override
|
|
||||||
{
|
|
||||||
throw Exception("Method getExtremes is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Более эффективные методы манипуляции */
|
|
||||||
Columns & getData() { return data; }
|
|
||||||
const Columns & getData() const { return data; }
|
|
||||||
|
|
||||||
Offsets_t & ALWAYS_INLINE getOffsets()
|
|
||||||
{
|
|
||||||
return static_cast<ColumnOffsets_t &>(*offsets.get()).getData();
|
|
||||||
}
|
|
||||||
|
|
||||||
const Offsets_t & ALWAYS_INLINE getOffsets() const
|
|
||||||
{
|
|
||||||
return static_cast<const ColumnOffsets_t &>(*offsets.get()).getData();
|
|
||||||
}
|
|
||||||
|
|
||||||
ColumnPtr & getOffsetsColumn() { return offsets; }
|
|
||||||
const ColumnPtr & getOffsetsColumn() const { return offsets; }
|
|
||||||
|
|
||||||
private:
|
|
||||||
Columns data;
|
|
||||||
ColumnPtr offsets;
|
|
||||||
|
|
||||||
size_t ALWAYS_INLINE offsetAt(size_t i) const { return i == 0 ? 0 : getOffsets()[i - 1]; }
|
|
||||||
size_t ALWAYS_INLINE sizeAt(size_t i) const { return i == 0 ? getOffsets()[0] : (getOffsets()[i] - getOffsets()[i - 1]); }
|
|
||||||
};
|
|
||||||
|
|
||||||
|
|
||||||
}
|
|
@ -7,6 +7,7 @@
|
|||||||
#include <DB/Columns/IColumn.h>
|
#include <DB/Columns/IColumn.h>
|
||||||
#include <DB/Common/Collator.h>
|
#include <DB/Common/Collator.h>
|
||||||
#include <DB/Common/PODArray.h>
|
#include <DB/Common/PODArray.h>
|
||||||
|
#include <DB/Common/Arena.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -115,6 +116,34 @@ public:
|
|||||||
offsets.push_back((offsets.size() == 0 ? 0 : offsets.back()) + length);
|
offsets.push_back((offsets.size() == 0 ? 0 : offsets.back()) + length);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override
|
||||||
|
{
|
||||||
|
size_t string_size = sizeAt(n);
|
||||||
|
size_t offset = offsetAt(n);
|
||||||
|
|
||||||
|
StringRef res;
|
||||||
|
res.size = sizeof(string_size) + string_size;
|
||||||
|
char * pos = arena.allocContinue(res.size, begin);
|
||||||
|
memcpy(pos, &string_size, sizeof(string_size));
|
||||||
|
memcpy(pos + sizeof(string_size), &chars[offset], string_size);
|
||||||
|
res.data = pos;
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
const char * deserializeAndInsertFromArena(const char * pos) override
|
||||||
|
{
|
||||||
|
size_t string_size = *reinterpret_cast<const size_t *>(pos);
|
||||||
|
pos += sizeof(string_size);
|
||||||
|
|
||||||
|
size_t old_size = chars.size();
|
||||||
|
chars.resize(old_size + string_size);
|
||||||
|
memcpy(&chars[old_size], pos, string_size);
|
||||||
|
|
||||||
|
offsets.push_back((offsets.size() == 0 ? 0 : offsets.back()) + string_size);
|
||||||
|
return pos + string_size;
|
||||||
|
}
|
||||||
|
|
||||||
ColumnPtr cut(size_t start, size_t length) const override
|
ColumnPtr cut(size_t start, size_t length) const override
|
||||||
{
|
{
|
||||||
if (length == 0)
|
if (length == 0)
|
||||||
|
@ -42,8 +42,8 @@ public:
|
|||||||
{
|
{
|
||||||
Array res;
|
Array res;
|
||||||
|
|
||||||
for (Columns::const_iterator it = columns.begin(); it != columns.end(); ++it)
|
for (const auto & column : columns)
|
||||||
res.push_back((**it)[n]);
|
res.push_back((*column)[n]);
|
||||||
|
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
@ -93,8 +93,26 @@ public:
|
|||||||
|
|
||||||
void insertDefault() override
|
void insertDefault() override
|
||||||
{
|
{
|
||||||
for (Columns::iterator it = columns.begin(); it != columns.end(); ++it)
|
for (auto & column : columns)
|
||||||
(*it)->insertDefault();
|
column->insertDefault();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override
|
||||||
|
{
|
||||||
|
size_t values_size = 0;
|
||||||
|
for (auto & column : columns)
|
||||||
|
values_size += column->serializeValueIntoArena(n, arena, begin).size;
|
||||||
|
|
||||||
|
return StringRef(begin, values_size);
|
||||||
|
}
|
||||||
|
|
||||||
|
const char * deserializeAndInsertFromArena(const char * pos) override
|
||||||
|
{
|
||||||
|
for (auto & column : columns)
|
||||||
|
pos = column->deserializeAndInsertFromArena(pos);
|
||||||
|
|
||||||
|
return pos;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -155,8 +173,8 @@ public:
|
|||||||
|
|
||||||
Less(const Columns & columns)
|
Less(const Columns & columns)
|
||||||
{
|
{
|
||||||
for (Columns::const_iterator it = columns.begin(); it != columns.end(); ++it)
|
for (const auto & column : columns)
|
||||||
plain_columns.push_back(&**it);
|
plain_columns.push_back(column.get());
|
||||||
}
|
}
|
||||||
|
|
||||||
bool operator() (size_t a, size_t b) const
|
bool operator() (size_t a, size_t b) const
|
||||||
@ -201,15 +219,15 @@ public:
|
|||||||
|
|
||||||
void reserve(size_t n) override
|
void reserve(size_t n) override
|
||||||
{
|
{
|
||||||
for (Columns::iterator it = columns.begin(); it != columns.end(); ++it)
|
for (auto & column : columns)
|
||||||
(*it)->reserve(n);
|
column->reserve(n);
|
||||||
}
|
}
|
||||||
|
|
||||||
size_t byteSize() const override
|
size_t byteSize() const override
|
||||||
{
|
{
|
||||||
size_t res = 0;
|
size_t res = 0;
|
||||||
for (Columns::const_iterator it = columns.begin(); it != columns.end(); ++it)
|
for (const auto & column : columns)
|
||||||
res += (*it)->byteSize();
|
res += column->byteSize();
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -2,8 +2,9 @@
|
|||||||
|
|
||||||
#include <string.h>
|
#include <string.h>
|
||||||
|
|
||||||
#include <DB/Core/Exception.h>
|
#include <DB/Common/Exception.h>
|
||||||
#include <DB/Core/ErrorCodes.h>
|
#include <DB/Core/ErrorCodes.h>
|
||||||
|
#include <DB/Common/Arena.h>
|
||||||
|
|
||||||
#include <DB/Columns/IColumn.h>
|
#include <DB/Columns/IColumn.h>
|
||||||
|
|
||||||
@ -150,6 +151,19 @@ public:
|
|||||||
data.push_back(T());
|
data.push_back(T());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override
|
||||||
|
{
|
||||||
|
auto pos = arena.allocContinue(sizeof(T), begin);
|
||||||
|
memcpy(pos, &data[n], sizeof(T));
|
||||||
|
return StringRef(pos, sizeof(T));
|
||||||
|
}
|
||||||
|
|
||||||
|
const char * deserializeAndInsertFromArena(const char * pos) override
|
||||||
|
{
|
||||||
|
data.push_back(*reinterpret_cast<const T *>(pos));
|
||||||
|
return pos + sizeof(T);
|
||||||
|
}
|
||||||
|
|
||||||
size_t byteSize() const override
|
size_t byteSize() const override
|
||||||
{
|
{
|
||||||
return data.size() * sizeof(data[0]);
|
return data.size() * sizeof(data[0]);
|
||||||
|
@ -6,7 +6,7 @@
|
|||||||
#include <DB/Common/typeid_cast.h>
|
#include <DB/Common/typeid_cast.h>
|
||||||
|
|
||||||
#include <DB/Core/Field.h>
|
#include <DB/Core/Field.h>
|
||||||
#include <DB/Core/Exception.h>
|
#include <DB/Common/Exception.h>
|
||||||
#include <DB/Core/ErrorCodes.h>
|
#include <DB/Core/ErrorCodes.h>
|
||||||
#include <DB/Core/StringRef.h>
|
#include <DB/Core/StringRef.h>
|
||||||
|
|
||||||
@ -22,6 +22,8 @@ typedef std::vector<ColumnPtr> Columns;
|
|||||||
typedef std::vector<IColumn *> ColumnPlainPtrs;
|
typedef std::vector<IColumn *> ColumnPlainPtrs;
|
||||||
typedef std::vector<const IColumn *> ConstColumnPlainPtrs;
|
typedef std::vector<const IColumn *> ConstColumnPlainPtrs;
|
||||||
|
|
||||||
|
class Arena;
|
||||||
|
|
||||||
|
|
||||||
/** Интерфейс для хранения столбцов значений в оперативке.
|
/** Интерфейс для хранения столбцов значений в оперативке.
|
||||||
*/
|
*/
|
||||||
@ -134,6 +136,20 @@ public:
|
|||||||
*/
|
*/
|
||||||
virtual void insertDefault() = 0;
|
virtual void insertDefault() = 0;
|
||||||
|
|
||||||
|
/** Сериализовать значение, расположив его в непрерывном куске памяти в Arena.
|
||||||
|
* Значение можно будет потом прочитать обратно. Используется для агрегации.
|
||||||
|
* Метод похож на getDataAt, но может работать для тех случаев,
|
||||||
|
* когда значение не однозначно соответствует какому-то уже существующему непрерывному куску памяти
|
||||||
|
* - например, для массива строк, чтобы получить однозначное соответствие, надо укладывать строки вместе с их размерами.
|
||||||
|
* Параметр begin - см. метод Arena::allocContinue.
|
||||||
|
*/
|
||||||
|
virtual StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const = 0;
|
||||||
|
|
||||||
|
/** Десериализовать значение, которое было сериализовано с помощью serializeValueIntoArena.
|
||||||
|
* Вернуть указатель на позицию после прочитанных данных.
|
||||||
|
*/
|
||||||
|
virtual const char * deserializeAndInsertFromArena(const char * pos) = 0;
|
||||||
|
|
||||||
/** Соединить столбец с одним или несколькими другими.
|
/** Соединить столбец с одним или несколькими другими.
|
||||||
* Используется при склейке маленьких блоков.
|
* Используется при склейке маленьких блоков.
|
||||||
*/
|
*/
|
||||||
|
@ -29,6 +29,16 @@ public:
|
|||||||
StringRef getDataAt(size_t n) const override { throw Exception("Method getDataAt is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); }
|
StringRef getDataAt(size_t n) const override { throw Exception("Method getDataAt is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); }
|
||||||
void insertData(const char * pos, size_t length) override { throw Exception("Method insertData is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); }
|
void insertData(const char * pos, size_t length) override { throw Exception("Method insertData is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); }
|
||||||
|
|
||||||
|
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override
|
||||||
|
{
|
||||||
|
throw Exception("Method serializeValueIntoArena is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
}
|
||||||
|
|
||||||
|
const char * deserializeAndInsertFromArena(const char * pos) override
|
||||||
|
{
|
||||||
|
throw Exception("Method deserializeAndInsertFromArena is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
}
|
||||||
|
|
||||||
void getExtremes(Field & min, Field & max) const override
|
void getExtremes(Field & min, Field & max) const override
|
||||||
{
|
{
|
||||||
throw Exception("Method getExtremes is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
throw Exception("Method getExtremes is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
52
dbms/include/DB/Common/AIO.h
Normal file
52
dbms/include/DB/Common/AIO.h
Normal file
@ -0,0 +1,52 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <linux/aio_abi.h>
|
||||||
|
#include <unistd.h>
|
||||||
|
#include <sys/syscall.h>
|
||||||
|
|
||||||
|
#include <boost/noncopyable.hpp>
|
||||||
|
|
||||||
|
#include <DB/Common/Exception.h>
|
||||||
|
|
||||||
|
|
||||||
|
/** Небольшие обёртки для асинхронного ввода-вывода.
|
||||||
|
*/
|
||||||
|
|
||||||
|
|
||||||
|
inline int io_setup(unsigned nr, aio_context_t *ctxp)
|
||||||
|
{
|
||||||
|
return syscall(__NR_io_setup, nr, ctxp);
|
||||||
|
}
|
||||||
|
|
||||||
|
inline int io_destroy(aio_context_t ctx)
|
||||||
|
{
|
||||||
|
return syscall(__NR_io_destroy, ctx);
|
||||||
|
}
|
||||||
|
|
||||||
|
inline int io_submit(aio_context_t ctx, long nr, struct iocb **iocbpp)
|
||||||
|
{
|
||||||
|
return syscall(__NR_io_submit, ctx, nr, iocbpp);
|
||||||
|
}
|
||||||
|
|
||||||
|
inline int io_getevents(aio_context_t ctx, long min_nr, long max_nr, io_event *events, struct timespec *timeout)
|
||||||
|
{
|
||||||
|
return syscall(__NR_io_getevents, ctx, min_nr, max_nr, events, timeout);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
struct AIOContext : private boost::noncopyable
|
||||||
|
{
|
||||||
|
aio_context_t ctx;
|
||||||
|
|
||||||
|
AIOContext(unsigned int nr_events = 128)
|
||||||
|
{
|
||||||
|
ctx = 0;
|
||||||
|
if (io_setup(nr_events, &ctx) < 0)
|
||||||
|
DB::throwFromErrno("io_setup failed");
|
||||||
|
}
|
||||||
|
|
||||||
|
~AIOContext()
|
||||||
|
{
|
||||||
|
io_destroy(ctx);
|
||||||
|
}
|
||||||
|
};
|
@ -5,7 +5,7 @@
|
|||||||
#include <sys/mman.h>
|
#include <sys/mman.h>
|
||||||
|
|
||||||
#include <DB/Common/MemoryTracker.h>
|
#include <DB/Common/MemoryTracker.h>
|
||||||
#include <DB/Core/Exception.h>
|
#include <DB/Common/Exception.h>
|
||||||
#include <DB/Core/ErrorCodes.h>
|
#include <DB/Core/ErrorCodes.h>
|
||||||
|
|
||||||
|
|
||||||
|
@ -4,7 +4,8 @@
|
|||||||
#include <memory>
|
#include <memory>
|
||||||
#include <vector>
|
#include <vector>
|
||||||
#include <Poco/SharedPtr.h>
|
#include <Poco/SharedPtr.h>
|
||||||
#include <Yandex/likely.h>
|
#include <common/likely.h>
|
||||||
|
#include <DB/Core/Defines.h>
|
||||||
#include <DB/Common/ProfileEvents.h>
|
#include <DB/Common/ProfileEvents.h>
|
||||||
#include <DB/Common/Allocator.h>
|
#include <DB/Common/Allocator.h>
|
||||||
|
|
||||||
@ -84,7 +85,7 @@ private:
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// Добавить следующий непрерывный кусок памяти размера не меньше заданного.
|
/// Добавить следующий непрерывный кусок памяти размера не меньше заданного.
|
||||||
void addChunk(size_t min_size)
|
void NO_INLINE addChunk(size_t min_size)
|
||||||
{
|
{
|
||||||
head = new Chunk(nextSize(min_size), head);
|
head = new Chunk(nextSize(min_size), head);
|
||||||
size_in_bytes += head->size();
|
size_in_bytes += head->size();
|
||||||
@ -127,16 +128,15 @@ public:
|
|||||||
*/
|
*/
|
||||||
char * allocContinue(size_t size, char const *& begin)
|
char * allocContinue(size_t size, char const *& begin)
|
||||||
{
|
{
|
||||||
if (unlikely(head->pos + size > head->end))
|
while (unlikely(head->pos + size > head->end))
|
||||||
{
|
{
|
||||||
char * prev_end = head->pos;
|
char * prev_end = head->pos;
|
||||||
addChunk(size);
|
addChunk(size);
|
||||||
|
|
||||||
if (begin)
|
if (begin)
|
||||||
{
|
|
||||||
begin = insert(begin, prev_end - begin);
|
begin = insert(begin, prev_end - begin);
|
||||||
return allocContinue(size, begin);
|
else
|
||||||
}
|
break;
|
||||||
}
|
}
|
||||||
|
|
||||||
char * res = head->pos;
|
char * res = head->pos;
|
||||||
|
@ -2,7 +2,7 @@
|
|||||||
|
|
||||||
#include <cstddef>
|
#include <cstddef>
|
||||||
|
|
||||||
#include <Yandex/likely.h>
|
#include <common/likely.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
@ -3,10 +3,10 @@
|
|||||||
#include <unicode/ucol.h>
|
#include <unicode/ucol.h>
|
||||||
|
|
||||||
#include <DB/Core/ErrorCodes.h>
|
#include <DB/Core/ErrorCodes.h>
|
||||||
#include <DB/Core/Exception.h>
|
#include <DB/Common/Exception.h>
|
||||||
#include <DB/IO/WriteHelpers.h>
|
#include <DB/IO/WriteHelpers.h>
|
||||||
|
|
||||||
#include <Yandex/Common.h>
|
#include <common/Common.h>
|
||||||
|
|
||||||
#include <Poco/String.h>
|
#include <Poco/String.h>
|
||||||
|
|
||||||
|
@ -2,7 +2,7 @@
|
|||||||
|
|
||||||
#include <DB/Common/HashTable/SmallTable.h>
|
#include <DB/Common/HashTable/SmallTable.h>
|
||||||
#include <DB/Common/HashTable/HashSet.h>
|
#include <DB/Common/HashTable/HashSet.h>
|
||||||
#include <statdaemons/HyperLogLogCounter.h>
|
#include <DB/Common/HyperLogLogCounter.h>
|
||||||
#include <DB/Core/Defines.h>
|
#include <DB/Core/Defines.h>
|
||||||
|
|
||||||
|
|
||||||
|
65
dbms/include/DB/Common/ConfigProcessor.h
Normal file
65
dbms/include/DB/Common/ConfigProcessor.h
Normal file
@ -0,0 +1,65 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <string>
|
||||||
|
#include <Poco/DOM/Document.h>
|
||||||
|
#include <Poco/DOM/DOMParser.h>
|
||||||
|
#include <Poco/DOM/DOMWriter.h>
|
||||||
|
#include <Poco/DOM/NodeList.h>
|
||||||
|
#include <Poco/DOM/NamedNodeMap.h>
|
||||||
|
#include <Poco/AutoPtr.h>
|
||||||
|
#include <Poco/File.h>
|
||||||
|
#include <Poco/Path.h>
|
||||||
|
#include <Poco/DirectoryIterator.h>
|
||||||
|
#include <Poco/ConsoleChannel.h>
|
||||||
|
#include <Poco/Util/AbstractConfiguration.h>
|
||||||
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
|
typedef Poco::AutoPtr<Poco::Util::AbstractConfiguration> ConfigurationPtr;
|
||||||
|
typedef Poco::AutoPtr<Poco::XML::Document> XMLDocumentPtr;
|
||||||
|
|
||||||
|
class ConfigProcessor
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
using Substitutions = std::vector<std::pair<std::string, std::string> >;
|
||||||
|
|
||||||
|
/// log_to_console нужно использовать, если система логгирования еще не инициализирована.
|
||||||
|
ConfigProcessor(bool throw_on_bad_incl = false, bool log_to_console = false, const Substitutions & substitutions = Substitutions());
|
||||||
|
|
||||||
|
/** Выполняет подстановки в конфиге и возвращает XML-документ.
|
||||||
|
*
|
||||||
|
* Пусть в качестве path передана "/path/file.xml"
|
||||||
|
* 1) Объединяем xml-дерево из /path/file.xml со всеми деревьями из файлов /path/{conf,file}.d/ *.{conf,xml}
|
||||||
|
* Если у элемента есть атрибут replace, заменяем на него подходящий элемент.
|
||||||
|
* Если у элемента есть атрибут remove, удаляем подходящий элемент.
|
||||||
|
* Иначе объединяем детей рекурсивно.
|
||||||
|
* 2) Берем из конфига путь к файлу, из которого будем делать подстановки: <include_from>/path2/metrika.xml</include_from>.
|
||||||
|
* Если путь не указан, используем /etc/metrika.xml
|
||||||
|
* 3) Заменяем элементы вида "<foo incl="bar"/>" на "<foo>содержимое элемента yandex.bar из metrika.xml</foo>"
|
||||||
|
* 4) Заменяет "<layer/>" на "<layer>номер слоя из имени хоста</layer>"
|
||||||
|
*/
|
||||||
|
XMLDocumentPtr processConfig(const std::string & path);
|
||||||
|
|
||||||
|
/** Делает processConfig и создает из результата Poco::Util::XMLConfiguration.
|
||||||
|
* Еще сохраняет результат в файл по пути, полученному из path приписыванием строки "-preprocessed" к имени файла.
|
||||||
|
*/
|
||||||
|
ConfigurationPtr loadConfig(const std::string & path);
|
||||||
|
|
||||||
|
private:
|
||||||
|
Logger * log;
|
||||||
|
Poco::AutoPtr<Poco::Channel> channel_ptr;
|
||||||
|
bool throw_on_bad_incl;
|
||||||
|
Substitutions substitutions;
|
||||||
|
|
||||||
|
typedef XMLDocumentPtr DocumentPtr;
|
||||||
|
typedef Poco::AutoPtr<Poco::XML::Node> NodePtr;
|
||||||
|
|
||||||
|
void mergeRecursive(DocumentPtr config, Poco::XML::Node * config_node, Poco::XML::Node * with_node);
|
||||||
|
|
||||||
|
void merge(DocumentPtr config, DocumentPtr with);
|
||||||
|
|
||||||
|
std::string layerFromHost();
|
||||||
|
|
||||||
|
void doIncludesRecursive(DocumentPtr config, DocumentPtr include_from, Poco::XML::Node * node);
|
||||||
|
|
||||||
|
void doIncludes(DocumentPtr config, DocumentPtr include_from);
|
||||||
|
};
|
188
dbms/include/DB/Common/CounterInFile.h
Normal file
188
dbms/include/DB/Common/CounterInFile.h
Normal file
@ -0,0 +1,188 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <fcntl.h>
|
||||||
|
#include <sys/file.h>
|
||||||
|
|
||||||
|
#include <string>
|
||||||
|
#include <iostream>
|
||||||
|
|
||||||
|
#include <Poco/File.h>
|
||||||
|
#include <Poco/Exception.h>
|
||||||
|
#include <Poco/Mutex.h>
|
||||||
|
#include <Poco/ScopedLock.h>
|
||||||
|
|
||||||
|
#include <boost/function.hpp>
|
||||||
|
|
||||||
|
#include <DB/Common/Exception.h>
|
||||||
|
#include <DB/IO/ReadBufferFromFileDescriptor.h>
|
||||||
|
#include <DB/IO/WriteBufferFromFileDescriptor.h>
|
||||||
|
#include <DB/IO/ReadHelpers.h>
|
||||||
|
#include <DB/IO/WriteHelpers.h>
|
||||||
|
|
||||||
|
#include <common/Common.h>
|
||||||
|
|
||||||
|
#define SMALL_READ_WRITE_BUFFER_SIZE 16
|
||||||
|
|
||||||
|
|
||||||
|
/** Хранит в файле число.
|
||||||
|
* Предназначен для редких вызовов (не рассчитан на производительность).
|
||||||
|
*/
|
||||||
|
class CounterInFile
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
/// path - имя файла, включая путь
|
||||||
|
CounterInFile(const std::string & path_) : path(path_) {}
|
||||||
|
|
||||||
|
/** Добавить delta к числу в файле и вернуть новое значение.
|
||||||
|
* Если параметр create_if_need не установлен в true, то
|
||||||
|
* в файле уже должно быть записано какое-нибудь число (если нет - создайте файл вручную с нулём).
|
||||||
|
*
|
||||||
|
* Для защиты от race condition-ов между разными процессами, используются файловые блокировки.
|
||||||
|
* (Но при первом создании файла race condition возможен, так что лучше создать файл заранее.)
|
||||||
|
*
|
||||||
|
* locked_callback вызывается при заблокированном файле со счетчиком. В него передается новое значение.
|
||||||
|
* locked_callback можно использовать, чтобы делать что-нибудь атомарно с увеличением счетчика (например, переименовывать файлы).
|
||||||
|
*/
|
||||||
|
template <typename Callback>
|
||||||
|
Int64 add(Int64 delta, Callback && locked_callback, bool create_if_need = false)
|
||||||
|
{
|
||||||
|
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||||
|
|
||||||
|
Int64 res = -1;
|
||||||
|
|
||||||
|
bool file_doesnt_exists = !Poco::File(path).exists();
|
||||||
|
if (file_doesnt_exists && !create_if_need)
|
||||||
|
{
|
||||||
|
throw Poco::Exception("File " + path + " does not exist. "
|
||||||
|
"You must create it manulally with appropriate value or 0 for first start.");
|
||||||
|
}
|
||||||
|
|
||||||
|
int fd = open(path.c_str(), O_RDWR | O_CREAT, 0666);
|
||||||
|
if (-1 == fd)
|
||||||
|
DB::throwFromErrno("Cannot open file " + path);
|
||||||
|
|
||||||
|
try
|
||||||
|
{
|
||||||
|
int flock_ret = flock(fd, LOCK_EX);
|
||||||
|
if (-1 == flock_ret)
|
||||||
|
DB::throwFromErrno("Cannot lock file " + path);
|
||||||
|
|
||||||
|
if (!file_doesnt_exists)
|
||||||
|
{
|
||||||
|
DB::ReadBufferFromFileDescriptor rb(fd, SMALL_READ_WRITE_BUFFER_SIZE);
|
||||||
|
try
|
||||||
|
{
|
||||||
|
DB::readIntText(res, rb);
|
||||||
|
}
|
||||||
|
catch (const DB::Exception & e)
|
||||||
|
{
|
||||||
|
/// Более понятное сообщение об ошибке.
|
||||||
|
if (e.code() == DB::ErrorCodes::CANNOT_READ_ALL_DATA || e.code() == DB::ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF)
|
||||||
|
throw DB::Exception("File " + path + " is empty. You must fill it manually with appropriate value.", e.code());
|
||||||
|
else
|
||||||
|
throw;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
else
|
||||||
|
res = 0;
|
||||||
|
|
||||||
|
if (delta || file_doesnt_exists)
|
||||||
|
{
|
||||||
|
res += delta;
|
||||||
|
|
||||||
|
DB::WriteBufferFromFileDescriptor wb(fd, SMALL_READ_WRITE_BUFFER_SIZE);
|
||||||
|
wb.seek(0);
|
||||||
|
wb.truncate();
|
||||||
|
DB::writeIntText(res, wb);
|
||||||
|
DB::writeChar('\n', wb);
|
||||||
|
wb.sync();
|
||||||
|
}
|
||||||
|
|
||||||
|
locked_callback(res);
|
||||||
|
}
|
||||||
|
catch (...)
|
||||||
|
{
|
||||||
|
close(fd);
|
||||||
|
throw;
|
||||||
|
}
|
||||||
|
|
||||||
|
close(fd);
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
Int64 add(Int64 delta, bool create_if_need = false)
|
||||||
|
{
|
||||||
|
return add(delta, &CounterInFile::doNothing, create_if_need);
|
||||||
|
}
|
||||||
|
|
||||||
|
const std::string & getPath() const
|
||||||
|
{
|
||||||
|
return path;
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Изменить путь к файлу.
|
||||||
|
void setPath(std::string path_)
|
||||||
|
{
|
||||||
|
path = path_;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Не thread-safe и не синхронизирован между процессами.
|
||||||
|
void fixIfBroken(UInt64 value)
|
||||||
|
{
|
||||||
|
bool file_exists = Poco::File(path).exists();
|
||||||
|
|
||||||
|
int fd = open(path.c_str(), O_RDWR | O_CREAT, 0666);
|
||||||
|
if (-1 == fd)
|
||||||
|
DB::throwFromErrno("Cannot open file " + path);
|
||||||
|
|
||||||
|
try
|
||||||
|
{
|
||||||
|
bool broken = true;
|
||||||
|
|
||||||
|
if (file_exists)
|
||||||
|
{
|
||||||
|
DB::ReadBufferFromFileDescriptor rb(fd, SMALL_READ_WRITE_BUFFER_SIZE);
|
||||||
|
try
|
||||||
|
{
|
||||||
|
UInt64 current_value;
|
||||||
|
DB::readIntText(current_value, rb);
|
||||||
|
char c;
|
||||||
|
DB::readChar(c, rb);
|
||||||
|
if (rb.count() > 0 && c == '\n' && rb.eof())
|
||||||
|
broken = false;
|
||||||
|
}
|
||||||
|
catch (const DB::Exception & e)
|
||||||
|
{
|
||||||
|
if (e.code() != DB::ErrorCodes::CANNOT_READ_ALL_DATA && e.code() != DB::ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF)
|
||||||
|
throw;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (broken)
|
||||||
|
{
|
||||||
|
DB::WriteBufferFromFileDescriptor wb(fd, SMALL_READ_WRITE_BUFFER_SIZE);
|
||||||
|
wb.seek(0);
|
||||||
|
wb.truncate();
|
||||||
|
DB::writeIntText(value, wb);
|
||||||
|
DB::writeChar('\n', wb);
|
||||||
|
wb.sync();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (...)
|
||||||
|
{
|
||||||
|
close(fd);
|
||||||
|
throw;
|
||||||
|
}
|
||||||
|
|
||||||
|
close(fd);
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
std::string path;
|
||||||
|
Poco::FastMutex mutex;
|
||||||
|
|
||||||
|
static void doNothing(UInt64 a) {}
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
#undef SMALL_READ_WRITE_BUFFER_SIZE
|
113
dbms/include/DB/Common/Exception.h
Normal file
113
dbms/include/DB/Common/Exception.h
Normal file
@ -0,0 +1,113 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <cerrno>
|
||||||
|
#include <vector>
|
||||||
|
|
||||||
|
#include <Poco/Exception.h>
|
||||||
|
#include <Poco/SharedPtr.h>
|
||||||
|
|
||||||
|
#include <DB/Common/StackTrace.h>
|
||||||
|
|
||||||
|
namespace Poco { class Logger; }
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
class Exception : public Poco::Exception
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
Exception(int code = 0) : Poco::Exception(code) {}
|
||||||
|
Exception(const std::string & msg, int code = 0) : Poco::Exception(msg, code) {}
|
||||||
|
Exception(const std::string & msg, const std::string & arg, int code = 0) : Poco::Exception(msg, arg, code) {}
|
||||||
|
Exception(const std::string & msg, const Exception & exc, int code = 0) : Poco::Exception(msg, exc, code), trace(exc.trace) {}
|
||||||
|
Exception(const Exception & exc) : Poco::Exception(exc), trace(exc.trace) {}
|
||||||
|
explicit Exception(const Poco::Exception & exc) : Poco::Exception(exc.displayText()) {}
|
||||||
|
~Exception() throw() override {}
|
||||||
|
Exception & operator = (const Exception & exc)
|
||||||
|
{
|
||||||
|
Poco::Exception::operator=(exc);
|
||||||
|
trace = exc.trace;
|
||||||
|
return *this;
|
||||||
|
}
|
||||||
|
const char * name() const throw() override { return "DB::Exception"; }
|
||||||
|
const char * className() const throw() override { return "DB::Exception"; }
|
||||||
|
DB::Exception * clone() const override { return new DB::Exception(*this); }
|
||||||
|
void rethrow() const override { throw *this; }
|
||||||
|
|
||||||
|
/// Дописать к существующему сообщению что-нибудь ещё.
|
||||||
|
void addMessage(const std::string & arg) { extendedMessage(arg); }
|
||||||
|
|
||||||
|
const StackTrace & getStackTrace() const { return trace; }
|
||||||
|
|
||||||
|
private:
|
||||||
|
StackTrace trace;
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
/// Содержит дополнительный член saved_errno. См. функцию throwFromErrno.
|
||||||
|
class ErrnoException : public Exception
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
ErrnoException(int code = 0, int saved_errno_ = 0)
|
||||||
|
: Exception(code), saved_errno(saved_errno_) {}
|
||||||
|
ErrnoException(const std::string & msg, int code = 0, int saved_errno_ = 0)
|
||||||
|
: Exception(msg, code), saved_errno(saved_errno_) {}
|
||||||
|
ErrnoException(const std::string & msg, const std::string & arg, int code = 0, int saved_errno_ = 0)
|
||||||
|
: Exception(msg, arg, code), saved_errno(saved_errno_) {}
|
||||||
|
ErrnoException(const std::string & msg, const Exception & exc, int code = 0, int saved_errno_ = 0)
|
||||||
|
: Exception(msg, exc, code), saved_errno(saved_errno_) {}
|
||||||
|
ErrnoException(const ErrnoException & exc)
|
||||||
|
: Exception(exc), saved_errno(exc.saved_errno) {}
|
||||||
|
|
||||||
|
int getErrno() const { return saved_errno; }
|
||||||
|
|
||||||
|
private:
|
||||||
|
int saved_errno;
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
typedef std::vector<std::exception_ptr> Exceptions;
|
||||||
|
|
||||||
|
|
||||||
|
void throwFromErrno(const std::string & s, int code = 0, int the_errno = errno);
|
||||||
|
|
||||||
|
|
||||||
|
/** Попробовать записать исключение в лог (и забыть про него).
|
||||||
|
* Можно использовать в деструкторах в блоке catch (...).
|
||||||
|
*/
|
||||||
|
void tryLogCurrentException(const char * log_name, const std::string & start_of_message = "");
|
||||||
|
void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message = "");
|
||||||
|
|
||||||
|
std::string getCurrentExceptionMessage(bool with_stacktrace);
|
||||||
|
|
||||||
|
|
||||||
|
void tryLogException(std::exception_ptr e, const char * log_name, const std::string & start_of_message = "");
|
||||||
|
void tryLogException(std::exception_ptr e, Poco::Logger * logger, const std::string & start_of_message = "");
|
||||||
|
|
||||||
|
std::string getExceptionMessage(std::exception_ptr e, bool with_stacktrace);
|
||||||
|
|
||||||
|
|
||||||
|
void rethrowFirstException(Exceptions & exceptions);
|
||||||
|
|
||||||
|
Poco::SharedPtr<Poco::Exception> convertCurrentException();
|
||||||
|
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
typename std::enable_if<std::is_pointer<T>::value, T>::type exception_cast(std::exception_ptr e)
|
||||||
|
{
|
||||||
|
try
|
||||||
|
{
|
||||||
|
std::rethrow_exception(e);
|
||||||
|
}
|
||||||
|
catch (typename std::remove_pointer<T>::type & concrete)
|
||||||
|
{
|
||||||
|
return &concrete;
|
||||||
|
}
|
||||||
|
catch (...)
|
||||||
|
{
|
||||||
|
return nullptr;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -14,7 +14,7 @@
|
|||||||
#include <Poco/Net/HTMLForm.h>
|
#include <Poco/Net/HTMLForm.h>
|
||||||
#include <Poco/Net/PartHandler.h>
|
#include <Poco/Net/PartHandler.h>
|
||||||
#include <Poco/Net/MessageHeader.h>
|
#include <Poco/Net/MessageHeader.h>
|
||||||
#include <statdaemons/HTMLForm.h>
|
#include <DB/Common/HTMLForm.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -167,7 +167,7 @@ public:
|
|||||||
|
|
||||||
ExternalTablesHandler(Context & context_, Poco::Net::NameValueCollection params_) : context(context_), params(params_) { }
|
ExternalTablesHandler(Context & context_, Poco::Net::NameValueCollection params_) : context(context_), params(params_) { }
|
||||||
|
|
||||||
void handlePart(const Poco::Net::MessageHeader& header, std::istream& stream)
|
void handlePart(const Poco::Net::MessageHeader & header, std::istream & stream)
|
||||||
{
|
{
|
||||||
/// Буфер инициализируется здесь, а не в виртуальной функции initReadBuffer
|
/// Буфер инициализируется здесь, а не в виртуальной функции initReadBuffer
|
||||||
read_buffer.reset(new ReadBufferFromIStream(stream));
|
read_buffer.reset(new ReadBufferFromIStream(stream));
|
||||||
@ -194,7 +194,7 @@ public:
|
|||||||
NamesAndTypesListPtr columns = new NamesAndTypesList(sample_block.getColumnsList());
|
NamesAndTypesListPtr columns = new NamesAndTypesList(sample_block.getColumnsList());
|
||||||
StoragePtr storage = StorageMemory::create(data.second, columns);
|
StoragePtr storage = StorageMemory::create(data.second, columns);
|
||||||
context.addExternalTable(data.second, storage);
|
context.addExternalTable(data.second, storage);
|
||||||
BlockOutputStreamPtr output = storage->write(ASTPtr());
|
BlockOutputStreamPtr output = storage->write(ASTPtr(), context.getSettingsRef());
|
||||||
|
|
||||||
/// Записываем данные
|
/// Записываем данные
|
||||||
data.first->readPrefix();
|
data.first->readPrefix();
|
||||||
|
@ -1,16 +1,9 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Yandex/logger_useful.h>
|
|
||||||
#include <DB/Columns/IColumn.h>
|
|
||||||
#include <Poco/AutoPtr.h>
|
|
||||||
#include <Poco/Util/XMLConfiguration.h>
|
|
||||||
#include <Poco/Path.h>
|
|
||||||
#include <string>
|
#include <string>
|
||||||
|
#include <common/logger_useful.h>
|
||||||
#include <Poco/File.h>
|
#include <Poco/File.h>
|
||||||
#include <DB/Common/escapeForFileName.h>
|
|
||||||
|
|
||||||
#include <boost/property_tree/ptree.hpp>
|
|
||||||
#include <boost/property_tree/json_parser.hpp>
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -18,94 +11,35 @@ namespace DB
|
|||||||
/// хранит размеры всех столбцов, и может проверять не побились ли столбцы
|
/// хранит размеры всех столбцов, и может проверять не побились ли столбцы
|
||||||
class FileChecker
|
class FileChecker
|
||||||
{
|
{
|
||||||
|
private:
|
||||||
|
/// Имя файла -> размер.
|
||||||
|
using Map = std::map<std::string, size_t>;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
FileChecker(const std::string & file_info_path_) :
|
|
||||||
files_info_path(file_info_path_), files_info(), log(&Logger::get("FileChecker"))
|
|
||||||
{
|
|
||||||
Poco::Path path(files_info_path);
|
|
||||||
tmp_files_info_path = path.parent().toString() + "tmp_" + path.getFileName();
|
|
||||||
|
|
||||||
if (Poco::File(files_info_path).exists())
|
|
||||||
boost::property_tree::read_json(files_info_path, files_info);
|
|
||||||
}
|
|
||||||
|
|
||||||
void setPath(const std::string & file_info_path_)
|
|
||||||
{
|
|
||||||
files_info_path = file_info_path_;
|
|
||||||
}
|
|
||||||
|
|
||||||
using Files = std::vector<Poco::File>;
|
using Files = std::vector<Poco::File>;
|
||||||
|
|
||||||
void update(const Poco::File & file)
|
FileChecker(const std::string & file_info_path_);
|
||||||
{
|
void setPath(const std::string & file_info_path_);
|
||||||
updateTree(file);
|
void update(const Poco::File & file);
|
||||||
saveTree();
|
void update(const Files::const_iterator & begin, const Files::const_iterator & end);
|
||||||
}
|
|
||||||
|
|
||||||
void update(const Files::iterator & begin, const Files::iterator & end)
|
|
||||||
{
|
|
||||||
for (auto it = begin; it != end; ++it)
|
|
||||||
updateTree(*it);
|
|
||||||
saveTree();
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Проверяем файлы, параметры которых указаны в sizes.json
|
/// Проверяем файлы, параметры которых указаны в sizes.json
|
||||||
bool check() const
|
bool check() const;
|
||||||
{
|
|
||||||
bool correct = true;
|
|
||||||
if (!files_info.empty())
|
|
||||||
for (auto & node : files_info.get_child("yandex"))
|
|
||||||
{
|
|
||||||
std::string filename = unescapeForFileName(node.first);
|
|
||||||
size_t expected_size = std::stoull(node.second.template get<std::string>("size"));
|
|
||||||
|
|
||||||
Poco::File file(Poco::Path(files_info_path).parent().toString() + "/" + filename);
|
|
||||||
if (!file.exists())
|
|
||||||
{
|
|
||||||
LOG_ERROR(log, "File " << file.path() << " doesn't exist");
|
|
||||||
correct = false;
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
|
|
||||||
size_t real_size = file.getSize();
|
|
||||||
if (real_size != expected_size)
|
|
||||||
{
|
|
||||||
LOG_ERROR(log, "Size of " << file.path() << " is wrong. Size is " << real_size << " but should be " << expected_size);
|
|
||||||
correct = false;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return correct;
|
|
||||||
}
|
|
||||||
|
|
||||||
private:
|
private:
|
||||||
void updateTree(const Poco::File & file)
|
void initialize();
|
||||||
{
|
void updateImpl(const Poco::File & file);
|
||||||
files_info.put(std::string("yandex.") + escapeForFileName(Poco::Path(file.path()).getFileName()) + ".size", std::to_string(file.getSize()));
|
void save() const;
|
||||||
}
|
void load(Map & map) const;
|
||||||
|
|
||||||
void saveTree()
|
|
||||||
{
|
|
||||||
boost::property_tree::write_json(tmp_files_info_path, files_info, std::locale());
|
|
||||||
|
|
||||||
Poco::File current_file(files_info_path);
|
|
||||||
|
|
||||||
if (current_file.exists())
|
|
||||||
{
|
|
||||||
std::string old_file_name = files_info_path + ".old";
|
|
||||||
current_file.renameTo(old_file_name);
|
|
||||||
Poco::File(tmp_files_info_path).renameTo(files_info_path);
|
|
||||||
Poco::File(old_file_name).remove();
|
|
||||||
}
|
|
||||||
else
|
|
||||||
Poco::File(tmp_files_info_path).renameTo(files_info_path);
|
|
||||||
}
|
|
||||||
|
|
||||||
std::string files_info_path;
|
std::string files_info_path;
|
||||||
std::string tmp_files_info_path;
|
std::string tmp_files_info_path;
|
||||||
|
|
||||||
using PropertyTree = boost::property_tree::ptree;
|
/// Данные из файла читаются лениво.
|
||||||
PropertyTree files_info;
|
Map map;
|
||||||
|
bool initialized = false;
|
||||||
|
|
||||||
Logger * log;
|
Logger * log = &Logger::get("FileChecker");
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
26
dbms/include/DB/Common/HTMLForm.h
Normal file
26
dbms/include/DB/Common/HTMLForm.h
Normal file
@ -0,0 +1,26 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <sstream>
|
||||||
|
#include <Poco/Net/HTMLForm.h>
|
||||||
|
#include <Poco/Net/HTTPRequest.h>
|
||||||
|
#include <Poco/URI.h>
|
||||||
|
|
||||||
|
|
||||||
|
/** Почему-то при методе POST, Poco::Net::HTMLForm не считывает параметры из URL, а считывает только из тела.
|
||||||
|
* Этот помошник позволяет считывать параметры только из URL.
|
||||||
|
*/
|
||||||
|
struct HTMLForm : public Poco::Net::HTMLForm
|
||||||
|
{
|
||||||
|
HTMLForm(Poco::Net::HTTPRequest & request)
|
||||||
|
{
|
||||||
|
Poco::URI uri(request.getURI());
|
||||||
|
std::istringstream istr(uri.getRawQuery());
|
||||||
|
readUrl(istr);
|
||||||
|
}
|
||||||
|
|
||||||
|
HTMLForm(Poco::URI & uri)
|
||||||
|
{
|
||||||
|
std::istringstream istr(uri.getRawQuery());
|
||||||
|
readUrl(istr);
|
||||||
|
}
|
||||||
|
};
|
@ -9,13 +9,13 @@
|
|||||||
|
|
||||||
#include <boost/noncopyable.hpp>
|
#include <boost/noncopyable.hpp>
|
||||||
|
|
||||||
#include <Yandex/likely.h>
|
#include <common/likely.h>
|
||||||
|
|
||||||
#include <stats/IntHash.h>
|
#include <stats/IntHash.h>
|
||||||
|
|
||||||
#include <DB/Core/Defines.h>
|
#include <DB/Core/Defines.h>
|
||||||
#include <DB/Core/Types.h>
|
#include <DB/Core/Types.h>
|
||||||
#include <DB/Core/Exception.h>
|
#include <DB/Common/Exception.h>
|
||||||
#include <DB/Core/ErrorCodes.h>
|
#include <DB/Core/ErrorCodes.h>
|
||||||
|
|
||||||
#include <DB/IO/WriteBuffer.h>
|
#include <DB/IO/WriteBuffer.h>
|
||||||
@ -29,7 +29,7 @@
|
|||||||
#ifdef DBMS_HASH_MAP_DEBUG_RESIZES
|
#ifdef DBMS_HASH_MAP_DEBUG_RESIZES
|
||||||
#include <iostream>
|
#include <iostream>
|
||||||
#include <iomanip>
|
#include <iomanip>
|
||||||
#include <statdaemons/Stopwatch.h>
|
#include <DB/Common/Stopwatch.h>
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
|
|
||||||
|
@ -5,7 +5,7 @@
|
|||||||
#include <sys/mman.h>
|
#include <sys/mman.h>
|
||||||
|
|
||||||
#include <DB/Common/MemoryTracker.h>
|
#include <DB/Common/MemoryTracker.h>
|
||||||
#include <DB/Core/Exception.h>
|
#include <DB/Common/Exception.h>
|
||||||
#include <DB/Core/ErrorCodes.h>
|
#include <DB/Core/ErrorCodes.h>
|
||||||
|
|
||||||
/** При использовании HashTableAllocatorWithStackMemory, размещённом на стеке,
|
/** При использовании HashTableAllocatorWithStackMemory, размещённом на стеке,
|
||||||
|
109
dbms/include/DB/Common/HyperLogLogBiasEstimator.h
Normal file
109
dbms/include/DB/Common/HyperLogLogBiasEstimator.h
Normal file
@ -0,0 +1,109 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <DB/Common/Exception.h>
|
||||||
|
#include <DB/Core/ErrorCodes.h>
|
||||||
|
|
||||||
|
#include <algorithm>
|
||||||
|
#include <limits>
|
||||||
|
#include <tuple>
|
||||||
|
#include <type_traits>
|
||||||
|
|
||||||
|
/** Этот класс предоставляет способ, чтобы оценить погрешность результата применения алгоритма HyperLogLog.
|
||||||
|
* Эмирические наблюдения показывают, что большие погрешности возникают при E < 5 * 2^precision, где
|
||||||
|
* E - возвращаемое значение алгоритмом HyperLogLog, и precision - параметр точности HyperLogLog.
|
||||||
|
* См. "HyperLogLog in Practice: Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm".
|
||||||
|
* (S. Heule et al., Proceedings of the EDBT 2013 Conference).
|
||||||
|
*/
|
||||||
|
template <typename BiasData>
|
||||||
|
class HyperLogLogBiasEstimator
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
static constexpr bool isTrivial()
|
||||||
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Предельное количество уникальных значений до которого должна примениться поправка
|
||||||
|
/// из алгоритма LinearCounting.
|
||||||
|
static double getThreshold()
|
||||||
|
{
|
||||||
|
return BiasData::getThreshold();
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Вернуть оценку погрешности.
|
||||||
|
static double getBias(double raw_estimate)
|
||||||
|
{
|
||||||
|
const auto & estimates = BiasData::getRawEstimates();
|
||||||
|
const auto & biases = BiasData::getBiases();
|
||||||
|
|
||||||
|
auto it = std::lower_bound(estimates.begin(), estimates.end(), raw_estimate);
|
||||||
|
|
||||||
|
if (it == estimates.end())
|
||||||
|
{
|
||||||
|
return biases[estimates.size() - 1];
|
||||||
|
}
|
||||||
|
else if (*it == raw_estimate)
|
||||||
|
{
|
||||||
|
size_t index = std::distance(estimates.begin(), it);
|
||||||
|
return biases[index];
|
||||||
|
}
|
||||||
|
else if (it == estimates.begin())
|
||||||
|
{
|
||||||
|
return biases[0];
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
/// Получаем оценку погрешности путём линейной интерполяции.
|
||||||
|
size_t index = std::distance(estimates.begin(), it);
|
||||||
|
|
||||||
|
double estimate1 = estimates[index - 1];
|
||||||
|
double estimate2 = estimates[index];
|
||||||
|
|
||||||
|
double bias1 = biases[index - 1];
|
||||||
|
double bias2 = biases[index];
|
||||||
|
/// Предполагается, что условие estimate1 < estimate2 всегда выполнено.
|
||||||
|
double slope = (bias2 - bias1) / (estimate2 - estimate1);
|
||||||
|
|
||||||
|
return bias1 + slope * (raw_estimate - estimate1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
/// Статические проверки.
|
||||||
|
using TRawEstimatesRef = decltype(BiasData::getRawEstimates());
|
||||||
|
using TRawEstimates = typename std::remove_reference<TRawEstimatesRef>::type;
|
||||||
|
|
||||||
|
using TBiasDataRef = decltype(BiasData::getBiases());
|
||||||
|
using TBiasData = typename std::remove_reference<TBiasDataRef>::type;
|
||||||
|
|
||||||
|
static_assert(std::is_same<TRawEstimates, TBiasData>::value, "Bias estimator data have inconsistent types");
|
||||||
|
static_assert(std::tuple_size<TRawEstimates>::value > 0, "Bias estimator has no raw estimate data");
|
||||||
|
static_assert(std::tuple_size<TBiasData>::value > 0, "Bias estimator has no bias data");
|
||||||
|
static_assert(std::tuple_size<TRawEstimates>::value == std::tuple_size<TBiasData>::value,
|
||||||
|
"Bias estimator has inconsistent data");
|
||||||
|
};
|
||||||
|
|
||||||
|
/** Тривиальный случай HyperLogLogBiasEstimator: употребляется, если не хотим исправить
|
||||||
|
* погрешность. Это имеет смысль при маленьких значениях параметра точности, например 5 или 12.
|
||||||
|
* Тогда применяются поправки из оригинальной версии алгоритма HyperLogLog.
|
||||||
|
* См. "HyperLogLog: The analysis of a near-optimal cardinality estimation algorithm"
|
||||||
|
* (P. Flajolet et al., AOFA '07: Proceedings of the 2007 International Conference on Analysis
|
||||||
|
* of Algorithms)
|
||||||
|
*/
|
||||||
|
struct TrivialBiasEstimator
|
||||||
|
{
|
||||||
|
static constexpr bool isTrivial()
|
||||||
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
static double getThreshold()
|
||||||
|
{
|
||||||
|
return 0.0;
|
||||||
|
}
|
||||||
|
|
||||||
|
static double getBias(double raw_estimate)
|
||||||
|
{
|
||||||
|
return 0.0;
|
||||||
|
}
|
||||||
|
};
|
727
dbms/include/DB/Common/HyperLogLogCounter.h
Normal file
727
dbms/include/DB/Common/HyperLogLogCounter.h
Normal file
@ -0,0 +1,727 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <common/Common.h>
|
||||||
|
#include <stats/IntHash.h>
|
||||||
|
#include <DB/Common/HyperLogLogBiasEstimator.h>
|
||||||
|
|
||||||
|
#include <DB/IO/ReadBuffer.h>
|
||||||
|
#include <DB/IO/WriteBuffer.h>
|
||||||
|
#include <DB/IO/ReadHelpers.h>
|
||||||
|
#include <DB/IO/WriteHelpers.h>
|
||||||
|
#include <DB/Core/Defines.h>
|
||||||
|
|
||||||
|
#include <cmath>
|
||||||
|
#include <cstring>
|
||||||
|
|
||||||
|
namespace details
|
||||||
|
{
|
||||||
|
|
||||||
|
/// Look-up table логарифмов от целых чисел для использования в HyperLogLogCounter.
|
||||||
|
template<UInt8 K>
|
||||||
|
struct LogLUT
|
||||||
|
{
|
||||||
|
LogLUT()
|
||||||
|
{
|
||||||
|
log_table[0] = 0.0;
|
||||||
|
for (size_t i = 1; i <= M; ++i)
|
||||||
|
log_table[i] = log(static_cast<double>(i));
|
||||||
|
}
|
||||||
|
|
||||||
|
double getLog(size_t x) const
|
||||||
|
{
|
||||||
|
if (x <= M)
|
||||||
|
return log_table[x];
|
||||||
|
else
|
||||||
|
return log(static_cast<double>(x));
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
static constexpr size_t M = 1 << ((static_cast<unsigned int>(K) <= 12) ? K : 12);
|
||||||
|
|
||||||
|
double log_table[M + 1];
|
||||||
|
};
|
||||||
|
|
||||||
|
template<UInt8 K> struct MinCounterTypeHelper;
|
||||||
|
template<> struct MinCounterTypeHelper<0> { typedef UInt8 Type; };
|
||||||
|
template<> struct MinCounterTypeHelper<1> { typedef UInt16 Type; };
|
||||||
|
template<> struct MinCounterTypeHelper<2> { typedef UInt32 Type; };
|
||||||
|
template<> struct MinCounterTypeHelper<3> { typedef UInt64 Type; };
|
||||||
|
|
||||||
|
/// Вспомогательная структура для автоматического определения
|
||||||
|
/// минимального размера типа счетчика в зависимости от максимального значения.
|
||||||
|
/// Используется там, где нужна максимальная экономия памяти,
|
||||||
|
/// например, в HyperLogLogCounter
|
||||||
|
template<UInt64 MaxValue> struct MinCounterType
|
||||||
|
{
|
||||||
|
typedef typename MinCounterTypeHelper<
|
||||||
|
(MaxValue >= 1 << 8) +
|
||||||
|
(MaxValue >= 1 << 16) +
|
||||||
|
(MaxValue >= 1ULL << 32)
|
||||||
|
>::Type Type;
|
||||||
|
};
|
||||||
|
|
||||||
|
/** Компактный массив для хранения данных, размер L, в битах, которых составляет меньше одного байта.
|
||||||
|
* Вместо того, чтобы хранить каждое значение в 8-битную ячейку памяти, что приводит к растрате
|
||||||
|
* 37.5% пространства для L=5, CompactArray хранит смежные L-битные значения, именно компактные
|
||||||
|
* ячейки в массиве байтов, т.е. фактически CompactArray симулирует массив L-битных значений.
|
||||||
|
*/
|
||||||
|
template<typename BucketIndex, UInt8 content_width, size_t bucket_count>
|
||||||
|
class __attribute__ ((packed)) CompactArray final
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
class Locus;
|
||||||
|
|
||||||
|
public:
|
||||||
|
CompactArray() = default;
|
||||||
|
|
||||||
|
UInt8 ALWAYS_INLINE operator[](BucketIndex bucket_index) const
|
||||||
|
{
|
||||||
|
Locus locus(bucket_index);
|
||||||
|
|
||||||
|
if (locus.index_l == locus.index_r)
|
||||||
|
return locus.read(bitset[locus.index_l]);
|
||||||
|
else
|
||||||
|
return locus.read(bitset[locus.index_l], bitset[locus.index_r]);
|
||||||
|
}
|
||||||
|
|
||||||
|
Locus ALWAYS_INLINE operator[](BucketIndex bucket_index)
|
||||||
|
{
|
||||||
|
Locus locus(bucket_index);
|
||||||
|
|
||||||
|
locus.content_l = &bitset[locus.index_l];
|
||||||
|
|
||||||
|
if (locus.index_l == locus.index_r)
|
||||||
|
locus.content_r = locus.content_l;
|
||||||
|
else
|
||||||
|
locus.content_r = &bitset[locus.index_r];
|
||||||
|
|
||||||
|
return locus;
|
||||||
|
}
|
||||||
|
|
||||||
|
void readText(DB::ReadBuffer & in)
|
||||||
|
{
|
||||||
|
for (size_t i = 0; i < BITSET_SIZE; ++i)
|
||||||
|
{
|
||||||
|
if (i != 0)
|
||||||
|
DB::assertString(",", in);
|
||||||
|
DB::readIntText(bitset[i], in);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void writeText(DB::WriteBuffer & out) const
|
||||||
|
{
|
||||||
|
for (size_t i = 0; i < BITSET_SIZE; ++i)
|
||||||
|
{
|
||||||
|
if (i != 0)
|
||||||
|
writeCString(",", out);
|
||||||
|
DB::writeIntText(bitset[i], out);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
/// число байт в битсете
|
||||||
|
static constexpr size_t BITSET_SIZE = (static_cast<size_t>(bucket_count) * content_width + 7) / 8;
|
||||||
|
UInt8 bitset[BITSET_SIZE] = { 0 };
|
||||||
|
};
|
||||||
|
|
||||||
|
/** Структура Locus содержит необходимую информацию, чтобы найти для каждой компактной ячейки
|
||||||
|
* соответствующие физическую ячейку и смещение, в битах, от начала ячейки. Поскольку в общем
|
||||||
|
* случае размер одной физической ячейки не делится на размер одной компактной ячейки, возможны
|
||||||
|
* случаи, когда одна компактная ячейка перекрывает две физические ячейки. Поэтому структура
|
||||||
|
* Locus содержит две пары (индекс, смещение).
|
||||||
|
*/
|
||||||
|
template<typename BucketIndex, UInt8 content_width, size_t bucket_count>
|
||||||
|
class CompactArray<BucketIndex, content_width, bucket_count>::Locus final
|
||||||
|
{
|
||||||
|
friend class CompactArray;
|
||||||
|
|
||||||
|
public:
|
||||||
|
ALWAYS_INLINE operator UInt8() const
|
||||||
|
{
|
||||||
|
if (content_l == content_r)
|
||||||
|
return read(*content_l);
|
||||||
|
else
|
||||||
|
return read(*content_l, *content_r);
|
||||||
|
}
|
||||||
|
|
||||||
|
Locus ALWAYS_INLINE & operator=(UInt8 content)
|
||||||
|
{
|
||||||
|
if ((index_l == index_r) || (index_l == (BITSET_SIZE - 1)))
|
||||||
|
{
|
||||||
|
/// Компактная ячейка полностью влезает в одну физическую ячейку.
|
||||||
|
*content_l &= ~(((1 << content_width) - 1) << offset_l);
|
||||||
|
*content_l |= content << offset_l;
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
/// Компактная ячейка перекрывает две физические ячейки.
|
||||||
|
size_t left = 8 - offset_l;
|
||||||
|
|
||||||
|
*content_l &= ~(((1 << left) - 1) << offset_l);
|
||||||
|
*content_l |= (content & ((1 << left) - 1)) << offset_l;
|
||||||
|
|
||||||
|
*content_r &= ~((1 << offset_r) - 1);
|
||||||
|
*content_r |= content >> left;
|
||||||
|
}
|
||||||
|
|
||||||
|
return *this;
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
Locus() = default;
|
||||||
|
|
||||||
|
Locus(BucketIndex bucket_index)
|
||||||
|
{
|
||||||
|
size_t l = static_cast<size_t>(bucket_index) * content_width;
|
||||||
|
index_l = l >> 3;
|
||||||
|
offset_l = l & 7;
|
||||||
|
|
||||||
|
size_t r = static_cast<size_t>(bucket_index + 1) * content_width;
|
||||||
|
index_r = r >> 3;
|
||||||
|
offset_r = r & 7;
|
||||||
|
}
|
||||||
|
|
||||||
|
UInt8 ALWAYS_INLINE read(UInt8 value_l) const
|
||||||
|
{
|
||||||
|
/// Компактная ячейка полностью влезает в одну физическую ячейку.
|
||||||
|
return (value_l >> offset_l) & ((1 << content_width) - 1);
|
||||||
|
}
|
||||||
|
|
||||||
|
UInt8 ALWAYS_INLINE read(UInt8 value_l, UInt8 value_r) const
|
||||||
|
{
|
||||||
|
/// Компактная ячейка перекрывает две физические ячейки.
|
||||||
|
return ((value_l >> offset_l) & ((1 << (8 - offset_l)) - 1))
|
||||||
|
| ((value_r & ((1 << offset_r) - 1)) << (8 - offset_l));
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
size_t index_l;
|
||||||
|
size_t offset_l;
|
||||||
|
size_t index_r;
|
||||||
|
size_t offset_r;
|
||||||
|
|
||||||
|
UInt8 * content_l;
|
||||||
|
UInt8 * content_r;
|
||||||
|
|
||||||
|
/// Проверки
|
||||||
|
static_assert((content_width > 0) && (content_width < 8), "Invalid parameter value");
|
||||||
|
static_assert(bucket_count <= (std::numeric_limits<size_t>::max() / content_width), "Invalid parameter value");
|
||||||
|
};
|
||||||
|
|
||||||
|
/** Знаменатель формулы алгоритма HyperLogLog
|
||||||
|
*/
|
||||||
|
template<UInt8 precision, int max_rank, typename HashValueType, typename DenominatorType,
|
||||||
|
bool stable_denominator_if_big, typename Enable = void>
|
||||||
|
class __attribute__ ((packed)) Denominator;
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
|
||||||
|
/// Возвращает true, если хранилище для рангов большое.
|
||||||
|
constexpr bool isBigRankStore(UInt8 precision)
|
||||||
|
{
|
||||||
|
return precision >= 12;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Тип употребляемый для вычисления знаменателя.
|
||||||
|
*/
|
||||||
|
template <typename HashValueType>
|
||||||
|
struct IntermediateDenominator;
|
||||||
|
|
||||||
|
template <>
|
||||||
|
struct IntermediateDenominator<UInt32>
|
||||||
|
{
|
||||||
|
using Type = double;
|
||||||
|
};
|
||||||
|
|
||||||
|
template <>
|
||||||
|
struct IntermediateDenominator<UInt64>
|
||||||
|
{
|
||||||
|
using Type = long double;
|
||||||
|
};
|
||||||
|
|
||||||
|
/** "Лёгкая" реализация знаменателя формулы HyperLogLog.
|
||||||
|
* Занимает минимальный объём памяти, зато вычисления могут быть неустойчивы.
|
||||||
|
* Подходит, когда хранилище для рангов небольшое.
|
||||||
|
*/
|
||||||
|
template<UInt8 precision, int max_rank, typename HashValueType, typename DenominatorType,
|
||||||
|
bool stable_denominator_if_big>
|
||||||
|
class __attribute__ ((packed)) Denominator<precision, max_rank, HashValueType, DenominatorType,
|
||||||
|
stable_denominator_if_big,
|
||||||
|
typename std::enable_if<!details::isBigRankStore(precision) || !stable_denominator_if_big>::type>
|
||||||
|
{
|
||||||
|
private:
|
||||||
|
using T = typename IntermediateDenominator<HashValueType>::Type;
|
||||||
|
|
||||||
|
public:
|
||||||
|
Denominator(DenominatorType initial_value)
|
||||||
|
: denominator(initial_value)
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
public:
|
||||||
|
inline void update(UInt8 cur_rank, UInt8 new_rank)
|
||||||
|
{
|
||||||
|
denominator -= static_cast<T>(1.0) / (1ULL << cur_rank);
|
||||||
|
denominator += static_cast<T>(1.0) / (1ULL << new_rank);
|
||||||
|
}
|
||||||
|
|
||||||
|
inline void update(UInt8 rank)
|
||||||
|
{
|
||||||
|
denominator += static_cast<T>(1.0) / (1ULL << rank);
|
||||||
|
}
|
||||||
|
|
||||||
|
void clear()
|
||||||
|
{
|
||||||
|
denominator = 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
DenominatorType get() const
|
||||||
|
{
|
||||||
|
return denominator;
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
T denominator;
|
||||||
|
};
|
||||||
|
|
||||||
|
/** "Тяжёлая" версия знаменателя формулы HyperLogLog.
|
||||||
|
* Занимает больший объём памяти, чем лёгкая версия, зато вычисления всегда устойчивы.
|
||||||
|
* Подходит, когда хранилище для рангов довольно большое.
|
||||||
|
*/
|
||||||
|
template<UInt8 precision, int max_rank, typename HashValueType, typename DenominatorType,
|
||||||
|
bool stable_denominator_if_big>
|
||||||
|
class __attribute__ ((packed)) Denominator<precision, max_rank, HashValueType, DenominatorType,
|
||||||
|
stable_denominator_if_big,
|
||||||
|
typename std::enable_if<details::isBigRankStore(precision) && stable_denominator_if_big>::type>
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
Denominator(DenominatorType initial_value)
|
||||||
|
{
|
||||||
|
rank_count[0] = initial_value;
|
||||||
|
}
|
||||||
|
|
||||||
|
inline void update(UInt8 cur_rank, UInt8 new_rank)
|
||||||
|
{
|
||||||
|
--rank_count[cur_rank];
|
||||||
|
++rank_count[new_rank];
|
||||||
|
}
|
||||||
|
|
||||||
|
inline void update(UInt8 rank)
|
||||||
|
{
|
||||||
|
++rank_count[rank];
|
||||||
|
}
|
||||||
|
|
||||||
|
void clear()
|
||||||
|
{
|
||||||
|
memset(rank_count, 0, size * sizeof(UInt32));
|
||||||
|
}
|
||||||
|
|
||||||
|
DenominatorType get() const
|
||||||
|
{
|
||||||
|
long double val = rank_count[size - 1];
|
||||||
|
for (int i = size - 2; i >= 0; --i)
|
||||||
|
{
|
||||||
|
val /= 2.0;
|
||||||
|
val += rank_count[i];
|
||||||
|
}
|
||||||
|
return val;
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
static constexpr size_t size = max_rank + 1;
|
||||||
|
UInt32 rank_count[size] = { 0 };
|
||||||
|
};
|
||||||
|
|
||||||
|
/** Число хвостовых (младших) нулей.
|
||||||
|
*/
|
||||||
|
template <typename T>
|
||||||
|
struct TrailingZerosCounter;
|
||||||
|
|
||||||
|
template <>
|
||||||
|
struct TrailingZerosCounter<UInt32>
|
||||||
|
{
|
||||||
|
static int apply(UInt32 val)
|
||||||
|
{
|
||||||
|
return __builtin_ctz(val);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
template <>
|
||||||
|
struct TrailingZerosCounter<UInt64>
|
||||||
|
{
|
||||||
|
static int apply(UInt64 val)
|
||||||
|
{
|
||||||
|
return __builtin_ctzll(val);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
/** Размер счётчика ранга в битах.
|
||||||
|
*/
|
||||||
|
template <typename T>
|
||||||
|
struct RankWidth;
|
||||||
|
|
||||||
|
template <>
|
||||||
|
struct RankWidth<UInt32>
|
||||||
|
{
|
||||||
|
static constexpr UInt8 get()
|
||||||
|
{
|
||||||
|
return 5;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
template <>
|
||||||
|
struct RankWidth<UInt64>
|
||||||
|
{
|
||||||
|
static constexpr UInt8 get()
|
||||||
|
{
|
||||||
|
return 6;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Поведение класса HyperLogLogCounter.
|
||||||
|
*/
|
||||||
|
enum class HyperLogLogMode
|
||||||
|
{
|
||||||
|
Raw, /// Применить алгоритм HyperLogLog без исправления погрешности
|
||||||
|
LinearCounting, /// Исправить погрешность по алгоритму LinearCounting
|
||||||
|
BiasCorrected, /// Исправить погрешность по алгоритму HyperLogLog++
|
||||||
|
FullFeatured /// Исправить погрешность по алгоритму LinearCounting или HyperLogLog++
|
||||||
|
};
|
||||||
|
|
||||||
|
/** Подсчёт уникальных значений алгоритмом HyperLogLog.
|
||||||
|
*
|
||||||
|
* Теоретическая относительная погрешность ~1.04 / sqrt(2^precision)
|
||||||
|
* precision - длина префикса хэш-функции для индекса (число ячеек M = 2^precision)
|
||||||
|
* Рекомендуемые значения precision: 3..20
|
||||||
|
*
|
||||||
|
* Источник: "HyperLogLog: The analysis of a near-optimal cardinality estimation algorithm"
|
||||||
|
* (P. Flajolet et al., AOFA '07: Proceedings of the 2007 International Conference on Analysis
|
||||||
|
* of Algorithms)
|
||||||
|
*/
|
||||||
|
template <
|
||||||
|
UInt8 precision,
|
||||||
|
typename Hash = IntHash32<UInt64>,
|
||||||
|
typename HashValueType = UInt32,
|
||||||
|
typename DenominatorType = double,
|
||||||
|
typename BiasEstimator = TrivialBiasEstimator,
|
||||||
|
HyperLogLogMode mode = HyperLogLogMode::FullFeatured,
|
||||||
|
bool stable_denominator_if_big = true>
|
||||||
|
class __attribute__ ((packed)) HyperLogLogCounter : private Hash
|
||||||
|
{
|
||||||
|
private:
|
||||||
|
/// Число ячеек.
|
||||||
|
static constexpr size_t bucket_count = 1ULL << precision;
|
||||||
|
/// Размер счётчика ранга в битах.
|
||||||
|
static constexpr UInt8 rank_width = details::RankWidth<HashValueType>::get();
|
||||||
|
|
||||||
|
private:
|
||||||
|
using Value_t = UInt64;
|
||||||
|
using RankStore = details::CompactArray<HashValueType, rank_width, bucket_count>;
|
||||||
|
|
||||||
|
public:
|
||||||
|
void insert(Value_t value)
|
||||||
|
{
|
||||||
|
HashValueType hash = getHash(value);
|
||||||
|
|
||||||
|
/// Разбиваем хэш-значение на два подзначения. Первое из них является номером ячейки
|
||||||
|
/// в хранилище для рангов (rank_storage), а со второго вычисляем ранг.
|
||||||
|
HashValueType bucket = extractBitSequence(hash, 0, precision);
|
||||||
|
HashValueType tail = extractBitSequence(hash, precision, sizeof(HashValueType) * 8);
|
||||||
|
UInt8 rank = calculateRank(tail);
|
||||||
|
|
||||||
|
/// Обновляем максимальный ранг для текущей ячейки.
|
||||||
|
update(bucket, rank);
|
||||||
|
}
|
||||||
|
|
||||||
|
UInt32 size() const
|
||||||
|
{
|
||||||
|
/// Нормализующий коэффициент, входящий в среднее гармоническое.
|
||||||
|
static constexpr double alpha_m =
|
||||||
|
bucket_count == 2 ? 0.351 :
|
||||||
|
bucket_count == 4 ? 0.532 :
|
||||||
|
bucket_count == 8 ? 0.626 :
|
||||||
|
bucket_count == 16 ? 0.673 :
|
||||||
|
bucket_count == 32 ? 0.697 :
|
||||||
|
bucket_count == 64 ? 0.709 : 0.7213 / (1 + 1.079 / bucket_count);
|
||||||
|
|
||||||
|
/** Среднее гармоническое по всем корзинам из величин 2^rank равно:
|
||||||
|
* bucket_count / ∑ 2^-rank_i.
|
||||||
|
* Величина ∑ 2^-rank_i - это denominator.
|
||||||
|
*/
|
||||||
|
|
||||||
|
double raw_estimate = alpha_m * bucket_count * bucket_count / denominator.get();
|
||||||
|
|
||||||
|
double final_estimate = fixRawEstimate(raw_estimate);
|
||||||
|
|
||||||
|
return static_cast<UInt32>(final_estimate + 0.5);
|
||||||
|
}
|
||||||
|
|
||||||
|
void merge(const HyperLogLogCounter & rhs)
|
||||||
|
{
|
||||||
|
const auto & rhs_rank_store = rhs.rank_store;
|
||||||
|
for (HashValueType bucket = 0; bucket < bucket_count; ++bucket)
|
||||||
|
update(bucket, rhs_rank_store[bucket]);
|
||||||
|
}
|
||||||
|
|
||||||
|
void read(DB::ReadBuffer & in)
|
||||||
|
{
|
||||||
|
in.readStrict(reinterpret_cast<char *>(this), sizeof(*this));
|
||||||
|
}
|
||||||
|
|
||||||
|
void readAndMerge(DB::ReadBuffer & in)
|
||||||
|
{
|
||||||
|
RankStore other;
|
||||||
|
in.readStrict(reinterpret_cast<char *>(&other), sizeof(RankStore));
|
||||||
|
for (HashValueType bucket = 0; bucket < bucket_count; ++bucket)
|
||||||
|
{
|
||||||
|
UInt8 rank = other[bucket];
|
||||||
|
update(bucket, rank);
|
||||||
|
}
|
||||||
|
|
||||||
|
in.ignore(sizeof(DenominatorCalculatorType) + sizeof(ZerosCounterType));
|
||||||
|
}
|
||||||
|
|
||||||
|
static void skip(DB::ReadBuffer & in)
|
||||||
|
{
|
||||||
|
in.ignore(sizeof(RankStore) + sizeof(DenominatorCalculatorType) + sizeof(ZerosCounterType));
|
||||||
|
}
|
||||||
|
|
||||||
|
void write(DB::WriteBuffer & out) const
|
||||||
|
{
|
||||||
|
out.write(reinterpret_cast<const char *>(this), sizeof(*this));
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Запись и чтение в текстовом виде неэффективно (зато совместимо с OLAPServer-ом и Metrage).
|
||||||
|
void readText(DB::ReadBuffer & in)
|
||||||
|
{
|
||||||
|
rank_store.readText(in);
|
||||||
|
|
||||||
|
zeros = 0;
|
||||||
|
denominator.clear();
|
||||||
|
for (HashValueType bucket = 0; bucket < bucket_count; ++bucket)
|
||||||
|
{
|
||||||
|
UInt8 rank = rank_store[bucket];
|
||||||
|
if (rank == 0)
|
||||||
|
++zeros;
|
||||||
|
denominator.update(rank);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static void skipText(DB::ReadBuffer & in)
|
||||||
|
{
|
||||||
|
UInt8 dummy;
|
||||||
|
for (size_t i = 0; i < RankStore::size(); ++i)
|
||||||
|
{
|
||||||
|
if (i != 0)
|
||||||
|
DB::assertString(",", in);
|
||||||
|
DB::readIntText(dummy, in);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void writeText(DB::WriteBuffer & out) const
|
||||||
|
{
|
||||||
|
rank_store.writeText(out);
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
/// Извлечь подмножество битов [begin, end[.
|
||||||
|
inline HashValueType extractBitSequence(HashValueType val, UInt8 begin, UInt8 end) const
|
||||||
|
{
|
||||||
|
return (val >> begin) & ((1ULL << (end - begin)) - 1);
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Ранг = число хвостовых (младших) нулей + 1
|
||||||
|
inline UInt8 calculateRank(HashValueType val) const
|
||||||
|
{
|
||||||
|
if (unlikely(val == 0))
|
||||||
|
return max_rank;
|
||||||
|
|
||||||
|
auto zeros_plus_one = details::TrailingZerosCounter<HashValueType>::apply(val) + 1;
|
||||||
|
|
||||||
|
if (unlikely(zeros_plus_one) > max_rank)
|
||||||
|
return max_rank;
|
||||||
|
|
||||||
|
return zeros_plus_one;
|
||||||
|
}
|
||||||
|
|
||||||
|
inline HashValueType getHash(Value_t key) const
|
||||||
|
{
|
||||||
|
return Hash::operator()(key);
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Обновить максимальный ранг для заданной ячейки.
|
||||||
|
void update(HashValueType bucket, UInt8 rank)
|
||||||
|
{
|
||||||
|
typename RankStore::Locus content = rank_store[bucket];
|
||||||
|
UInt8 cur_rank = static_cast<UInt8>(content);
|
||||||
|
|
||||||
|
if (rank > cur_rank)
|
||||||
|
{
|
||||||
|
if (cur_rank == 0)
|
||||||
|
--zeros;
|
||||||
|
denominator.update(cur_rank, rank);
|
||||||
|
content = rank;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
template <HyperLogLogMode mode2 = mode>
|
||||||
|
inline double fixRawEstimate(double raw_estimate,
|
||||||
|
typename std::enable_if<(mode2 == HyperLogLogMode::Raw)
|
||||||
|
|| ((mode2 == HyperLogLogMode::BiasCorrected)
|
||||||
|
&& BiasEstimator::isTrivial())>::type * = nullptr) const
|
||||||
|
{
|
||||||
|
return raw_estimate;
|
||||||
|
}
|
||||||
|
|
||||||
|
template <HyperLogLogMode mode2 = mode>
|
||||||
|
inline double fixRawEstimate(double raw_estimate,
|
||||||
|
typename std::enable_if<(mode2 == HyperLogLogMode::LinearCounting)>::type * = nullptr) const
|
||||||
|
{
|
||||||
|
return applyLinearCorrection(raw_estimate);
|
||||||
|
}
|
||||||
|
|
||||||
|
template <HyperLogLogMode mode2 = mode>
|
||||||
|
inline double fixRawEstimate(double raw_estimate,
|
||||||
|
typename std::enable_if<(mode2 == HyperLogLogMode::BiasCorrected)
|
||||||
|
&& !BiasEstimator::isTrivial()>::type * = nullptr) const
|
||||||
|
{
|
||||||
|
return applyBiasCorrection(raw_estimate);
|
||||||
|
}
|
||||||
|
|
||||||
|
template <HyperLogLogMode mode2 = mode>
|
||||||
|
double fixRawEstimate(double raw_estimate,
|
||||||
|
typename std::enable_if<(mode2 == HyperLogLogMode::FullFeatured)>::type * = nullptr) const
|
||||||
|
{
|
||||||
|
static constexpr bool fix_big_cardinalities = std::is_same<HashValueType, UInt32>::value;
|
||||||
|
static constexpr double pow2_32 = 4294967296.0;
|
||||||
|
|
||||||
|
double fixed_estimate;
|
||||||
|
|
||||||
|
if (fix_big_cardinalities && (raw_estimate > (pow2_32 / 30.0)))
|
||||||
|
fixed_estimate = -pow2_32 * log(1.0 - raw_estimate / pow2_32);
|
||||||
|
else
|
||||||
|
fixed_estimate = applyCorrection(raw_estimate);
|
||||||
|
|
||||||
|
return fixed_estimate;
|
||||||
|
}
|
||||||
|
|
||||||
|
template <bool is_trivial = BiasEstimator::isTrivial()>
|
||||||
|
inline double applyCorrection(double raw_estimate, typename std::enable_if<is_trivial>::type * = nullptr) const
|
||||||
|
{
|
||||||
|
double fixed_estimate;
|
||||||
|
|
||||||
|
if (raw_estimate <= (2.5 * bucket_count))
|
||||||
|
{
|
||||||
|
/// Поправка в случае маленкой оценки.
|
||||||
|
fixed_estimate = applyLinearCorrection(raw_estimate);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
fixed_estimate = raw_estimate;
|
||||||
|
|
||||||
|
return fixed_estimate;
|
||||||
|
}
|
||||||
|
|
||||||
|
template <bool is_trivial = BiasEstimator::isTrivial()>
|
||||||
|
inline double applyCorrection(double raw_estimate, typename std::enable_if<!is_trivial>::type * = nullptr) const
|
||||||
|
{
|
||||||
|
double fixed_estimate = applyBiasCorrection(raw_estimate);
|
||||||
|
double linear_estimate = applyLinearCorrection(fixed_estimate);
|
||||||
|
|
||||||
|
if (linear_estimate < BiasEstimator::getThreshold())
|
||||||
|
fixed_estimate = linear_estimate;
|
||||||
|
|
||||||
|
return fixed_estimate;
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Поправка из алгоритма HyperLogLog++.
|
||||||
|
/// Источник: "HyperLogLog in Practice: Algorithmic Engineering of a State of The Art
|
||||||
|
/// Cardinality Estimation Algorithm".
|
||||||
|
/// (S. Heule et al., Proceedings of the EDBT 2013 Conference).
|
||||||
|
inline double applyBiasCorrection(double raw_estimate) const
|
||||||
|
{
|
||||||
|
double fixed_estimate;
|
||||||
|
|
||||||
|
if (raw_estimate <= (5 * bucket_count))
|
||||||
|
fixed_estimate = raw_estimate - BiasEstimator::getBias(raw_estimate);
|
||||||
|
else
|
||||||
|
fixed_estimate = raw_estimate;
|
||||||
|
|
||||||
|
return fixed_estimate;
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Подсчет уникальных значений по алгоритму LinearCounting.
|
||||||
|
/// Источник: "A Linear-time Probabilistic Counting Algorithm for Database Applications"
|
||||||
|
/// (Whang et al., ACM Trans. Database Syst., pp. 208-229, 1990)
|
||||||
|
inline double applyLinearCorrection(double raw_estimate) const
|
||||||
|
{
|
||||||
|
double fixed_estimate;
|
||||||
|
|
||||||
|
if (zeros != 0)
|
||||||
|
fixed_estimate = bucket_count * (log_lut.getLog(bucket_count) - log_lut.getLog(zeros));
|
||||||
|
else
|
||||||
|
fixed_estimate = raw_estimate;
|
||||||
|
|
||||||
|
return fixed_estimate;
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
/// Максимальный ранг.
|
||||||
|
static constexpr int max_rank = sizeof(HashValueType) * 8 - precision + 1;
|
||||||
|
|
||||||
|
/// Хранилище для рангов.
|
||||||
|
RankStore rank_store;
|
||||||
|
|
||||||
|
/// Знаменатель формулы алгоритма HyperLogLog.
|
||||||
|
using DenominatorCalculatorType = details::Denominator<precision, max_rank, HashValueType, DenominatorType, stable_denominator_if_big>;
|
||||||
|
DenominatorCalculatorType denominator{bucket_count};
|
||||||
|
|
||||||
|
/// Число нулей в хранилище для рангов.
|
||||||
|
using ZerosCounterType = typename details::MinCounterType<bucket_count>::Type;
|
||||||
|
ZerosCounterType zeros = bucket_count;
|
||||||
|
|
||||||
|
static details::LogLUT<precision> log_lut;
|
||||||
|
|
||||||
|
/// Проверки.
|
||||||
|
static_assert(precision < (sizeof(HashValueType) * 8), "Invalid parameter value");
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
/// Определения статических переменных, нужные во время линковки.
|
||||||
|
template
|
||||||
|
<
|
||||||
|
UInt8 precision,
|
||||||
|
typename Hash,
|
||||||
|
typename HashValueType,
|
||||||
|
typename DenominatorType,
|
||||||
|
typename BiasEstimator,
|
||||||
|
HyperLogLogMode mode,
|
||||||
|
bool stable_denominator_if_big
|
||||||
|
>
|
||||||
|
details::LogLUT<precision> HyperLogLogCounter
|
||||||
|
<
|
||||||
|
precision,
|
||||||
|
Hash,
|
||||||
|
HashValueType,
|
||||||
|
DenominatorType,
|
||||||
|
BiasEstimator,
|
||||||
|
mode,
|
||||||
|
stable_denominator_if_big
|
||||||
|
>::log_lut;
|
||||||
|
|
||||||
|
|
||||||
|
/// Для Metrage, используется лёгкая реализация знаменателя формулы HyperLogLog,
|
||||||
|
/// чтобы формат сериализации не изменился.
|
||||||
|
typedef HyperLogLogCounter<
|
||||||
|
12,
|
||||||
|
IntHash32<UInt64>,
|
||||||
|
UInt32,
|
||||||
|
double,
|
||||||
|
TrivialBiasEstimator,
|
||||||
|
HyperLogLogMode::FullFeatured,
|
||||||
|
false
|
||||||
|
> HLL12;
|
@ -1,6 +1,6 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <statdaemons/HyperLogLogCounter.h>
|
#include <DB/Common/HyperLogLogCounter.h>
|
||||||
#include <DB/Common/HashTable/SmallTable.h>
|
#include <DB/Common/HashTable/SmallTable.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
87
dbms/include/DB/Common/Increment.h
Normal file
87
dbms/include/DB/Common/Increment.h
Normal file
@ -0,0 +1,87 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <DB/Common/CounterInFile.h>
|
||||||
|
|
||||||
|
|
||||||
|
/** Позволяет получать авто-инкрементное число, храня его в файле.
|
||||||
|
* Предназначен для редких вызовов (не рассчитан на производительность).
|
||||||
|
*/
|
||||||
|
class Increment
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
/// path - имя файла, включая путь
|
||||||
|
Increment(const std::string & path_) : counter(path_) {}
|
||||||
|
|
||||||
|
/** Получить следующее число.
|
||||||
|
* Если параметр create_if_need не установлен в true, то
|
||||||
|
* в файле уже должно быть записано какое-нибудь число (если нет - создайте файл вручную с нулём).
|
||||||
|
*
|
||||||
|
* Для защиты от race condition-ов между разными процессами, используются файловые блокировки.
|
||||||
|
* (Но при первом создании файла race condition возможен, так что лучше создать файл заранее.)
|
||||||
|
*
|
||||||
|
* locked_callback вызывается при заблокированном файле со счетчиком. В него передается новое значение.
|
||||||
|
* locked_callback можно использовать, чтобы делать что-нибудь атомарно с увеличением счетчика (например, переименовывать файлы).
|
||||||
|
*/
|
||||||
|
template <typename Callback>
|
||||||
|
UInt64 get(Callback && locked_callback, bool create_if_need = false)
|
||||||
|
{
|
||||||
|
return static_cast<UInt64>(counter.add(1, std::forward<Callback>(locked_callback), create_if_need));
|
||||||
|
}
|
||||||
|
|
||||||
|
UInt64 get(bool create_if_need = false)
|
||||||
|
{
|
||||||
|
return getBunch(1, create_if_need);
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Посмотреть следующее значение.
|
||||||
|
UInt64 peek(bool create_if_need = false)
|
||||||
|
{
|
||||||
|
return getBunch(0, create_if_need);
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Получить следующее число и увеличить счетчик на count.
|
||||||
|
* Если параметр create_if_need не установлен в true, то
|
||||||
|
* в файле уже должно быть записано какое-нибудь число (если нет - создайте файл вручную с нулём).
|
||||||
|
*
|
||||||
|
* Для защиты от race condition-ов между разными процессами, используются файловые блокировки.
|
||||||
|
* (Но при первом создании файла race condition возможен, так что лучше создать файл заранее.)
|
||||||
|
*/
|
||||||
|
UInt64 getBunch(UInt64 count, bool create_if_need = false)
|
||||||
|
{
|
||||||
|
return static_cast<UInt64>(counter.add(static_cast<Int64>(count), create_if_need) - count + 1);
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Изменить путь к файлу.
|
||||||
|
void setPath(std::string path_)
|
||||||
|
{
|
||||||
|
counter.setPath(path_);
|
||||||
|
}
|
||||||
|
|
||||||
|
void fixIfBroken(UInt64 value)
|
||||||
|
{
|
||||||
|
counter.fixIfBroken(value);
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
CounterInFile counter;
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
/** То же самое, но без хранения в файле.
|
||||||
|
*/
|
||||||
|
struct SimpleIncrement : private boost::noncopyable
|
||||||
|
{
|
||||||
|
UInt64 value;
|
||||||
|
|
||||||
|
SimpleIncrement(UInt64 start = 0) : value(start) {}
|
||||||
|
|
||||||
|
void set(UInt64 new_value)
|
||||||
|
{
|
||||||
|
value = new_value;
|
||||||
|
}
|
||||||
|
|
||||||
|
UInt64 get()
|
||||||
|
{
|
||||||
|
return __sync_add_and_fetch(&value, 1);
|
||||||
|
}
|
||||||
|
};
|
@ -7,7 +7,7 @@
|
|||||||
#include <Poco/ScopedLock.h>
|
#include <Poco/ScopedLock.h>
|
||||||
#include <Poco/Mutex.h>
|
#include <Poco/Mutex.h>
|
||||||
#include <DB/Core/ErrorCodes.h>
|
#include <DB/Core/ErrorCodes.h>
|
||||||
#include <DB/Core/Exception.h>
|
#include <DB/Common/Exception.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
@ -1,6 +1,6 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Yandex/Common.h>
|
#include <common/Common.h>
|
||||||
|
|
||||||
|
|
||||||
/** Отслеживает потребление памяти.
|
/** Отслеживает потребление памяти.
|
||||||
|
27
dbms/include/DB/Common/NetException.h
Normal file
27
dbms/include/DB/Common/NetException.h
Normal file
@ -0,0 +1,27 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <DB/Common/Exception.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
class NetException : public DB::Exception
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
explicit NetException(int code = 0) : DB::Exception(code) {}
|
||||||
|
NetException(const std::string & msg, int code = 0) : DB::Exception(msg, code) {}
|
||||||
|
NetException(const std::string & msg, const std::string & arg, int code = 0) : DB::Exception(msg, arg, code) {}
|
||||||
|
NetException(const std::string & msg, const DB::Exception & exc, int code = 0) : DB::Exception(msg, exc, code) {}
|
||||||
|
|
||||||
|
explicit NetException(const DB::Exception & exc) : DB::Exception(exc) {}
|
||||||
|
explicit NetException(const Poco::Exception & exc) : DB::Exception(exc.displayText()) {}
|
||||||
|
NetException(const DB::NetException & exc) = default;
|
||||||
|
|
||||||
|
~NetException() throw() override {}
|
||||||
|
|
||||||
|
const char * name() const throw() override { return "DB::NetException"; }
|
||||||
|
const char * className() const throw() override { return "DB::NetException"; }
|
||||||
|
DB::NetException * clone() const override { return new DB::NetException(*this); }
|
||||||
|
void rethrow() const override { throw *this; }
|
||||||
|
};
|
||||||
|
}
|
106
dbms/include/DB/Common/OptimizedRegularExpression.h
Normal file
106
dbms/include/DB/Common/OptimizedRegularExpression.h
Normal file
@ -0,0 +1,106 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <string>
|
||||||
|
#include <vector>
|
||||||
|
#include <memory>
|
||||||
|
|
||||||
|
#include <Poco/SharedPtr.h>
|
||||||
|
|
||||||
|
#include <re2/re2.h>
|
||||||
|
#include <re2_st/re2.h>
|
||||||
|
|
||||||
|
|
||||||
|
/** Использует два способа оптимизации регулярного выражения:
|
||||||
|
* 1. Если регулярное выражение является тривиальным (сводится к поиску подстроки в строке),
|
||||||
|
* то заменяет поиск на strstr или strcasestr.
|
||||||
|
* 2. Если регулярное выражение содержит безальтернативную подстроку достаточной длины,
|
||||||
|
* то перед проверкой используется strstr или strcasestr достаточной длины;
|
||||||
|
* регулярное выражение проверяется полностью только если подстрока найдена.
|
||||||
|
* 3. В остальных случаях, используется движок re2.
|
||||||
|
*
|
||||||
|
* Это имеет смысл, так как strstr и strcasestr в libc под Linux хорошо оптимизированы.
|
||||||
|
*
|
||||||
|
* Подходит, если одновременно выполнены следующие условия:
|
||||||
|
* - если в большинстве вызовов, регулярное выражение не матчится;
|
||||||
|
* - если регулярное выражение совместимо с движком re2;
|
||||||
|
* - можете использовать на свой риск, так как, возможно, не все случаи учтены.
|
||||||
|
*/
|
||||||
|
|
||||||
|
namespace OptimizedRegularExpressionDetails
|
||||||
|
{
|
||||||
|
struct Match
|
||||||
|
{
|
||||||
|
std::string::size_type offset;
|
||||||
|
std::string::size_type length;
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
template <bool thread_safe>
|
||||||
|
class OptimizedRegularExpressionImpl
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
enum Options
|
||||||
|
{
|
||||||
|
RE_CASELESS = 0x00000001,
|
||||||
|
RE_NO_CAPTURE = 0x00000010,
|
||||||
|
RE_DOT_NL = 0x00000100
|
||||||
|
};
|
||||||
|
|
||||||
|
using Match = OptimizedRegularExpressionDetails::Match;
|
||||||
|
typedef std::vector<Match> MatchVec;
|
||||||
|
|
||||||
|
using RegexType = typename std::conditional<thread_safe, re2::RE2, re2_st::RE2>::type;
|
||||||
|
using StringPieceType = typename std::conditional<thread_safe, re2::StringPiece, re2_st::StringPiece>::type;
|
||||||
|
|
||||||
|
OptimizedRegularExpressionImpl(const std::string & regexp_, int options = 0);
|
||||||
|
|
||||||
|
bool match(const std::string & subject) const
|
||||||
|
{
|
||||||
|
return match(subject.data(), subject.size());
|
||||||
|
}
|
||||||
|
|
||||||
|
bool match(const std::string & subject, Match & match_) const
|
||||||
|
{
|
||||||
|
return match(subject.data(), subject.size(), match_);
|
||||||
|
}
|
||||||
|
|
||||||
|
unsigned match(const std::string & subject, MatchVec & matches) const
|
||||||
|
{
|
||||||
|
return match(subject.data(), subject.size(), matches);
|
||||||
|
}
|
||||||
|
|
||||||
|
unsigned match(const char * subject, size_t subject_size, MatchVec & matches) const
|
||||||
|
{
|
||||||
|
return match(subject, subject_size, matches, number_of_subpatterns + 1);
|
||||||
|
}
|
||||||
|
|
||||||
|
bool match(const char * subject, size_t subject_size) const;
|
||||||
|
bool match(const char * subject, size_t subject_size, Match & match) const;
|
||||||
|
unsigned match(const char * subject, size_t subject_size, MatchVec & matches, unsigned limit) const;
|
||||||
|
|
||||||
|
unsigned getNumberOfSubpatterns() const { return number_of_subpatterns; }
|
||||||
|
|
||||||
|
/// Получить регексп re2 или nullptr, если шаблон тривиален (для вывода в лог).
|
||||||
|
const std::unique_ptr<RegexType>& getRE2() const { return re2; }
|
||||||
|
|
||||||
|
static void analyze(const std::string & regexp_, std::string & required_substring, bool & is_trivial, bool & required_substring_is_prefix);
|
||||||
|
|
||||||
|
void getAnalyzeResult(std::string & out_required_substring, bool & out_is_trivial, bool & out_required_substring_is_prefix) const
|
||||||
|
{
|
||||||
|
out_required_substring = required_substring;
|
||||||
|
out_is_trivial = is_trivial;
|
||||||
|
out_required_substring_is_prefix = required_substring_is_prefix;
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
bool is_trivial;
|
||||||
|
bool required_substring_is_prefix;
|
||||||
|
bool is_case_insensitive;
|
||||||
|
std::string required_substring;
|
||||||
|
std::unique_ptr<RegexType> re2;
|
||||||
|
unsigned number_of_subpatterns;
|
||||||
|
};
|
||||||
|
|
||||||
|
using OptimizedRegularExpression = OptimizedRegularExpressionImpl<true>;
|
||||||
|
|
||||||
|
#include "OptimizedRegularExpression.inl"
|
410
dbms/include/DB/Common/OptimizedRegularExpression.inl
Normal file
410
dbms/include/DB/Common/OptimizedRegularExpression.inl
Normal file
@ -0,0 +1,410 @@
|
|||||||
|
#include <iostream>
|
||||||
|
|
||||||
|
#include <Poco/Exception.h>
|
||||||
|
|
||||||
|
#include <DB/Common/OptimizedRegularExpression.h>
|
||||||
|
|
||||||
|
|
||||||
|
#define MIN_LENGTH_FOR_STRSTR 3
|
||||||
|
#define MAX_SUBPATTERNS 5
|
||||||
|
|
||||||
|
template <bool b>
|
||||||
|
void OptimizedRegularExpressionImpl<b>::analyze(const std::string & regexp, std::string & required_substring, bool & is_trivial, bool & required_substring_is_prefix)
|
||||||
|
{
|
||||||
|
/** Выражение тривиально, если в нём все метасимволы эскейплены.
|
||||||
|
* Безальтернативная строка - это
|
||||||
|
* строка вне скобок,
|
||||||
|
* в которой все метасимволы эскейплены,
|
||||||
|
* а также если вне скобок нет '|',
|
||||||
|
* а также избегаются подстроки вида http:// или www.
|
||||||
|
*/
|
||||||
|
const char * begin = regexp.data();
|
||||||
|
const char * pos = begin;
|
||||||
|
const char * end = regexp.data() + regexp.size();
|
||||||
|
int depth = 0;
|
||||||
|
is_trivial = true;
|
||||||
|
required_substring_is_prefix = false;
|
||||||
|
required_substring.clear();
|
||||||
|
bool has_alternative_on_depth_0 = false;
|
||||||
|
|
||||||
|
/// Подстрока с позицией.
|
||||||
|
typedef std::pair<std::string, size_t> Substring;
|
||||||
|
|
||||||
|
typedef std::vector<Substring> Substrings;
|
||||||
|
Substrings trivial_substrings(1);
|
||||||
|
Substring * last_substring = &trivial_substrings.back();
|
||||||
|
|
||||||
|
bool in_curly_braces = false;
|
||||||
|
bool in_square_braces = false;
|
||||||
|
|
||||||
|
while (pos != end)
|
||||||
|
{
|
||||||
|
switch (*pos)
|
||||||
|
{
|
||||||
|
case '\0':
|
||||||
|
pos = end;
|
||||||
|
break;
|
||||||
|
case '\\':
|
||||||
|
{
|
||||||
|
++pos;
|
||||||
|
if (pos == end)
|
||||||
|
break;
|
||||||
|
|
||||||
|
switch (*pos)
|
||||||
|
{
|
||||||
|
case '|': case '(': case ')': case '^': case '$': case '.': case '[': case '?': case '*': case '+': case '{':
|
||||||
|
if (depth == 0 && !in_curly_braces && !in_square_braces)
|
||||||
|
{
|
||||||
|
if (last_substring->first.empty())
|
||||||
|
last_substring->second = pos - begin;
|
||||||
|
last_substring->first.push_back(*pos);
|
||||||
|
}
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
/// все остальные escape-последовательности не поддерживаем
|
||||||
|
is_trivial = false;
|
||||||
|
if (!last_substring->first.empty())
|
||||||
|
{
|
||||||
|
trivial_substrings.resize(trivial_substrings.size() + 1);
|
||||||
|
last_substring = &trivial_substrings.back();
|
||||||
|
}
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
|
++pos;
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case '|':
|
||||||
|
if (depth == 0)
|
||||||
|
has_alternative_on_depth_0 = true;
|
||||||
|
is_trivial = false;
|
||||||
|
if (!in_square_braces && !last_substring->first.empty())
|
||||||
|
{
|
||||||
|
trivial_substrings.resize(trivial_substrings.size() + 1);
|
||||||
|
last_substring = &trivial_substrings.back();
|
||||||
|
}
|
||||||
|
++pos;
|
||||||
|
break;
|
||||||
|
case '(':
|
||||||
|
if (!in_square_braces)
|
||||||
|
{
|
||||||
|
++depth;
|
||||||
|
is_trivial = false;
|
||||||
|
if (!last_substring->first.empty())
|
||||||
|
{
|
||||||
|
trivial_substrings.resize(trivial_substrings.size() + 1);
|
||||||
|
last_substring = &trivial_substrings.back();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
++pos;
|
||||||
|
break;
|
||||||
|
case '[':
|
||||||
|
in_square_braces = true;
|
||||||
|
++depth;
|
||||||
|
is_trivial = false;
|
||||||
|
if (!last_substring->first.empty())
|
||||||
|
{
|
||||||
|
trivial_substrings.resize(trivial_substrings.size() + 1);
|
||||||
|
last_substring = &trivial_substrings.back();
|
||||||
|
}
|
||||||
|
++pos;
|
||||||
|
break;
|
||||||
|
case ']':
|
||||||
|
in_square_braces = false;
|
||||||
|
--depth;
|
||||||
|
is_trivial = false;
|
||||||
|
if (!last_substring->first.empty())
|
||||||
|
{
|
||||||
|
trivial_substrings.resize(trivial_substrings.size() + 1);
|
||||||
|
last_substring = &trivial_substrings.back();
|
||||||
|
}
|
||||||
|
++pos;
|
||||||
|
break;
|
||||||
|
case ')':
|
||||||
|
if (!in_square_braces)
|
||||||
|
{
|
||||||
|
--depth;
|
||||||
|
is_trivial = false;
|
||||||
|
if (!last_substring->first.empty())
|
||||||
|
{
|
||||||
|
trivial_substrings.resize(trivial_substrings.size() + 1);
|
||||||
|
last_substring = &trivial_substrings.back();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
++pos;
|
||||||
|
break;
|
||||||
|
case '^': case '$': case '.': case '+':
|
||||||
|
is_trivial = false;
|
||||||
|
if (!last_substring->first.empty() && !in_square_braces)
|
||||||
|
{
|
||||||
|
trivial_substrings.resize(trivial_substrings.size() + 1);
|
||||||
|
last_substring = &trivial_substrings.back();
|
||||||
|
}
|
||||||
|
++pos;
|
||||||
|
break;
|
||||||
|
/// Квантификаторы, допускающие нулевое количество.
|
||||||
|
case '{':
|
||||||
|
in_curly_braces = true;
|
||||||
|
case '?': case '*':
|
||||||
|
is_trivial = false;
|
||||||
|
if (!last_substring->first.empty() && !in_square_braces)
|
||||||
|
{
|
||||||
|
last_substring->first.resize(last_substring->first.size() - 1);
|
||||||
|
trivial_substrings.resize(trivial_substrings.size() + 1);
|
||||||
|
last_substring = &trivial_substrings.back();
|
||||||
|
}
|
||||||
|
++pos;
|
||||||
|
break;
|
||||||
|
case '}':
|
||||||
|
in_curly_braces = false;
|
||||||
|
++pos;
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
if (depth == 0 && !in_curly_braces && !in_square_braces)
|
||||||
|
{
|
||||||
|
if (last_substring->first.empty())
|
||||||
|
last_substring->second = pos - begin;
|
||||||
|
last_substring->first.push_back(*pos);
|
||||||
|
}
|
||||||
|
++pos;
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (last_substring && last_substring->first.empty())
|
||||||
|
trivial_substrings.pop_back();
|
||||||
|
|
||||||
|
if (!is_trivial)
|
||||||
|
{
|
||||||
|
if (!has_alternative_on_depth_0)
|
||||||
|
{
|
||||||
|
/** Выберем безальтернативную подстроку максимальной длины, среди префиксов,
|
||||||
|
* или безальтернативную подстроку максимальной длины.
|
||||||
|
*/
|
||||||
|
size_t max_length = 0;
|
||||||
|
Substrings::const_iterator candidate_it = trivial_substrings.begin();
|
||||||
|
for (Substrings::const_iterator it = trivial_substrings.begin(); it != trivial_substrings.end(); ++it)
|
||||||
|
{
|
||||||
|
if (((it->second == 0 && candidate_it->second != 0)
|
||||||
|
|| ((it->second == 0) == (candidate_it->second == 0) && it->first.size() > max_length))
|
||||||
|
/// Тюнинг для предметной области
|
||||||
|
&& (it->first.size() > strlen("://") || strncmp(it->first.data(), "://", strlen("://")))
|
||||||
|
&& (it->first.size() > strlen("http://") || strncmp(it->first.data(), "http", strlen("http")))
|
||||||
|
&& (it->first.size() > strlen("www.") || strncmp(it->first.data(), "www", strlen("www")))
|
||||||
|
&& (it->first.size() > strlen("Windows ") || strncmp(it->first.data(), "Windows ", strlen("Windows "))))
|
||||||
|
{
|
||||||
|
max_length = it->first.size();
|
||||||
|
candidate_it = it;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (max_length >= MIN_LENGTH_FOR_STRSTR)
|
||||||
|
{
|
||||||
|
required_substring = candidate_it->first;
|
||||||
|
required_substring_is_prefix = candidate_it->second == 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
required_substring = trivial_substrings.front().first;
|
||||||
|
required_substring_is_prefix = trivial_substrings.front().second == 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/* std::cerr
|
||||||
|
<< "regexp: " << regexp
|
||||||
|
<< ", is_trivial: " << is_trivial
|
||||||
|
<< ", required_substring: " << required_substring
|
||||||
|
<< ", required_substring_is_prefix: " << required_substring_is_prefix
|
||||||
|
<< std::endl;*/
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
template <bool b>
|
||||||
|
OptimizedRegularExpressionImpl<b>::OptimizedRegularExpressionImpl(const std::string & regexp_, int options)
|
||||||
|
{
|
||||||
|
analyze(regexp_, required_substring, is_trivial, required_substring_is_prefix);
|
||||||
|
|
||||||
|
/// Поддерживаются 3 опции
|
||||||
|
if (options & (~(RE_CASELESS | RE_NO_CAPTURE | RE_DOT_NL)))
|
||||||
|
throw Poco::Exception("OptimizedRegularExpression: Unsupported option.");
|
||||||
|
|
||||||
|
is_case_insensitive = options & RE_CASELESS;
|
||||||
|
bool is_no_capture = options & RE_NO_CAPTURE;
|
||||||
|
bool is_dot_nl = options & RE_DOT_NL;
|
||||||
|
|
||||||
|
number_of_subpatterns = 0;
|
||||||
|
if (!is_trivial)
|
||||||
|
{
|
||||||
|
/// Скомпилируем регулярное выражение re2.
|
||||||
|
typename RegexType::Options options;
|
||||||
|
|
||||||
|
if (is_case_insensitive)
|
||||||
|
options.set_case_sensitive(false);
|
||||||
|
|
||||||
|
if (is_dot_nl)
|
||||||
|
options.set_dot_nl(true);
|
||||||
|
|
||||||
|
re2.reset(new RegexType(regexp_, options));
|
||||||
|
if (!re2->ok())
|
||||||
|
throw Poco::Exception("OptimizedRegularExpression: cannot compile re2: " + regexp_ + ", error: " + re2->error());
|
||||||
|
|
||||||
|
if (!is_no_capture)
|
||||||
|
{
|
||||||
|
number_of_subpatterns = re2->NumberOfCapturingGroups();
|
||||||
|
if (number_of_subpatterns > MAX_SUBPATTERNS)
|
||||||
|
throw Poco::Exception("OptimizedRegularExpression: too many subpatterns in regexp: " + regexp_);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
template <bool b>
|
||||||
|
bool OptimizedRegularExpressionImpl<b>::match(const char * subject, size_t subject_size) const
|
||||||
|
{
|
||||||
|
if (is_trivial)
|
||||||
|
{
|
||||||
|
if (is_case_insensitive)
|
||||||
|
return nullptr != strcasestr(subject, required_substring.data());
|
||||||
|
else
|
||||||
|
return nullptr != strstr(subject, required_substring.data());
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
if (!required_substring.empty())
|
||||||
|
{
|
||||||
|
const char * pos;
|
||||||
|
if (is_case_insensitive)
|
||||||
|
pos = strcasestr(subject, required_substring.data());
|
||||||
|
else
|
||||||
|
pos = strstr(subject, required_substring.data());
|
||||||
|
|
||||||
|
if (nullptr == pos)
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
return re2->Match(StringPieceType(subject, subject_size), 0, subject_size, RegexType::UNANCHORED, nullptr, 0);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
template <bool b>
|
||||||
|
bool OptimizedRegularExpressionImpl<b>::match(const char * subject, size_t subject_size, Match & match) const
|
||||||
|
{
|
||||||
|
if (is_trivial)
|
||||||
|
{
|
||||||
|
const char * pos;
|
||||||
|
if (is_case_insensitive)
|
||||||
|
pos = strcasestr(subject, required_substring.data());
|
||||||
|
else
|
||||||
|
pos = strstr(subject, required_substring.data());
|
||||||
|
|
||||||
|
if (pos == nullptr)
|
||||||
|
return 0;
|
||||||
|
else
|
||||||
|
{
|
||||||
|
match.offset = pos - subject;
|
||||||
|
match.length = required_substring.size();
|
||||||
|
return 1;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
if (!required_substring.empty())
|
||||||
|
{
|
||||||
|
const char * pos;
|
||||||
|
if (is_case_insensitive)
|
||||||
|
pos = strcasestr(subject, required_substring.data());
|
||||||
|
else
|
||||||
|
pos = strstr(subject, required_substring.data());
|
||||||
|
|
||||||
|
if (nullptr == pos)
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
StringPieceType piece;
|
||||||
|
|
||||||
|
if (!RegexType::PartialMatch(StringPieceType(subject, subject_size), *re2, &piece))
|
||||||
|
return 0;
|
||||||
|
else
|
||||||
|
{
|
||||||
|
match.offset = piece.data() - subject;
|
||||||
|
match.length = piece.length();
|
||||||
|
return 1;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
template <bool b>
|
||||||
|
unsigned OptimizedRegularExpressionImpl<b>::match(const char * subject, size_t subject_size, MatchVec & matches, unsigned limit) const
|
||||||
|
{
|
||||||
|
matches.clear();
|
||||||
|
|
||||||
|
if (limit == 0)
|
||||||
|
return 0;
|
||||||
|
|
||||||
|
if (limit > number_of_subpatterns + 1)
|
||||||
|
limit = number_of_subpatterns + 1;
|
||||||
|
|
||||||
|
if (is_trivial)
|
||||||
|
{
|
||||||
|
const char * pos;
|
||||||
|
if (is_case_insensitive)
|
||||||
|
pos = strcasestr(subject, required_substring.data());
|
||||||
|
else
|
||||||
|
pos = strstr(subject, required_substring.data());
|
||||||
|
|
||||||
|
if (pos == nullptr)
|
||||||
|
return 0;
|
||||||
|
else
|
||||||
|
{
|
||||||
|
Match match;
|
||||||
|
match.offset = pos - subject;
|
||||||
|
match.length = required_substring.size();
|
||||||
|
matches.push_back(match);
|
||||||
|
return 1;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
if (!required_substring.empty())
|
||||||
|
{
|
||||||
|
const char * pos;
|
||||||
|
if (is_case_insensitive)
|
||||||
|
pos = strcasestr(subject, required_substring.data());
|
||||||
|
else
|
||||||
|
pos = strstr(subject, required_substring.data());
|
||||||
|
|
||||||
|
if (nullptr == pos)
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
StringPieceType pieces[MAX_SUBPATTERNS];
|
||||||
|
|
||||||
|
if (!re2->Match(StringPieceType(subject, subject_size), 0, subject_size, RegexType::UNANCHORED, pieces, limit))
|
||||||
|
return 0;
|
||||||
|
else
|
||||||
|
{
|
||||||
|
matches.resize(limit);
|
||||||
|
for (size_t i = 0; i < limit; ++i)
|
||||||
|
{
|
||||||
|
if (pieces[i] != nullptr)
|
||||||
|
{
|
||||||
|
matches[i].offset = pieces[i].data() - subject;
|
||||||
|
matches[i].length = pieces[i].length();
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
matches[i].offset = std::string::npos;
|
||||||
|
matches[i].length = 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return limit;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
#undef MIN_LENGTH_FOR_STRSTR
|
||||||
|
#undef MAX_SUBPATTERNS
|
||||||
|
|
@ -8,11 +8,11 @@
|
|||||||
#include <boost/noncopyable.hpp>
|
#include <boost/noncopyable.hpp>
|
||||||
#include <boost/iterator_adaptors.hpp>
|
#include <boost/iterator_adaptors.hpp>
|
||||||
|
|
||||||
#include <Yandex/likely.h>
|
#include <common/likely.h>
|
||||||
#include <Yandex/strong_typedef.h>
|
#include <common/strong_typedef.h>
|
||||||
|
|
||||||
#include <DB/Common/Allocator.h>
|
#include <DB/Common/Allocator.h>
|
||||||
#include <DB/Core/Exception.h>
|
#include <DB/Common/Exception.h>
|
||||||
#include <DB/Core/ErrorCodes.h>
|
#include <DB/Core/ErrorCodes.h>
|
||||||
|
|
||||||
|
|
||||||
|
146
dbms/include/DB/Common/PoolBase.h
Normal file
146
dbms/include/DB/Common/PoolBase.h
Normal file
@ -0,0 +1,146 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Poco/SharedPtr.h>
|
||||||
|
#include <Poco/Mutex.h>
|
||||||
|
#include <Poco/Condition.h>
|
||||||
|
#include <Poco/Timespan.h>
|
||||||
|
#include <boost/noncopyable.hpp>
|
||||||
|
|
||||||
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
|
|
||||||
|
/** Класс, от которого можно унаследоваться и получить пул чего-нибудь. Используется для пулов соединений с БД.
|
||||||
|
* Наследник должен предоставить метод для создания нового объекта для помещения в пул.
|
||||||
|
*/
|
||||||
|
|
||||||
|
template <typename TObject>
|
||||||
|
class PoolBase : private boost::noncopyable
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
typedef TObject Object;
|
||||||
|
typedef Poco::SharedPtr<Object> ObjectPtr;
|
||||||
|
typedef Poco::SharedPtr<PoolBase<TObject> > Ptr;
|
||||||
|
|
||||||
|
private:
|
||||||
|
|
||||||
|
/** Объект с флагом, используется ли он сейчас. */
|
||||||
|
struct PooledObject
|
||||||
|
{
|
||||||
|
PooledObject(Poco::Condition & available_, ObjectPtr object_)
|
||||||
|
: object(object_), available(available_)
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
ObjectPtr object;
|
||||||
|
bool in_use = false;
|
||||||
|
Poco::Condition & available;
|
||||||
|
};
|
||||||
|
|
||||||
|
typedef std::vector<Poco::SharedPtr<PooledObject> > Objects;
|
||||||
|
|
||||||
|
/** Помощник, который устанавливает флаг использования объекта, а в деструкторе - снимает,
|
||||||
|
* а также уведомляет о событии с помощью condvar-а.
|
||||||
|
*/
|
||||||
|
struct PoolEntryHelper
|
||||||
|
{
|
||||||
|
PoolEntryHelper(PooledObject & data_) : data(data_) { data.in_use = true; }
|
||||||
|
~PoolEntryHelper() { data.in_use = false; data.available.signal(); }
|
||||||
|
|
||||||
|
PooledObject & data;
|
||||||
|
};
|
||||||
|
|
||||||
|
public:
|
||||||
|
/** То, что выдаётся пользователю. */
|
||||||
|
class Entry
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
friend class PoolBase<Object>;
|
||||||
|
|
||||||
|
Entry() {} /// Для отложенной инициализации.
|
||||||
|
|
||||||
|
/** Объект Entry защищает ресурс от использования другим потоком.
|
||||||
|
* Следующие методы запрещены для rvalue, чтобы нельзя было написать подобное
|
||||||
|
*
|
||||||
|
* sqxxl::Query q = pool.Get()->query("SELECT .."); // Упс, после этой строчки Entry уничтожился
|
||||||
|
* q.execute(); // Кто-то еще может использовать этот Connection
|
||||||
|
*/
|
||||||
|
Object * operator->() && = delete;
|
||||||
|
const Object * operator->() const && = delete;
|
||||||
|
Object & operator*() && = delete;
|
||||||
|
const Object & operator*() const && = delete;
|
||||||
|
|
||||||
|
Object * operator->() & { return &*data->data.object; }
|
||||||
|
const Object * operator->() const & { return &*data->data.object; }
|
||||||
|
Object & operator*() & { return *data->data.object; }
|
||||||
|
const Object & operator*() const & { return *data->data.object; }
|
||||||
|
|
||||||
|
bool isNull() const { return data.isNull(); }
|
||||||
|
|
||||||
|
private:
|
||||||
|
Poco::SharedPtr<PoolEntryHelper> data;
|
||||||
|
|
||||||
|
Entry(PooledObject & object) : data(new PoolEntryHelper(object)) {}
|
||||||
|
};
|
||||||
|
|
||||||
|
virtual ~PoolBase() {}
|
||||||
|
|
||||||
|
/** Выделяет объект для работы. При timeout < 0 таймаут бесконечный. */
|
||||||
|
Entry get(Poco::Timespan::TimeDiff timeout)
|
||||||
|
{
|
||||||
|
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||||
|
|
||||||
|
while (true)
|
||||||
|
{
|
||||||
|
for (typename Objects::iterator it = items.begin(); it != items.end(); it++)
|
||||||
|
if (!(*it)->in_use)
|
||||||
|
return Entry(**it);
|
||||||
|
|
||||||
|
if (items.size() < max_items)
|
||||||
|
{
|
||||||
|
ObjectPtr object = allocObject();
|
||||||
|
items.push_back(new PooledObject(available, object));
|
||||||
|
return Entry(*items.back());
|
||||||
|
}
|
||||||
|
|
||||||
|
LOG_INFO(log, "No free connections in pool. Waiting.");
|
||||||
|
|
||||||
|
if (timeout < 0)
|
||||||
|
available.wait(mutex);
|
||||||
|
else
|
||||||
|
available.wait(mutex, timeout);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void reserve(size_t count)
|
||||||
|
{
|
||||||
|
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||||
|
|
||||||
|
while (items.size() < count)
|
||||||
|
items.push_back(new PooledObject(available, allocObject()));
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
/** Максимальный размер пула. */
|
||||||
|
unsigned max_items;
|
||||||
|
|
||||||
|
/** Пул. */
|
||||||
|
Objects items;
|
||||||
|
|
||||||
|
/** Блокировка для доступа к пулу. */
|
||||||
|
Poco::FastMutex mutex;
|
||||||
|
Poco::Condition available;
|
||||||
|
|
||||||
|
protected:
|
||||||
|
|
||||||
|
Logger * log;
|
||||||
|
|
||||||
|
PoolBase(unsigned max_items_, Logger * log_)
|
||||||
|
: max_items(max_items_), log(log_)
|
||||||
|
{
|
||||||
|
items.reserve(max_items);
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Создает новый объект для помещения в пул. */
|
||||||
|
virtual ObjectPtr allocObject() = 0;
|
||||||
|
};
|
||||||
|
|
311
dbms/include/DB/Common/PoolWithFailoverBase.h
Normal file
311
dbms/include/DB/Common/PoolWithFailoverBase.h
Normal file
@ -0,0 +1,311 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <time.h>
|
||||||
|
#include <DB/Common/PoolBase.h>
|
||||||
|
#include <DB/Common/ProfileEvents.h>
|
||||||
|
#include <DB/Common/NetException.h>
|
||||||
|
#include <DB/Common/Exception.h>
|
||||||
|
#include <DB/Core/ErrorCodes.h>
|
||||||
|
#include <DB/Interpreters/Settings.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
/** Класс, который употребляется для того, чтобы оптимизировать выделение
|
||||||
|
* нескольких ресурсов в PoolWithFailoverBase. Проверки границ не проводятся,
|
||||||
|
* потому что мы предполагаем, что PoolWithFailoverBase делает все нужные
|
||||||
|
* проверки.
|
||||||
|
*/
|
||||||
|
class ResourceTracker
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
ResourceTracker(size_t s)
|
||||||
|
: handles(s), unallocated_size(s)
|
||||||
|
{
|
||||||
|
size_t i = 0;
|
||||||
|
for (auto & index : handles)
|
||||||
|
{
|
||||||
|
index = i;
|
||||||
|
++i;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
size_t getHandle(size_t i) const
|
||||||
|
{
|
||||||
|
return handles[i];
|
||||||
|
}
|
||||||
|
|
||||||
|
size_t getUnallocatedSize() const
|
||||||
|
{
|
||||||
|
return unallocated_size;
|
||||||
|
}
|
||||||
|
|
||||||
|
void markAsAllocated(size_t i)
|
||||||
|
{
|
||||||
|
std::swap(handles[i], handles[unallocated_size - 1]);
|
||||||
|
--unallocated_size;
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
std::vector<size_t> handles;
|
||||||
|
size_t unallocated_size;
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Класс, от которого можно унаследоваться и получить пул с отказоустойчивостью. Используется для пулов соединений с реплицированной БД.
|
||||||
|
* Инициализируется несколькими другими PoolBase-ами.
|
||||||
|
* При получении соединения, пытается создать или выбрать живое соединение из какого-нибудь пула,
|
||||||
|
* перебирая их в некотором порядке, используя не более указанного количества попыток.
|
||||||
|
* Пулы перебираются в порядке лексикографического возрастания тройки (приоритет, число ошибок, случайное число).
|
||||||
|
*
|
||||||
|
* Замечание: если один из вложенных пулов заблокируется из-за переполнения, то этот пул тоже заблокируется.
|
||||||
|
*
|
||||||
|
* Наследник должен предоставить метод, достающий соединение из вложенного пула.
|
||||||
|
* Еще наследник можнет назначать приоритеты вложенным пулам.
|
||||||
|
*/
|
||||||
|
|
||||||
|
template <typename TNestedPool>
|
||||||
|
class PoolWithFailoverBase : private boost::noncopyable
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
typedef TNestedPool NestedPool;
|
||||||
|
typedef Poco::SharedPtr<NestedPool> NestedPoolPtr;
|
||||||
|
typedef typename NestedPool::Entry Entry;
|
||||||
|
typedef std::vector<NestedPoolPtr> NestedPools;
|
||||||
|
|
||||||
|
virtual ~PoolWithFailoverBase() {}
|
||||||
|
|
||||||
|
PoolWithFailoverBase(NestedPools & nested_pools_,
|
||||||
|
size_t max_tries_,
|
||||||
|
time_t decrease_error_period_,
|
||||||
|
Logger * log_)
|
||||||
|
: nested_pools(nested_pools_.begin(), nested_pools_.end(), decrease_error_period_), max_tries(max_tries_),
|
||||||
|
log(log_)
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Выделяет соединение для работы. */
|
||||||
|
Entry get(const DB::Settings * settings)
|
||||||
|
{
|
||||||
|
Entry entry;
|
||||||
|
std::stringstream fail_messages;
|
||||||
|
|
||||||
|
bool skip_unavailable = settings ? UInt64(settings->skip_unavailable_shards) : false;
|
||||||
|
|
||||||
|
if (getResource(entry, fail_messages, nullptr, settings))
|
||||||
|
return entry;
|
||||||
|
else if (!skip_unavailable)
|
||||||
|
throw DB::NetException("All connection tries failed. Log: \n\n" + fail_messages.str() + "\n", DB::ErrorCodes::ALL_CONNECTION_TRIES_FAILED);
|
||||||
|
else
|
||||||
|
return {};
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Выделяет до указанного количества соединений для работы
|
||||||
|
* Соединения предоставляют доступ к разным репликам одного шарда.
|
||||||
|
*/
|
||||||
|
std::vector<Entry> getMany(const DB::Settings * settings)
|
||||||
|
{
|
||||||
|
ResourceTracker resource_tracker{nested_pools.size()};
|
||||||
|
|
||||||
|
UInt64 max_connections = settings ? UInt64(settings->max_parallel_replicas) : 1;
|
||||||
|
bool skip_unavailable = settings ? UInt64(settings->skip_unavailable_shards) : false;
|
||||||
|
|
||||||
|
std::vector<Entry> connections;
|
||||||
|
connections.reserve(max_connections);
|
||||||
|
|
||||||
|
for (UInt64 i = 0; i < max_connections; ++i)
|
||||||
|
{
|
||||||
|
Entry entry;
|
||||||
|
std::stringstream fail_messages;
|
||||||
|
|
||||||
|
if (getResource(entry, fail_messages, &resource_tracker, settings))
|
||||||
|
connections.push_back(entry);
|
||||||
|
else if (i == 0 && !skip_unavailable)
|
||||||
|
throw DB::NetException("All connection tries failed. Log: \n\n" + fail_messages.str() + "\n", DB::ErrorCodes::ALL_CONNECTION_TRIES_FAILED);
|
||||||
|
else
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
|
return connections;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
protected:
|
||||||
|
struct PoolWithErrorCount
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
PoolWithErrorCount(const NestedPoolPtr & pool_) : pool(pool_)
|
||||||
|
{
|
||||||
|
struct timespec times;
|
||||||
|
if (clock_gettime(CLOCK_THREAD_CPUTIME_ID, ×))
|
||||||
|
DB::throwFromErrno("Cannot clock_gettime.", DB::ErrorCodes::CANNOT_CLOCK_GETTIME);
|
||||||
|
|
||||||
|
srand48_r(reinterpret_cast<intptr_t>(this) ^ times.tv_nsec, &rand_state);
|
||||||
|
}
|
||||||
|
|
||||||
|
void randomize()
|
||||||
|
{
|
||||||
|
long int rand_res;
|
||||||
|
lrand48_r(&rand_state, &rand_res);
|
||||||
|
state.random = rand_res;
|
||||||
|
}
|
||||||
|
|
||||||
|
public:
|
||||||
|
struct State
|
||||||
|
{
|
||||||
|
static bool compare(const State & lhs, const State & rhs)
|
||||||
|
{
|
||||||
|
return std::tie(lhs.priority, lhs.error_count, lhs.random)
|
||||||
|
< std::tie(rhs.priority, rhs.error_count, rhs.random);
|
||||||
|
}
|
||||||
|
|
||||||
|
Int64 priority = 0;
|
||||||
|
UInt64 error_count = 0;
|
||||||
|
UInt32 random = 0;
|
||||||
|
};
|
||||||
|
|
||||||
|
public:
|
||||||
|
NestedPoolPtr pool;
|
||||||
|
State state;
|
||||||
|
drand48_data rand_state;
|
||||||
|
};
|
||||||
|
|
||||||
|
using States = std::vector<typename PoolWithErrorCount::State>;
|
||||||
|
|
||||||
|
class PoolsWithErrorCount : public std::vector<PoolWithErrorCount>
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
PoolsWithErrorCount(typename NestedPools::iterator begin_, typename NestedPools::iterator end_,
|
||||||
|
time_t decrease_error_period_)
|
||||||
|
: std::vector<PoolWithErrorCount>(begin_, end_),
|
||||||
|
decrease_error_period(decrease_error_period_)
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Эта функция возвращает собственную копию состояния каждого пула, чтобы не возникло
|
||||||
|
/// состояния гонки при выделении соединений.
|
||||||
|
States update()
|
||||||
|
{
|
||||||
|
States states;
|
||||||
|
states.reserve(this->size());
|
||||||
|
|
||||||
|
{
|
||||||
|
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||||
|
|
||||||
|
for (auto & pool : *this)
|
||||||
|
pool.randomize();
|
||||||
|
|
||||||
|
/// Каждые N секунд уменьшаем количество ошибок в 2 раза
|
||||||
|
time_t current_time = time(0);
|
||||||
|
|
||||||
|
if (last_decrease_time)
|
||||||
|
{
|
||||||
|
time_t delta = current_time - last_decrease_time;
|
||||||
|
|
||||||
|
if (delta >= 0)
|
||||||
|
{
|
||||||
|
/// Каждые decrease_error_period секунд, делим количество ошибок на два.
|
||||||
|
size_t shift_amount = delta / decrease_error_period;
|
||||||
|
/// обновляем время, не чаще раз в период
|
||||||
|
/// в противном случае при частых вызовах счетчик никогда не будет уменьшаться
|
||||||
|
if (shift_amount)
|
||||||
|
last_decrease_time = current_time;
|
||||||
|
|
||||||
|
if (shift_amount >= sizeof(UInt64))
|
||||||
|
{
|
||||||
|
for (auto & pool : *this)
|
||||||
|
pool.state.error_count = 0;
|
||||||
|
}
|
||||||
|
else if (shift_amount)
|
||||||
|
{
|
||||||
|
for (auto & pool : *this)
|
||||||
|
pool.state.error_count >>= shift_amount;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
else
|
||||||
|
last_decrease_time = current_time;
|
||||||
|
|
||||||
|
for (auto & pool : *this)
|
||||||
|
states.push_back(pool.state);
|
||||||
|
}
|
||||||
|
|
||||||
|
return states;
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
/// Время, когда последний раз уменьшался счётчик ошибок.
|
||||||
|
time_t last_decrease_time = 0;
|
||||||
|
time_t decrease_error_period;
|
||||||
|
Poco::FastMutex mutex;
|
||||||
|
};
|
||||||
|
|
||||||
|
PoolsWithErrorCount nested_pools;
|
||||||
|
size_t max_tries;
|
||||||
|
Logger * log;
|
||||||
|
|
||||||
|
virtual bool tryGet(NestedPoolPtr pool, const DB::Settings * settings, Entry & out_entry, std::stringstream & fail_message) = 0;
|
||||||
|
|
||||||
|
|
||||||
|
private:
|
||||||
|
/** Выделяет соединение из одной реплики для работы. */
|
||||||
|
bool getResource(Entry & entry, std::stringstream & fail_messages, ResourceTracker * resource_tracker, const DB::Settings * settings)
|
||||||
|
{
|
||||||
|
/// Обновление случайных чисел, а также счётчиков ошибок.
|
||||||
|
States states = nested_pools.update();
|
||||||
|
|
||||||
|
struct IndexedPoolWithErrorCount
|
||||||
|
{
|
||||||
|
PoolWithErrorCount * pool;
|
||||||
|
typename PoolWithErrorCount::State * state;
|
||||||
|
size_t index;
|
||||||
|
};
|
||||||
|
|
||||||
|
using PoolPtrs = std::vector<IndexedPoolWithErrorCount>;
|
||||||
|
|
||||||
|
size_t pools_size = resource_tracker ? resource_tracker->getUnallocatedSize() : nested_pools.size();
|
||||||
|
PoolPtrs pool_ptrs(pools_size);
|
||||||
|
|
||||||
|
for (size_t i = 0; i < pools_size; ++i)
|
||||||
|
{
|
||||||
|
auto & record = pool_ptrs[i];
|
||||||
|
size_t pool_index = resource_tracker ? resource_tracker->getHandle(i) : i;
|
||||||
|
record.pool = &nested_pools[pool_index];
|
||||||
|
record.state = &states[pool_index];
|
||||||
|
record.index = i;
|
||||||
|
}
|
||||||
|
|
||||||
|
std::sort(pool_ptrs.begin(), pool_ptrs.end(),
|
||||||
|
[](const IndexedPoolWithErrorCount & lhs, const IndexedPoolWithErrorCount & rhs)
|
||||||
|
{
|
||||||
|
return PoolWithErrorCount::State::compare(*(lhs.state), *(rhs.state));
|
||||||
|
});
|
||||||
|
|
||||||
|
for (size_t try_no = 0; try_no < max_tries; ++try_no)
|
||||||
|
{
|
||||||
|
for (size_t i = 0; i < pools_size; ++i)
|
||||||
|
{
|
||||||
|
std::stringstream fail_message;
|
||||||
|
|
||||||
|
if (tryGet(pool_ptrs[i].pool->pool, settings, entry, fail_message))
|
||||||
|
{
|
||||||
|
if (resource_tracker)
|
||||||
|
resource_tracker->markAsAllocated(pool_ptrs[i].index);
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
ProfileEvents::increment(ProfileEvents::DistributedConnectionFailTry);
|
||||||
|
|
||||||
|
LOG_WARNING(log, "Connection failed at try №"
|
||||||
|
<< (try_no + 1) << ", reason: " << fail_message.str());
|
||||||
|
|
||||||
|
fail_messages << fail_message.str() << std::endl;
|
||||||
|
|
||||||
|
__sync_fetch_and_add(&pool_ptrs[i].pool->state.error_count, 1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
ProfileEvents::increment(ProfileEvents::DistributedConnectionFailAtAll);
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
};
|
55
dbms/include/DB/Common/SimpleCache.h
Normal file
55
dbms/include/DB/Common/SimpleCache.h
Normal file
@ -0,0 +1,55 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <map>
|
||||||
|
#include <tuple>
|
||||||
|
#include <mutex>
|
||||||
|
#include <ext/function_traits.hpp>
|
||||||
|
|
||||||
|
|
||||||
|
/** Простейший кэш для свободной функции.
|
||||||
|
* Можете также передать статический метод класса или лямбду без захвата.
|
||||||
|
* Размер неограничен. Значения не устаревают.
|
||||||
|
* Для синхронизации используется mutex.
|
||||||
|
* Подходит только для простейших случаев.
|
||||||
|
*
|
||||||
|
* Использование:
|
||||||
|
*
|
||||||
|
* SimpleCache<decltype(func), &func> func_cached;
|
||||||
|
* std::cerr << func_cached(args...);
|
||||||
|
*/
|
||||||
|
template <typename F, F* f>
|
||||||
|
class SimpleCache
|
||||||
|
{
|
||||||
|
private:
|
||||||
|
using Key = typename function_traits<F>::arguments_decay;
|
||||||
|
using Result = typename function_traits<F>::result;
|
||||||
|
|
||||||
|
std::map<Key, Result> cache;
|
||||||
|
std::mutex mutex;
|
||||||
|
|
||||||
|
public:
|
||||||
|
template <typename... Args>
|
||||||
|
Result operator() (Args &&... args)
|
||||||
|
{
|
||||||
|
{
|
||||||
|
std::lock_guard<std::mutex> lock(mutex);
|
||||||
|
|
||||||
|
Key key{std::forward<Args>(args)...};
|
||||||
|
auto it = cache.find(key);
|
||||||
|
|
||||||
|
if (cache.end() != it)
|
||||||
|
return it->second;
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Сами вычисления делаются не под mutex-ом.
|
||||||
|
Result res = f(std::forward<Args>(args)...);
|
||||||
|
|
||||||
|
{
|
||||||
|
std::lock_guard<std::mutex> lock(mutex);
|
||||||
|
|
||||||
|
cache.emplace(std::forward_as_tuple(args...), res);
|
||||||
|
}
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
};
|
23
dbms/include/DB/Common/StackTrace.h
Normal file
23
dbms/include/DB/Common/StackTrace.h
Normal file
@ -0,0 +1,23 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <string>
|
||||||
|
#include <vector>
|
||||||
|
|
||||||
|
#define STACK_TRACE_MAX_DEPTH 32
|
||||||
|
|
||||||
|
|
||||||
|
/// Позволяет получить стек-трейс
|
||||||
|
class StackTrace
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
/// Стектрейс снимается в момент создания объекта
|
||||||
|
StackTrace();
|
||||||
|
|
||||||
|
/// Вывести в строку
|
||||||
|
std::string toString() const;
|
||||||
|
|
||||||
|
private:
|
||||||
|
typedef void* Frame;
|
||||||
|
Frame frames[STACK_TRACE_MAX_DEPTH];
|
||||||
|
size_t frames_size;
|
||||||
|
};
|
75
dbms/include/DB/Common/Stopwatch.h
Normal file
75
dbms/include/DB/Common/Stopwatch.h
Normal file
@ -0,0 +1,75 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <time.h>
|
||||||
|
#include <Poco/Mutex.h>
|
||||||
|
#include <Poco/ScopedLock.h>
|
||||||
|
#include <common/Common.h>
|
||||||
|
|
||||||
|
|
||||||
|
/** Отличается от Poco::Stopwatch только тем, что использует clock_gettime вместо gettimeofday,
|
||||||
|
* возвращает наносекунды вместо микросекунд, а также другими незначительными отличиями.
|
||||||
|
*/
|
||||||
|
class Stopwatch
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
/** CLOCK_MONOTONIC работает сравнительно эффективно (~15 млн. вызовов в сек.) и не приводит к системному вызову.
|
||||||
|
* Поставьте CLOCK_MONOTONIC_COARSE, если нужна больше производительность, но достаточно погрешности в несколько мс.
|
||||||
|
*/
|
||||||
|
Stopwatch(clockid_t clock_type_ = CLOCK_MONOTONIC) : clock_type(clock_type_) { restart(); }
|
||||||
|
|
||||||
|
void start() { setStart(); is_running = true; }
|
||||||
|
void stop() { updateElapsed(); is_running = false; }
|
||||||
|
void restart() { elapsed_ns = 0; start(); }
|
||||||
|
UInt64 elapsed() const { updateElapsed(); return elapsed_ns; }
|
||||||
|
double elapsedSeconds() const { updateElapsed(); return static_cast<double>(elapsed_ns) / 1000000000ULL; }
|
||||||
|
|
||||||
|
private:
|
||||||
|
mutable UInt64 start_ns;
|
||||||
|
mutable UInt64 elapsed_ns;
|
||||||
|
clockid_t clock_type;
|
||||||
|
bool is_running;
|
||||||
|
|
||||||
|
void setStart()
|
||||||
|
{
|
||||||
|
struct timespec ts;
|
||||||
|
clock_gettime(clock_type, &ts);
|
||||||
|
start_ns = ts.tv_sec * 1000000000ULL + ts.tv_nsec;
|
||||||
|
}
|
||||||
|
|
||||||
|
void updateElapsed() const
|
||||||
|
{
|
||||||
|
if (is_running)
|
||||||
|
{
|
||||||
|
struct timespec ts;
|
||||||
|
clock_gettime(clock_type, &ts);
|
||||||
|
UInt64 current_ns = ts.tv_sec * 1000000000ULL + ts.tv_nsec;
|
||||||
|
elapsed_ns += current_ns - start_ns;
|
||||||
|
start_ns = current_ns;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
class StopwatchWithLock : public Stopwatch
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
/** Если прошло указанное количество секунд, то перезапускает таймер и возвращает true.
|
||||||
|
* Иначе возвращает false.
|
||||||
|
* thread-safe.
|
||||||
|
*/
|
||||||
|
bool lockTestAndRestart(double seconds)
|
||||||
|
{
|
||||||
|
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||||
|
|
||||||
|
if (elapsedSeconds() >= seconds)
|
||||||
|
{
|
||||||
|
restart();
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
else
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
Poco::FastMutex mutex;
|
||||||
|
};
|
@ -2,58 +2,79 @@
|
|||||||
|
|
||||||
#include <mutex>
|
#include <mutex>
|
||||||
#include <memory>
|
#include <memory>
|
||||||
#include <statdaemons/Stopwatch.h>
|
#include <DB/Common/Stopwatch.h>
|
||||||
|
#include <DB/Common/Exception.h>
|
||||||
|
#include <DB/IO/WriteHelpers.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
/** Позволяет ограничить скорость чего либо (в штуках в секунду) с помощью sleep.
|
/** Позволяет ограничить скорость чего либо (в штуках в секунду) с помощью sleep.
|
||||||
* Особенности работы:
|
* Особенности работы:
|
||||||
* - считается только средняя скорость, от момента первого вызова функции add;
|
* - считается только средняя скорость, от момента первого вызова функции add;
|
||||||
* если были периоды с низкой скоростью, то в течение промежутка времени после них, скорость будет выше;
|
* если были периоды с низкой скоростью, то в течение промежутка времени после них, скорость будет выше;
|
||||||
|
*
|
||||||
|
* Также позволяет задать ограничение на максимальное количество в штуках. При превышении кидается исключение.
|
||||||
*/
|
*/
|
||||||
class Throttler
|
class Throttler
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
Throttler(size_t max_speed_) : max_speed(max_speed_) {}
|
Throttler(size_t max_speed_, size_t limit_, const char * limit_exceeded_exception_message_)
|
||||||
|
: max_speed(max_speed_), limit(limit_), limit_exceeded_exception_message(limit_exceeded_exception_message_) {}
|
||||||
|
|
||||||
void add(size_t amount)
|
void add(size_t amount)
|
||||||
{
|
{
|
||||||
size_t new_count;
|
size_t new_count;
|
||||||
UInt64 elapsed_ns;
|
UInt64 elapsed_ns = 0;
|
||||||
|
|
||||||
{
|
{
|
||||||
std::lock_guard<std::mutex> lock(mutex);
|
std::lock_guard<std::mutex> lock(mutex);
|
||||||
|
|
||||||
if (0 == count)
|
if (max_speed)
|
||||||
{
|
{
|
||||||
watch.start();
|
if (0 == count)
|
||||||
elapsed_ns = 0;
|
{
|
||||||
|
watch.start();
|
||||||
|
elapsed_ns = 0;
|
||||||
|
}
|
||||||
|
else
|
||||||
|
elapsed_ns = watch.elapsed();
|
||||||
}
|
}
|
||||||
else
|
|
||||||
elapsed_ns = watch.elapsed();
|
|
||||||
|
|
||||||
count += amount;
|
count += amount;
|
||||||
new_count = count;
|
new_count = count;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Сколько должно было бы пройти времени, если бы скорость была равна max_speed.
|
if (limit && new_count > limit)
|
||||||
UInt64 desired_ns = new_count * 1000000000 / max_speed;
|
throw Exception(limit_exceeded_exception_message + std::string(" Maximum: ") + toString(limit), ErrorCodes::LIMIT_EXCEEDED);
|
||||||
|
|
||||||
if (desired_ns > elapsed_ns)
|
if (max_speed)
|
||||||
{
|
{
|
||||||
UInt64 sleep_ns = desired_ns - elapsed_ns;
|
/// Сколько должно было бы пройти времени, если бы скорость была равна max_speed.
|
||||||
timespec sleep_ts;
|
UInt64 desired_ns = new_count * 1000000000 / max_speed;
|
||||||
sleep_ts.tv_sec = sleep_ns / 1000000000;
|
|
||||||
sleep_ts.tv_nsec = sleep_ns % 1000000000;
|
if (desired_ns > elapsed_ns)
|
||||||
nanosleep(&sleep_ts, nullptr); /// NOTE Завершается раньше в случае сигнала. Это считается нормальным.
|
{
|
||||||
|
UInt64 sleep_ns = desired_ns - elapsed_ns;
|
||||||
|
timespec sleep_ts;
|
||||||
|
sleep_ts.tv_sec = sleep_ns / 1000000000;
|
||||||
|
sleep_ts.tv_nsec = sleep_ns % 1000000000;
|
||||||
|
nanosleep(&sleep_ts, nullptr); /// NOTE Завершается раньше в случае сигнала. Это считается нормальным.
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private:
|
private:
|
||||||
size_t max_speed;
|
size_t max_speed = 0;
|
||||||
size_t count = 0;
|
size_t count = 0;
|
||||||
|
size_t limit = 0; /// 0 - не ограничено.
|
||||||
|
const char * limit_exceeded_exception_message = nullptr;
|
||||||
Stopwatch watch {CLOCK_MONOTONIC_COARSE};
|
Stopwatch watch {CLOCK_MONOTONIC_COARSE};
|
||||||
std::mutex mutex;
|
std::mutex mutex;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
typedef std::shared_ptr<Throttler> ThrottlerPtr;
|
typedef std::shared_ptr<Throttler> ThrottlerPtr;
|
||||||
|
|
||||||
|
}
|
||||||
|
@ -89,6 +89,9 @@ public:
|
|||||||
/// Если не найдено - возвращается конец haystack.
|
/// Если не найдено - возвращается конец haystack.
|
||||||
const char * search(const char * haystack, size_t haystack_size) const
|
const char * search(const char * haystack, size_t haystack_size) const
|
||||||
{
|
{
|
||||||
|
if (needle_size == 0)
|
||||||
|
return haystack;
|
||||||
|
|
||||||
const char * haystack_end = haystack + haystack_size;
|
const char * haystack_end = haystack + haystack_size;
|
||||||
|
|
||||||
if (needle_size == 1)
|
if (needle_size == 1)
|
||||||
|
@ -1,13 +1,17 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <string>
|
#include <string>
|
||||||
|
#include <DB/IO/WriteBuffer.h>
|
||||||
|
|
||||||
|
|
||||||
/// Выводит переданный размер в байтах в виде 123.45 GiB.
|
/// Выводит переданный размер в байтах в виде 123.45 GiB.
|
||||||
|
void formatReadableSizeWithBinarySuffix(double value, DB::WriteBuffer & out, int precision = 2);
|
||||||
std::string formatReadableSizeWithBinarySuffix(double value, int precision = 2);
|
std::string formatReadableSizeWithBinarySuffix(double value, int precision = 2);
|
||||||
|
|
||||||
/// Выводит переданный размер в байтах в виде 132.55 GB.
|
/// Выводит переданный размер в байтах в виде 132.55 GB.
|
||||||
|
void formatReadableSizeWithDecimalSuffix(double value, DB::WriteBuffer & out, int precision = 2);
|
||||||
std::string formatReadableSizeWithDecimalSuffix(double value, int precision = 2);
|
std::string formatReadableSizeWithDecimalSuffix(double value, int precision = 2);
|
||||||
|
|
||||||
/// Выводит число в виде 123.45 billion.
|
/// Выводит число в виде 123.45 billion.
|
||||||
|
void formatReadableQuantity(double value, DB::WriteBuffer & out, int precision = 2);
|
||||||
std::string formatReadableQuantity(double value, int precision = 2);
|
std::string formatReadableQuantity(double value, int precision = 2);
|
||||||
|
7
dbms/include/DB/Common/setThreadName.h
Normal file
7
dbms/include/DB/Common/setThreadName.h
Normal file
@ -0,0 +1,7 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
/** Устанавливает имя потока (максимальная длина - 15 байт),
|
||||||
|
* которое будет видно в ps, gdb, /proc,
|
||||||
|
* для удобства наблюдений и отладки.
|
||||||
|
*/
|
||||||
|
void setThreadName(const char * name);
|
@ -8,7 +8,7 @@
|
|||||||
#include <DB/Core/BlockInfo.h>
|
#include <DB/Core/BlockInfo.h>
|
||||||
#include <DB/Core/ColumnWithTypeAndName.h>
|
#include <DB/Core/ColumnWithTypeAndName.h>
|
||||||
#include <DB/Core/NamesAndTypes.h>
|
#include <DB/Core/NamesAndTypes.h>
|
||||||
#include <DB/Core/Exception.h>
|
#include <DB/Common/Exception.h>
|
||||||
#include <DB/Core/ErrorCodes.h>
|
#include <DB/Core/ErrorCodes.h>
|
||||||
#include "ColumnsWithTypeAndName.h"
|
#include "ColumnsWithTypeAndName.h"
|
||||||
|
|
||||||
|
@ -1,7 +1,7 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <DB/Core/Types.h>
|
#include <DB/Core/Types.h>
|
||||||
#include <DB/Core/Exception.h>
|
#include <DB/Common/Exception.h>
|
||||||
#include <DB/Core/ErrorCodes.h>
|
#include <DB/Core/ErrorCodes.h>
|
||||||
#include <DB/IO/ReadBuffer.h>
|
#include <DB/IO/ReadBuffer.h>
|
||||||
#include <DB/IO/WriteBuffer.h>
|
#include <DB/IO/WriteBuffer.h>
|
||||||
|
@ -20,7 +20,6 @@ namespace ErrorCodes
|
|||||||
POSITION_OUT_OF_BOUND = 11,
|
POSITION_OUT_OF_BOUND = 11,
|
||||||
PARAMETER_OUT_OF_BOUND = 12,
|
PARAMETER_OUT_OF_BOUND = 12,
|
||||||
SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH = 13,
|
SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH = 13,
|
||||||
EMPTY_TUPLE = 14,
|
|
||||||
DUPLICATE_COLUMN = 15,
|
DUPLICATE_COLUMN = 15,
|
||||||
NO_SUCH_COLUMN_IN_TABLE = 16,
|
NO_SUCH_COLUMN_IN_TABLE = 16,
|
||||||
DELIMITER_IN_STRING_LITERAL_DOESNT_MATCH = 17,
|
DELIMITER_IN_STRING_LITERAL_DOESNT_MATCH = 17,
|
||||||
@ -253,7 +252,6 @@ namespace ErrorCodes
|
|||||||
CORRUPTED_DATA = 246,
|
CORRUPTED_DATA = 246,
|
||||||
INCORRECT_MARK = 247,
|
INCORRECT_MARK = 247,
|
||||||
INVALID_PARTITION_NAME = 248,
|
INVALID_PARTITION_NAME = 248,
|
||||||
NOT_LEADER = 249,
|
|
||||||
NOT_ENOUGH_BLOCK_NUMBERS = 250,
|
NOT_ENOUGH_BLOCK_NUMBERS = 250,
|
||||||
NO_SUCH_REPLICA = 251,
|
NO_SUCH_REPLICA = 251,
|
||||||
TOO_MUCH_PARTS = 252,
|
TOO_MUCH_PARTS = 252,
|
||||||
@ -286,7 +284,16 @@ namespace ErrorCodes
|
|||||||
NO_AVAILABLE_DATA = 280,
|
NO_AVAILABLE_DATA = 280,
|
||||||
DICTIONARY_IS_EMPTY = 281,
|
DICTIONARY_IS_EMPTY = 281,
|
||||||
INCORRECT_INDEX = 282,
|
INCORRECT_INDEX = 282,
|
||||||
|
UNKNOWN_DISTRIBUTED_PRODUCT_MODE = 283,
|
||||||
UNKNOWN_GLOBAL_SUBQUERIES_METHOD = 284,
|
UNKNOWN_GLOBAL_SUBQUERIES_METHOD = 284,
|
||||||
|
TOO_LESS_LIVE_REPLICAS = 285,
|
||||||
|
UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE = 286,
|
||||||
|
UNKNOWN_FORMAT_VERSION = 287,
|
||||||
|
DISTRIBUTED_IN_JOIN_SUBQUERY_DENIED = 288,
|
||||||
|
REPLICA_IS_NOT_IN_QUORUM = 289,
|
||||||
|
LIMIT_EXCEEDED = 290,
|
||||||
|
DATABASE_ACCESS_DENIED = 291,
|
||||||
|
LEADERSHIP_CHANGED = 292,
|
||||||
|
|
||||||
KEEPER_EXCEPTION = 999,
|
KEEPER_EXCEPTION = 999,
|
||||||
POCO_EXCEPTION = 1000,
|
POCO_EXCEPTION = 1000,
|
||||||
|
@ -1,40 +0,0 @@
|
|||||||
#pragma once
|
|
||||||
|
|
||||||
#include <cerrno>
|
|
||||||
#include <vector>
|
|
||||||
|
|
||||||
#include <statdaemons/Exception.h>
|
|
||||||
#include <Poco/SharedPtr.h>
|
|
||||||
|
|
||||||
namespace Poco { class Logger; }
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
using Poco::SharedPtr;
|
|
||||||
|
|
||||||
typedef SharedPtr<Poco::Exception> ExceptionPtr;
|
|
||||||
typedef std::vector<ExceptionPtr> Exceptions;
|
|
||||||
|
|
||||||
|
|
||||||
void throwFromErrno(const std::string & s, int code = 0, int the_errno = errno);
|
|
||||||
|
|
||||||
|
|
||||||
/** Для использования в блоке catch (...).
|
|
||||||
* Преобразует Exception, Poco::Exception, std::exception или неизвестный exception в ExceptionPtr.
|
|
||||||
*/
|
|
||||||
ExceptionPtr cloneCurrentException();
|
|
||||||
|
|
||||||
/** Попробовать записать исключение в лог (и забыть про него).
|
|
||||||
* Можно использовать в деструкторах в блоке catch (...).
|
|
||||||
*/
|
|
||||||
void tryLogCurrentException(const char * log_name, const std::string & start_of_message = "");
|
|
||||||
void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message = "");
|
|
||||||
|
|
||||||
std::string getCurrentExceptionMessage(bool with_stacktrace);
|
|
||||||
|
|
||||||
|
|
||||||
void rethrowFirstException(Exceptions & exceptions);
|
|
||||||
|
|
||||||
|
|
||||||
}
|
|
@ -11,7 +11,7 @@
|
|||||||
#include <mysqlxx/Manip.h>
|
#include <mysqlxx/Manip.h>
|
||||||
|
|
||||||
#include <DB/Core/Types.h>
|
#include <DB/Core/Types.h>
|
||||||
#include <DB/Core/Exception.h>
|
#include <DB/Common/Exception.h>
|
||||||
#include <DB/Core/ErrorCodes.h>
|
#include <DB/Core/ErrorCodes.h>
|
||||||
#include <DB/IO/ReadHelpers.h>
|
#include <DB/IO/ReadHelpers.h>
|
||||||
#include <DB/IO/WriteHelpers.h>
|
#include <DB/IO/WriteHelpers.h>
|
||||||
|
@ -3,7 +3,7 @@
|
|||||||
#include <string>
|
#include <string>
|
||||||
#include <vector>
|
#include <vector>
|
||||||
#include <Poco/Types.h>
|
#include <Poco/Types.h>
|
||||||
#include <Yandex/strong_typedef.h>
|
#include <common/strong_typedef.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
@ -1,6 +1,6 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Yandex/Common.h> /// VisitID_t
|
#include <common/Common.h> /// VisitID_t
|
||||||
|
|
||||||
#include <DB/Core/Field.h>
|
#include <DB/Core/Field.h>
|
||||||
|
|
||||||
|
@ -55,6 +55,9 @@ protected:
|
|||||||
Block res = children.back()->read();
|
Block res = children.back()->read();
|
||||||
if (!res)
|
if (!res)
|
||||||
return res;
|
return res;
|
||||||
|
/** @todo if somehow block does not contain values for implicitly-defaulted columns that are prerequisites
|
||||||
|
* for explicitly-defaulted ones, exception will be thrown during evaluating such columns
|
||||||
|
* (implicitly-defaulted columns are evaluated on the line after following one. */
|
||||||
evaluateMissingDefaults(res, *required_columns, column_defaults, context);
|
evaluateMissingDefaults(res, *required_columns, column_defaults, context);
|
||||||
res.addDefaults(*required_columns);
|
res.addDefaults(*required_columns);
|
||||||
return res;
|
return res;
|
||||||
|
@ -37,10 +37,14 @@ public:
|
|||||||
Block res = block;
|
Block res = block;
|
||||||
|
|
||||||
/// Вычисляет явно указанные (в column_defaults) значения по-умолчанию.
|
/// Вычисляет явно указанные (в column_defaults) значения по-умолчанию.
|
||||||
|
/** @todo if somehow block does not contain values for implicitly-defaulted columns that are prerequisites
|
||||||
|
* for explicitly-defaulted ones, exception will be thrown during evaluating such columns
|
||||||
|
* (implicitly-defaulted columns are evaluated on the line after following one. */
|
||||||
evaluateMissingDefaults(res, *required_columns, column_defaults, context);
|
evaluateMissingDefaults(res, *required_columns, column_defaults, context);
|
||||||
|
|
||||||
/// Добавляет не указанные значения по-умолчанию.
|
/// Добавляет не указанные значения по-умолчанию.
|
||||||
if (!only_explicit_column_defaults)
|
if (!only_explicit_column_defaults)
|
||||||
|
/// @todo this line may be moved before `evaluateMissingDefaults` with passing {required_columns - explicitly-defaulted columns}
|
||||||
res.addDefaults(*required_columns);
|
res.addDefaults(*required_columns);
|
||||||
|
|
||||||
output->write(res);
|
output->write(res);
|
||||||
|
@ -1,6 +1,6 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Yandex/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
#include <DB/Core/Row.h>
|
#include <DB/Core/Row.h>
|
||||||
#include <DB/Core/ColumnNumbers.h>
|
#include <DB/Core/ColumnNumbers.h>
|
||||||
|
@ -1,10 +1,11 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <statdaemons/threadpool.hpp>
|
#include <common/threadpool.hpp>
|
||||||
|
|
||||||
#include <Poco/Event.h>
|
#include <Poco/Event.h>
|
||||||
|
|
||||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||||
|
#include <DB/Common/setThreadName.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -47,7 +48,7 @@ public:
|
|||||||
{
|
{
|
||||||
pool.wait();
|
pool.wait();
|
||||||
if (exception)
|
if (exception)
|
||||||
exception->rethrow();
|
std::rethrow_exception(exception);
|
||||||
children.back()->readSuffix();
|
children.back()->readSuffix();
|
||||||
started = false;
|
started = false;
|
||||||
}
|
}
|
||||||
@ -81,7 +82,7 @@ protected:
|
|||||||
bool started = false;
|
bool started = false;
|
||||||
|
|
||||||
Block block;
|
Block block;
|
||||||
ExceptionPtr exception;
|
std::exception_ptr exception;
|
||||||
|
|
||||||
|
|
||||||
Block readImpl() override
|
Block readImpl() override
|
||||||
@ -96,7 +97,7 @@ protected:
|
|||||||
pool.wait();
|
pool.wait();
|
||||||
|
|
||||||
if (exception)
|
if (exception)
|
||||||
exception->rethrow();
|
std::rethrow_exception(exception);
|
||||||
|
|
||||||
Block res = block;
|
Block res = block;
|
||||||
if (!res)
|
if (!res)
|
||||||
@ -120,6 +121,7 @@ protected:
|
|||||||
/// Вычисления, которые могут выполняться в отдельном потоке
|
/// Вычисления, которые могут выполняться в отдельном потоке
|
||||||
void calculate(MemoryTracker * memory_tracker)
|
void calculate(MemoryTracker * memory_tracker)
|
||||||
{
|
{
|
||||||
|
setThreadName("AsyncBlockInput");
|
||||||
current_memory_tracker = memory_tracker;
|
current_memory_tracker = memory_tracker;
|
||||||
|
|
||||||
try
|
try
|
||||||
@ -128,7 +130,7 @@ protected:
|
|||||||
}
|
}
|
||||||
catch (...)
|
catch (...)
|
||||||
{
|
{
|
||||||
exception = cloneCurrentException();
|
exception = std::current_exception();
|
||||||
}
|
}
|
||||||
|
|
||||||
ready.set();
|
ready.set();
|
||||||
|
@ -1,7 +1,7 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <vector>
|
#include <vector>
|
||||||
#include <statdaemons/Stopwatch.h>
|
#include <DB/Common/Stopwatch.h>
|
||||||
|
|
||||||
#include <DB/Core/Types.h>
|
#include <DB/Core/Types.h>
|
||||||
|
|
||||||
|
@ -1,5 +1,5 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
#include <Yandex/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||||
#include <DB/Core/SortDescription.h>
|
#include <DB/Core/SortDescription.h>
|
||||||
#include <DB/Columns/ColumnsNumber.h>
|
#include <DB/Columns/ColumnsNumber.h>
|
||||||
|
@ -1,6 +1,6 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Yandex/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
#include <DB/DataStreams/MergingSortedBlockInputStream.h>
|
#include <DB/DataStreams/MergingSortedBlockInputStream.h>
|
||||||
|
|
||||||
|
@ -38,6 +38,15 @@ protected:
|
|||||||
return Block();
|
return Block();
|
||||||
|
|
||||||
children.push_back(input);
|
children.push_back(input);
|
||||||
|
|
||||||
|
if (IProfilingBlockInputStream * p_input = dynamic_cast<IProfilingBlockInputStream *>(input.get()))
|
||||||
|
{
|
||||||
|
/// Они могли быть установлены раньше, но не были протащены в input.
|
||||||
|
if (progress_callback)
|
||||||
|
p_input->setProgressCallback(progress_callback);
|
||||||
|
if (process_list_elem)
|
||||||
|
p_input->setProcessListElement(process_list_elem);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
return input->read();
|
return input->read();
|
||||||
|
@ -2,7 +2,7 @@
|
|||||||
|
|
||||||
#include <DB/Columns/ColumnConst.h>
|
#include <DB/Columns/ColumnConst.h>
|
||||||
#include <DB/DataStreams/IBlockOutputStream.h>
|
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||||
#include <statdaemons/ext/range.hpp>
|
#include <ext/range.hpp>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
@ -3,7 +3,7 @@
|
|||||||
#include <queue>
|
#include <queue>
|
||||||
#include <Poco/TemporaryFile.h>
|
#include <Poco/TemporaryFile.h>
|
||||||
|
|
||||||
#include <Yandex/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
#include <DB/Core/SortDescription.h>
|
#include <DB/Core/SortDescription.h>
|
||||||
|
|
||||||
|
@ -30,224 +30,14 @@ class MergingAggregatedMemoryEfficientBlockInputStream : public IProfilingBlockI
|
|||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
MergingAggregatedMemoryEfficientBlockInputStream(BlockInputStreams inputs_, const Names & keys_names_,
|
MergingAggregatedMemoryEfficientBlockInputStream(BlockInputStreams inputs_, const Names & keys_names_,
|
||||||
const AggregateDescriptions & aggregates_, bool overflow_row_, bool final_)
|
const AggregateDescriptions & aggregates_, bool overflow_row_, bool final_);
|
||||||
: aggregator(keys_names_, aggregates_, overflow_row_, 0, OverflowMode::THROW, nullptr, 0, 0),
|
|
||||||
final(final_),
|
|
||||||
inputs(inputs_.begin(), inputs_.end())
|
|
||||||
{
|
|
||||||
children = inputs_;
|
|
||||||
}
|
|
||||||
|
|
||||||
String getName() const override { return "MergingAggregatedMemoryEfficient"; }
|
String getName() const override { return "MergingAggregatedMemoryEfficient"; }
|
||||||
|
|
||||||
String getID() const override
|
String getID() const override;
|
||||||
{
|
|
||||||
std::stringstream res;
|
|
||||||
res << "MergingAggregatedMemoryEfficient(" << aggregator.getID();
|
|
||||||
for (size_t i = 0, size = children.size(); i < size; ++i)
|
|
||||||
res << ", " << children.back()->getID();
|
|
||||||
res << ")";
|
|
||||||
return res.str();
|
|
||||||
}
|
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
Block readImpl() override
|
Block readImpl() override;
|
||||||
{
|
|
||||||
/// Если child - RemoteBlockInputStream, то отправляет запрос на все удалённые серверы, инициируя вычисления.
|
|
||||||
if (!started)
|
|
||||||
{
|
|
||||||
started = true;
|
|
||||||
for (auto & child : children)
|
|
||||||
child->readPrefix();
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Имеем несколько источников.
|
|
||||||
* Из каждого из них могут приходить следующие данные:
|
|
||||||
*
|
|
||||||
* 1. Блок, с указанным bucket_num.
|
|
||||||
* Это значит, что на удалённом сервере, данные были разрезаны по корзинам.
|
|
||||||
* И данные для одного bucket_num с разных серверов можно независимо объединять.
|
|
||||||
* При этом, даннные для разных bucket_num будут идти по возрастанию.
|
|
||||||
*
|
|
||||||
* 2. Блок без указания bucket_num.
|
|
||||||
* Это значит, что на удалённом сервере, данные не были разрезаны по корзинам.
|
|
||||||
* В случае, когда со всех серверов прийдут такие данные, их можно всех объединить.
|
|
||||||
* А если с другой части серверов прийдут данные, разрезанные по корзинам,
|
|
||||||
* то данные, не разрезанные по корзинам, нужно сначала разрезать, а потом объединять.
|
|
||||||
*
|
|
||||||
* 3. Блоки с указанием is_overflows.
|
|
||||||
* Это дополнительные данные для строк, не прошедших через max_rows_to_group_by.
|
|
||||||
* Они должны объединяться друг с другом отдельно.
|
|
||||||
*/
|
|
||||||
|
|
||||||
constexpr size_t NUM_BUCKETS = 256;
|
|
||||||
|
|
||||||
++current_bucket_num;
|
|
||||||
|
|
||||||
for (auto & input : inputs)
|
|
||||||
{
|
|
||||||
if (input.is_exhausted)
|
|
||||||
continue;
|
|
||||||
|
|
||||||
if (input.block.info.bucket_num >= current_bucket_num)
|
|
||||||
continue;
|
|
||||||
|
|
||||||
/// Если придёт блок не с основными данными, а с overflows, то запомним его и повторим чтение.
|
|
||||||
while (true)
|
|
||||||
{
|
|
||||||
// std::cerr << "reading block\n";
|
|
||||||
Block block = input.stream->read();
|
|
||||||
|
|
||||||
if (!block)
|
|
||||||
{
|
|
||||||
// std::cerr << "input is exhausted\n";
|
|
||||||
input.is_exhausted = true;
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (block.info.bucket_num != -1)
|
|
||||||
{
|
|
||||||
/// Один из разрезанных блоков для двухуровневых данных.
|
|
||||||
// std::cerr << "block for bucket " << block.info.bucket_num << "\n";
|
|
||||||
|
|
||||||
has_two_level = true;
|
|
||||||
input.block = block;
|
|
||||||
}
|
|
||||||
else if (block.info.is_overflows)
|
|
||||||
{
|
|
||||||
// std::cerr << "block for overflows\n";
|
|
||||||
|
|
||||||
has_overflows = true;
|
|
||||||
input.overflow_block = block;
|
|
||||||
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
/// Блок для неразрезанных (одноуровневых) данных.
|
|
||||||
// std::cerr << "block without bucket\n";
|
|
||||||
|
|
||||||
input.block = block;
|
|
||||||
}
|
|
||||||
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
while (true)
|
|
||||||
{
|
|
||||||
if (current_bucket_num == NUM_BUCKETS)
|
|
||||||
{
|
|
||||||
/// Обработали все основные данные. Остались, возможно, только overflows-блоки.
|
|
||||||
// std::cerr << "at end\n";
|
|
||||||
|
|
||||||
if (has_overflows)
|
|
||||||
{
|
|
||||||
// std::cerr << "merging overflows\n";
|
|
||||||
|
|
||||||
has_overflows = false;
|
|
||||||
BlocksList blocks_to_merge;
|
|
||||||
|
|
||||||
for (auto & input : inputs)
|
|
||||||
if (input.overflow_block)
|
|
||||||
blocks_to_merge.emplace_back(std::move(input.overflow_block));
|
|
||||||
|
|
||||||
return aggregator.mergeBlocks(blocks_to_merge, final);
|
|
||||||
}
|
|
||||||
else
|
|
||||||
return {};
|
|
||||||
}
|
|
||||||
else if (has_two_level)
|
|
||||||
{
|
|
||||||
/** Есть двухуровневые данные.
|
|
||||||
* Будем обрабатывать номера корзин по возрастанию.
|
|
||||||
* Найдём минимальный номер корзины, для которой есть данные,
|
|
||||||
* затем померджим эти данные.
|
|
||||||
*/
|
|
||||||
// std::cerr << "has two level\n";
|
|
||||||
|
|
||||||
int min_bucket_num = NUM_BUCKETS;
|
|
||||||
|
|
||||||
for (auto & input : inputs)
|
|
||||||
{
|
|
||||||
/// Изначально разрезанные (двухуровневые) блоки.
|
|
||||||
if (input.block.info.bucket_num != -1 && input.block.info.bucket_num < min_bucket_num)
|
|
||||||
min_bucket_num = input.block.info.bucket_num;
|
|
||||||
|
|
||||||
/// Ещё не разрезанный по корзинам блок. Разрезаем его и кладём результат в splitted_blocks.
|
|
||||||
if (input.block.info.bucket_num == -1 && input.block && input.splitted_blocks.empty())
|
|
||||||
{
|
|
||||||
LOG_TRACE(&Logger::get("MergingAggregatedMemoryEfficient"), "Having block without bucket: will split.");
|
|
||||||
|
|
||||||
input.splitted_blocks = aggregator.convertBlockToTwoLevel(input.block);
|
|
||||||
|
|
||||||
/** Нельзя уничтожать исходный блок.
|
|
||||||
* Потому что он владеет Arena с состояниями агрегатных функций,
|
|
||||||
* а splitted_blocks ей не владеют, но ссылаются на эти состояния.
|
|
||||||
*/
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Блоки, которые мы получили разрезанием одноуровневых блоков.
|
|
||||||
if (!input.splitted_blocks.empty())
|
|
||||||
{
|
|
||||||
for (const auto & block : input.splitted_blocks)
|
|
||||||
{
|
|
||||||
if (block && block.info.bucket_num < min_bucket_num)
|
|
||||||
{
|
|
||||||
min_bucket_num = block.info.bucket_num;
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
current_bucket_num = min_bucket_num;
|
|
||||||
|
|
||||||
// std::cerr << "current_bucket_num = " << current_bucket_num << "\n";
|
|
||||||
|
|
||||||
/// Блоков с основными данными больше нет.
|
|
||||||
if (current_bucket_num == NUM_BUCKETS)
|
|
||||||
continue;
|
|
||||||
|
|
||||||
/// Теперь собираем блоки для current_bucket_num, чтобы их померджить.
|
|
||||||
BlocksList blocks_to_merge;
|
|
||||||
|
|
||||||
for (auto & input : inputs)
|
|
||||||
{
|
|
||||||
if (input.block.info.bucket_num == current_bucket_num)
|
|
||||||
{
|
|
||||||
// std::cerr << "having block for current_bucket_num\n";
|
|
||||||
|
|
||||||
blocks_to_merge.emplace_back(std::move(input.block));
|
|
||||||
input.block = Block();
|
|
||||||
}
|
|
||||||
else if (!input.splitted_blocks.empty() && input.splitted_blocks[min_bucket_num])
|
|
||||||
{
|
|
||||||
// std::cerr << "having splitted data for bucket\n";
|
|
||||||
|
|
||||||
blocks_to_merge.emplace_back(std::move(input.splitted_blocks[min_bucket_num]));
|
|
||||||
input.splitted_blocks[min_bucket_num] = Block();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return aggregator.mergeBlocks(blocks_to_merge, final);
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
/// Есть только одноуровневые данные. Просто мерджим их.
|
|
||||||
// std::cerr << "don't have two level\n";
|
|
||||||
|
|
||||||
BlocksList blocks_to_merge;
|
|
||||||
|
|
||||||
for (auto & input : inputs)
|
|
||||||
if (input.block)
|
|
||||||
blocks_to_merge.emplace_back(std::move(input.block));
|
|
||||||
|
|
||||||
current_bucket_num = NUM_BUCKETS;
|
|
||||||
return aggregator.mergeBlocks(blocks_to_merge, final);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private:
|
private:
|
||||||
Aggregator aggregator;
|
Aggregator aggregator;
|
||||||
|
@ -2,7 +2,7 @@
|
|||||||
|
|
||||||
#include <queue>
|
#include <queue>
|
||||||
|
|
||||||
#include <Yandex/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
#include <DB/Core/Row.h>
|
#include <DB/Core/Row.h>
|
||||||
#include <DB/Core/SortDescription.h>
|
#include <DB/Core/SortDescription.h>
|
||||||
|
@ -163,7 +163,7 @@ private:
|
|||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
void onException(ExceptionPtr & exception, size_t thread_num)
|
void onException(std::exception_ptr & exception, size_t thread_num)
|
||||||
{
|
{
|
||||||
parent.exceptions[thread_num] = exception;
|
parent.exceptions[thread_num] = exception;
|
||||||
parent.cancel();
|
parent.cancel();
|
||||||
|
@ -6,9 +6,10 @@
|
|||||||
#include <thread>
|
#include <thread>
|
||||||
#include <mutex>
|
#include <mutex>
|
||||||
|
|
||||||
#include <Yandex/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||||
|
#include <DB/Common/setThreadName.h>
|
||||||
|
|
||||||
|
|
||||||
/** Позволяет обработать множество источников блоков параллельно, используя указанное количество потоков.
|
/** Позволяет обработать множество источников блоков параллельно, используя указанное количество потоков.
|
||||||
@ -37,7 +38,7 @@ struct ParallelInputsHandler
|
|||||||
void onFinish() {}
|
void onFinish() {}
|
||||||
|
|
||||||
/// Обработка исключения. Разумно вызывать в этом методе метод ParallelInputsProcessor::cancel, а также передавать эксепшен в основной поток.
|
/// Обработка исключения. Разумно вызывать в этом методе метод ParallelInputsProcessor::cancel, а также передавать эксепшен в основной поток.
|
||||||
void onException(ExceptionPtr & exception, size_t thread_num) {}
|
void onException(std::exception_ptr & exception, size_t thread_num) {}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
@ -139,7 +140,9 @@ private:
|
|||||||
void thread(MemoryTracker * memory_tracker, size_t thread_num)
|
void thread(MemoryTracker * memory_tracker, size_t thread_num)
|
||||||
{
|
{
|
||||||
current_memory_tracker = memory_tracker;
|
current_memory_tracker = memory_tracker;
|
||||||
ExceptionPtr exception;
|
std::exception_ptr exception;
|
||||||
|
|
||||||
|
setThreadName("ParalInputsProc");
|
||||||
|
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
@ -147,7 +150,7 @@ private:
|
|||||||
}
|
}
|
||||||
catch (...)
|
catch (...)
|
||||||
{
|
{
|
||||||
exception = cloneCurrentException();
|
exception = std::current_exception();
|
||||||
}
|
}
|
||||||
|
|
||||||
if (exception)
|
if (exception)
|
||||||
@ -168,7 +171,7 @@ private:
|
|||||||
}
|
}
|
||||||
catch (...)
|
catch (...)
|
||||||
{
|
{
|
||||||
exception = cloneCurrentException();
|
exception = std::current_exception();
|
||||||
}
|
}
|
||||||
|
|
||||||
if (exception)
|
if (exception)
|
||||||
|
@ -22,6 +22,11 @@ public:
|
|||||||
: context(context_), query_ptr(query_ptr_)
|
: context(context_), query_ptr(query_ptr_)
|
||||||
{
|
{
|
||||||
storage = context.getTable(database, table);
|
storage = context.getTable(database, table);
|
||||||
|
|
||||||
|
/** TODO Это очень важная строчка. При любой вставке в таблицу один из stream-ов должен владеть lock-ом.
|
||||||
|
* Хотя сейчас любая вставка в таблицу делается через PushingToViewsBlockOutputStream,
|
||||||
|
* но ясно, что здесь - не лучшее место для этой функциональности.
|
||||||
|
*/
|
||||||
addTableLock(storage->lockStructure(true));
|
addTableLock(storage->lockStructure(true));
|
||||||
|
|
||||||
Dependencies dependencies = context.getDependencies(database, table);
|
Dependencies dependencies = context.getDependencies(database, table);
|
||||||
@ -32,7 +37,7 @@ public:
|
|||||||
}
|
}
|
||||||
|
|
||||||
if (storage->getName() != "View")
|
if (storage->getName() != "View")
|
||||||
output = storage->write(query_ptr);
|
output = storage->write(query_ptr, context.getSettingsRef());
|
||||||
}
|
}
|
||||||
|
|
||||||
void write(const Block & block) override
|
void write(const Block & block) override
|
||||||
|
@ -1,6 +1,6 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Yandex/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||||
#include <DB/DataStreams/OneBlockInputStream.h>
|
#include <DB/DataStreams/OneBlockInputStream.h>
|
||||||
|
@ -3,9 +3,9 @@
|
|||||||
#include <DB/DataStreams/IBlockOutputStream.h>
|
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||||
|
|
||||||
#include <DB/Client/Connection.h>
|
#include <DB/Client/Connection.h>
|
||||||
#include <Yandex/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
#include <statdaemons/NetException.h>
|
#include <DB/Common/NetException.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
@ -1,13 +1,13 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Yandex/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
#include <DB/Core/Row.h>
|
#include <DB/Core/Row.h>
|
||||||
#include <DB/Core/ColumnNumbers.h>
|
#include <DB/Core/ColumnNumbers.h>
|
||||||
#include <DB/DataStreams/MergingSortedBlockInputStream.h>
|
#include <DB/DataStreams/MergingSortedBlockInputStream.h>
|
||||||
#include <DB/Storages/MergeTree/PKCondition.h>
|
#include <DB/Storages/MergeTree/PKCondition.h>
|
||||||
#include <statdaemons/ext/range.hpp>
|
#include <ext/range.hpp>
|
||||||
#include <statdaemons/ext/map.hpp>
|
#include <ext/map.hpp>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
@ -1,6 +1,6 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Yandex/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
#include <DB/Common/ConcurrentBoundedQueue.h>
|
#include <DB/Common/ConcurrentBoundedQueue.h>
|
||||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||||
@ -97,7 +97,7 @@ protected:
|
|||||||
|
|
||||||
LOG_TRACE(log, "Waiting for threads to finish");
|
LOG_TRACE(log, "Waiting for threads to finish");
|
||||||
|
|
||||||
ExceptionPtr exception;
|
std::exception_ptr exception;
|
||||||
if (!all_read)
|
if (!all_read)
|
||||||
{
|
{
|
||||||
/** Прочитаем всё до конца, чтобы ParallelInputsProcessor не заблокировался при попытке вставить в очередь.
|
/** Прочитаем всё до конца, чтобы ParallelInputsProcessor не заблокировался при попытке вставить в очередь.
|
||||||
@ -113,8 +113,8 @@ protected:
|
|||||||
{
|
{
|
||||||
if (!exception)
|
if (!exception)
|
||||||
exception = res.exception;
|
exception = res.exception;
|
||||||
else if (DB::Exception * e = dynamic_cast<DB::Exception *>(&*exception))
|
else if (Exception * e = exception_cast<Exception *>(exception))
|
||||||
e->addMessage("\n" + res.exception->displayText());
|
e->addMessage("\n" + getExceptionMessage(res.exception, false));
|
||||||
}
|
}
|
||||||
else if (!res.block)
|
else if (!res.block)
|
||||||
break;
|
break;
|
||||||
@ -128,7 +128,7 @@ protected:
|
|||||||
LOG_TRACE(log, "Waited for threads to finish");
|
LOG_TRACE(log, "Waited for threads to finish");
|
||||||
|
|
||||||
if (exception)
|
if (exception)
|
||||||
exception->rethrow();
|
std::rethrow_exception(exception);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Возможны следующие варианты:
|
/** Возможны следующие варианты:
|
||||||
@ -157,7 +157,7 @@ protected:
|
|||||||
output_queue.pop(res);
|
output_queue.pop(res);
|
||||||
|
|
||||||
if (res.exception)
|
if (res.exception)
|
||||||
res.exception->rethrow();
|
std::rethrow_exception(res.exception);
|
||||||
|
|
||||||
if (!res.block)
|
if (!res.block)
|
||||||
all_read = true;
|
all_read = true;
|
||||||
@ -183,11 +183,11 @@ private:
|
|||||||
struct OutputData
|
struct OutputData
|
||||||
{
|
{
|
||||||
Block block;
|
Block block;
|
||||||
ExceptionPtr exception;
|
std::exception_ptr exception;
|
||||||
|
|
||||||
OutputData() {}
|
OutputData() {}
|
||||||
OutputData(Block & block_) : block(block_) {}
|
OutputData(Block & block_) : block(block_) {}
|
||||||
OutputData(ExceptionPtr & exception_) : exception(exception_) {}
|
OutputData(std::exception_ptr & exception_) : exception(exception_) {}
|
||||||
};
|
};
|
||||||
|
|
||||||
/** Очередь готовых блоков. Также туда можно положить эксепшен вместо блока.
|
/** Очередь готовых блоков. Также туда можно положить эксепшен вместо блока.
|
||||||
@ -216,7 +216,7 @@ private:
|
|||||||
parent.output_queue.push(OutputData());
|
parent.output_queue.push(OutputData());
|
||||||
}
|
}
|
||||||
|
|
||||||
void onException(ExceptionPtr & exception, size_t thread_num)
|
void onException(std::exception_ptr & exception, size_t thread_num)
|
||||||
{
|
{
|
||||||
//std::cerr << "pushing exception\n";
|
//std::cerr << "pushing exception\n";
|
||||||
|
|
||||||
|
@ -27,7 +27,9 @@ public:
|
|||||||
|
|
||||||
void flush() override { ostr.next(); }
|
void flush() override { ostr.next(); }
|
||||||
|
|
||||||
private:
|
protected:
|
||||||
|
virtual void writeValue(const Field & field) const;
|
||||||
|
|
||||||
WriteBuffer & ostr;
|
WriteBuffer & ostr;
|
||||||
const Block sample;
|
const Block sample;
|
||||||
DataTypes data_types;
|
DataTypes data_types;
|
||||||
@ -39,5 +41,18 @@ private:
|
|||||||
Pads_t pads;
|
Pads_t pads;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
/** То же самое, но строки выводятся без экранирования.
|
||||||
|
*/
|
||||||
|
class VerticalRawRowOutputStream : public VerticalRowOutputStream
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
VerticalRawRowOutputStream(WriteBuffer & ostr_, const Block & sample_)
|
||||||
|
: VerticalRowOutputStream(ostr_, sample_) {}
|
||||||
|
|
||||||
|
protected:
|
||||||
|
void writeValue(const Field & field) const override;
|
||||||
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -28,7 +28,7 @@ public:
|
|||||||
|
|
||||||
std::string getFunctionName() const { return function->getName(); }
|
std::string getFunctionName() const { return function->getName(); }
|
||||||
|
|
||||||
std::string getName() const
|
std::string getName() const override
|
||||||
{
|
{
|
||||||
std::stringstream stream;
|
std::stringstream stream;
|
||||||
stream << "AggregateFunction(" << function->getName();
|
stream << "AggregateFunction(" << function->getName();
|
||||||
@ -55,24 +55,24 @@ public:
|
|||||||
DataTypePtr getReturnType() const { return function->getReturnType(); };
|
DataTypePtr getReturnType() const { return function->getReturnType(); };
|
||||||
DataTypes getArgumentsDataTypes() const { return argument_types; }
|
DataTypes getArgumentsDataTypes() const { return argument_types; }
|
||||||
|
|
||||||
DataTypePtr clone() const { return new DataTypeAggregateFunction(function, argument_types, parameters); }
|
DataTypePtr clone() const override { return new DataTypeAggregateFunction(function, argument_types, parameters); }
|
||||||
|
|
||||||
void serializeBinary(const Field & field, WriteBuffer & ostr) const;
|
void serializeBinary(const Field & field, WriteBuffer & ostr) const override;
|
||||||
void deserializeBinary(Field & field, ReadBuffer & istr) const;
|
void deserializeBinary(Field & field, ReadBuffer & istr) const override;
|
||||||
void serializeBinary(const IColumn & column, WriteBuffer & ostr, size_t offset = 0, size_t limit = 0) const;
|
void serializeBinary(const IColumn & column, WriteBuffer & ostr, size_t offset = 0, size_t limit = 0) const override;
|
||||||
void deserializeBinary(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const;
|
void deserializeBinary(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override;
|
||||||
void serializeText(const Field & field, WriteBuffer & ostr) const;
|
void serializeText(const Field & field, WriteBuffer & ostr) const override;
|
||||||
void deserializeText(Field & field, ReadBuffer & istr) const;
|
void deserializeText(Field & field, ReadBuffer & istr) const override;
|
||||||
void serializeTextEscaped(const Field & field, WriteBuffer & ostr) const;
|
void serializeTextEscaped(const Field & field, WriteBuffer & ostr) const override;
|
||||||
void deserializeTextEscaped(Field & field, ReadBuffer & istr) const;
|
void deserializeTextEscaped(Field & field, ReadBuffer & istr) const override;
|
||||||
void serializeTextQuoted(const Field & field, WriteBuffer & ostr) const;
|
void serializeTextQuoted(const Field & field, WriteBuffer & ostr) const override;
|
||||||
void deserializeTextQuoted(Field & field, ReadBuffer & istr) const;
|
void deserializeTextQuoted(Field & field, ReadBuffer & istr) const override;
|
||||||
void serializeTextJSON(const Field & field, WriteBuffer & ostr) const;
|
void serializeTextJSON(const Field & field, WriteBuffer & ostr) const override;
|
||||||
|
|
||||||
ColumnPtr createColumn() const;
|
ColumnPtr createColumn() const override;
|
||||||
ColumnPtr createConstColumn(size_t size, const Field & field) const;
|
ColumnPtr createConstColumn(size_t size, const Field & field) const override;
|
||||||
|
|
||||||
Field getDefault() const
|
Field getDefault() const override
|
||||||
{
|
{
|
||||||
throw Exception("There is no default value for AggregateFunction data type", ErrorCodes::THERE_IS_NO_DEFAULT_VALUE);
|
throw Exception("There is no default value for AggregateFunction data type", ErrorCodes::THERE_IS_NO_DEFAULT_VALUE);
|
||||||
}
|
}
|
||||||
|
@ -20,29 +20,29 @@ private:
|
|||||||
public:
|
public:
|
||||||
DataTypeArray(DataTypePtr nested_);
|
DataTypeArray(DataTypePtr nested_);
|
||||||
|
|
||||||
std::string getName() const
|
std::string getName() const override
|
||||||
{
|
{
|
||||||
return "Array(" + nested->getName() + ")";
|
return "Array(" + nested->getName() + ")";
|
||||||
}
|
}
|
||||||
|
|
||||||
DataTypePtr clone() const
|
DataTypePtr clone() const override
|
||||||
{
|
{
|
||||||
return new DataTypeArray(nested);
|
return new DataTypeArray(nested);
|
||||||
}
|
}
|
||||||
|
|
||||||
void serializeBinary(const Field & field, WriteBuffer & ostr) const;
|
void serializeBinary(const Field & field, WriteBuffer & ostr) const override;
|
||||||
void deserializeBinary(Field & field, ReadBuffer & istr) const;
|
void deserializeBinary(Field & field, ReadBuffer & istr) const override;
|
||||||
|
|
||||||
void serializeText(const Field & field, WriteBuffer & ostr) const;
|
void serializeText(const Field & field, WriteBuffer & ostr) const override;
|
||||||
void deserializeText(Field & field, ReadBuffer & istr) const;
|
void deserializeText(Field & field, ReadBuffer & istr) const override;
|
||||||
|
|
||||||
void serializeTextEscaped(const Field & field, WriteBuffer & ostr) const;
|
void serializeTextEscaped(const Field & field, WriteBuffer & ostr) const override;
|
||||||
void deserializeTextEscaped(Field & field, ReadBuffer & istr) const;
|
void deserializeTextEscaped(Field & field, ReadBuffer & istr) const override;
|
||||||
|
|
||||||
void serializeTextQuoted(const Field & field, WriteBuffer & ostr) const;
|
void serializeTextQuoted(const Field & field, WriteBuffer & ostr) const override;
|
||||||
void deserializeTextQuoted(Field & field, ReadBuffer & istr) const;
|
void deserializeTextQuoted(Field & field, ReadBuffer & istr) const override;
|
||||||
|
|
||||||
void serializeTextJSON(const Field & field, WriteBuffer & ostr) const;
|
void serializeTextJSON(const Field & field, WriteBuffer & ostr) const override;
|
||||||
|
|
||||||
/** Потоковая сериализация массивов устроена по-особенному:
|
/** Потоковая сериализация массивов устроена по-особенному:
|
||||||
* - записываются/читаются элементы, уложенные подряд, без размеров массивов;
|
* - записываются/читаются элементы, уложенные подряд, без размеров массивов;
|
||||||
@ -52,12 +52,12 @@ public:
|
|||||||
*/
|
*/
|
||||||
|
|
||||||
/** Записать только значения, без размеров. Вызывающая сторона также должна куда-нибудь записать смещения. */
|
/** Записать только значения, без размеров. Вызывающая сторона также должна куда-нибудь записать смещения. */
|
||||||
void serializeBinary(const IColumn & column, WriteBuffer & ostr, size_t offset = 0, size_t limit = 0) const;
|
void serializeBinary(const IColumn & column, WriteBuffer & ostr, size_t offset = 0, size_t limit = 0) const override;
|
||||||
|
|
||||||
/** Прочитать только значения, без размеров.
|
/** Прочитать только значения, без размеров.
|
||||||
* При этом, в column уже заранее должны быть считаны все размеры.
|
* При этом, в column уже заранее должны быть считаны все размеры.
|
||||||
*/
|
*/
|
||||||
void deserializeBinary(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const;
|
void deserializeBinary(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override;
|
||||||
|
|
||||||
/** Записать размеры. */
|
/** Записать размеры. */
|
||||||
void serializeOffsets(const IColumn & column, WriteBuffer & ostr, size_t offset = 0, size_t limit = 0) const;
|
void serializeOffsets(const IColumn & column, WriteBuffer & ostr, size_t offset = 0, size_t limit = 0) const;
|
||||||
@ -65,10 +65,10 @@ public:
|
|||||||
/** Прочитать размеры. Вызывайте этот метод перед чтением значений. */
|
/** Прочитать размеры. Вызывайте этот метод перед чтением значений. */
|
||||||
void deserializeOffsets(IColumn & column, ReadBuffer & istr, size_t limit) const;
|
void deserializeOffsets(IColumn & column, ReadBuffer & istr, size_t limit) const;
|
||||||
|
|
||||||
ColumnPtr createColumn() const;
|
ColumnPtr createColumn() const override;
|
||||||
ColumnPtr createConstColumn(size_t size, const Field & field) const;
|
ColumnPtr createConstColumn(size_t size, const Field & field) const override;
|
||||||
|
|
||||||
Field getDefault() const
|
Field getDefault() const override
|
||||||
{
|
{
|
||||||
return Array();
|
return Array();
|
||||||
}
|
}
|
||||||
|
@ -14,49 +14,49 @@ class DataTypeDate final : public IDataTypeNumberFixed<UInt16, ColumnUInt16>
|
|||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
DataTypeDate() {}
|
DataTypeDate() {}
|
||||||
|
|
||||||
bool behavesAsNumber() const { return false; }
|
|
||||||
|
|
||||||
std::string getName() const { return "Date"; }
|
bool behavesAsNumber() const override { return false; }
|
||||||
DataTypePtr clone() const { return new DataTypeDate; }
|
|
||||||
|
|
||||||
void serializeText(const Field & field, WriteBuffer & ostr) const
|
std::string getName() const override { return "Date"; }
|
||||||
|
DataTypePtr clone() const override { return new DataTypeDate; }
|
||||||
|
|
||||||
|
void serializeText(const Field & field, WriteBuffer & ostr) const override
|
||||||
{
|
{
|
||||||
writeDateText(DayNum_t(get<UInt64>(field)), ostr);
|
writeDateText(DayNum_t(get<UInt64>(field)), ostr);
|
||||||
}
|
}
|
||||||
|
|
||||||
void deserializeText(Field & field, ReadBuffer & istr) const
|
void deserializeText(Field & field, ReadBuffer & istr) const override
|
||||||
{
|
{
|
||||||
DayNum_t x;
|
DayNum_t x;
|
||||||
readDateText(x, istr);
|
readDateText(x, istr);
|
||||||
field = static_cast<UInt64>(x);
|
field = static_cast<UInt64>(x);
|
||||||
}
|
}
|
||||||
|
|
||||||
void serializeTextEscaped(const Field & field, WriteBuffer & ostr) const
|
void serializeTextEscaped(const Field & field, WriteBuffer & ostr) const override
|
||||||
{
|
{
|
||||||
serializeText(field, ostr);
|
serializeText(field, ostr);
|
||||||
}
|
}
|
||||||
|
|
||||||
void deserializeTextEscaped(Field & field, ReadBuffer & istr) const
|
void deserializeTextEscaped(Field & field, ReadBuffer & istr) const override
|
||||||
{
|
{
|
||||||
deserializeText(field, istr);
|
deserializeText(field, istr);
|
||||||
}
|
}
|
||||||
|
|
||||||
void serializeTextQuoted(const Field & field, WriteBuffer & ostr) const
|
void serializeTextQuoted(const Field & field, WriteBuffer & ostr) const override
|
||||||
{
|
{
|
||||||
writeChar('\'', ostr);
|
writeChar('\'', ostr);
|
||||||
serializeText(field, ostr);
|
serializeText(field, ostr);
|
||||||
writeChar('\'', ostr);
|
writeChar('\'', ostr);
|
||||||
}
|
}
|
||||||
|
|
||||||
void deserializeTextQuoted(Field & field, ReadBuffer & istr) const
|
void deserializeTextQuoted(Field & field, ReadBuffer & istr) const override
|
||||||
{
|
{
|
||||||
assertString("'", istr);
|
assertString("'", istr);
|
||||||
deserializeText(field, istr);
|
deserializeText(field, istr);
|
||||||
assertString("'", istr);
|
assertString("'", istr);
|
||||||
}
|
}
|
||||||
|
|
||||||
void serializeTextJSON(const Field & field, WriteBuffer & ostr) const
|
void serializeTextJSON(const Field & field, WriteBuffer & ostr) const override
|
||||||
{
|
{
|
||||||
writeChar('"', ostr);
|
writeChar('"', ostr);
|
||||||
serializeText(field, ostr);
|
serializeText(field, ostr);
|
||||||
|
@ -14,49 +14,49 @@ class DataTypeDateTime final : public IDataTypeNumberFixed<UInt32, ColumnUInt32>
|
|||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
DataTypeDateTime() {}
|
DataTypeDateTime() {}
|
||||||
|
|
||||||
bool behavesAsNumber() const { return false; }
|
|
||||||
|
|
||||||
std::string getName() const { return "DateTime"; }
|
bool behavesAsNumber() const override { return false; }
|
||||||
DataTypePtr clone() const { return new DataTypeDateTime; }
|
|
||||||
|
|
||||||
void serializeText(const Field & field, WriteBuffer & ostr) const
|
std::string getName() const override { return "DateTime"; }
|
||||||
|
DataTypePtr clone() const override { return new DataTypeDateTime; }
|
||||||
|
|
||||||
|
void serializeText(const Field & field, WriteBuffer & ostr) const override
|
||||||
{
|
{
|
||||||
writeDateTimeText(get<UInt64>(field), ostr);
|
writeDateTimeText(get<UInt64>(field), ostr);
|
||||||
}
|
}
|
||||||
|
|
||||||
void deserializeText(Field & field, ReadBuffer & istr) const
|
void deserializeText(Field & field, ReadBuffer & istr) const override
|
||||||
{
|
{
|
||||||
time_t x;
|
time_t x;
|
||||||
readDateTimeText(x, istr);
|
readDateTimeText(x, istr);
|
||||||
field = NearestFieldType<UInt32>::Type(x);
|
field = NearestFieldType<UInt32>::Type(x);
|
||||||
}
|
}
|
||||||
|
|
||||||
void serializeTextEscaped(const Field & field, WriteBuffer & ostr) const
|
void serializeTextEscaped(const Field & field, WriteBuffer & ostr) const override
|
||||||
{
|
{
|
||||||
serializeText(field, ostr);
|
serializeText(field, ostr);
|
||||||
}
|
}
|
||||||
|
|
||||||
void deserializeTextEscaped(Field & field, ReadBuffer & istr) const
|
void deserializeTextEscaped(Field & field, ReadBuffer & istr) const override
|
||||||
{
|
{
|
||||||
deserializeText(field, istr);
|
deserializeText(field, istr);
|
||||||
}
|
}
|
||||||
|
|
||||||
void serializeTextQuoted(const Field & field, WriteBuffer & ostr) const
|
void serializeTextQuoted(const Field & field, WriteBuffer & ostr) const override
|
||||||
{
|
{
|
||||||
writeChar('\'', ostr);
|
writeChar('\'', ostr);
|
||||||
serializeText(field, ostr);
|
serializeText(field, ostr);
|
||||||
writeChar('\'', ostr);
|
writeChar('\'', ostr);
|
||||||
}
|
}
|
||||||
|
|
||||||
void serializeTextJSON(const Field & field, WriteBuffer & ostr) const
|
void serializeTextJSON(const Field & field, WriteBuffer & ostr) const override
|
||||||
{
|
{
|
||||||
writeChar('"', ostr);
|
writeChar('"', ostr);
|
||||||
serializeText(field, ostr);
|
serializeText(field, ostr);
|
||||||
writeChar('"', ostr);
|
writeChar('"', ostr);
|
||||||
}
|
}
|
||||||
|
|
||||||
void deserializeTextQuoted(Field & field, ReadBuffer & istr) const
|
void deserializeTextQuoted(Field & field, ReadBuffer & istr) const override
|
||||||
{
|
{
|
||||||
assertString("'", istr);
|
assertString("'", istr);
|
||||||
deserializeText(field, istr);
|
deserializeText(field, istr);
|
||||||
|
@ -5,7 +5,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Лямбда-выражение.
|
* Лямбда-выражение.
|
||||||
*/
|
*/
|
||||||
@ -14,13 +14,13 @@ class DataTypeExpression final : public IDataTypeDummy
|
|||||||
private:
|
private:
|
||||||
DataTypes argument_types;
|
DataTypes argument_types;
|
||||||
DataTypePtr return_type;
|
DataTypePtr return_type;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
/// Некоторые типы могут быть еще не известны.
|
/// Некоторые типы могут быть еще не известны.
|
||||||
DataTypeExpression(DataTypes argument_types_ = DataTypes(), DataTypePtr return_type_ = nullptr)
|
DataTypeExpression(DataTypes argument_types_ = DataTypes(), DataTypePtr return_type_ = nullptr)
|
||||||
: argument_types(argument_types_), return_type(return_type_) {}
|
: argument_types(argument_types_), return_type(return_type_) {}
|
||||||
|
|
||||||
std::string getName() const
|
std::string getName() const override
|
||||||
{
|
{
|
||||||
std::string res = "Expression(";
|
std::string res = "Expression(";
|
||||||
if (argument_types.size() > 1)
|
if (argument_types.size() > 1)
|
||||||
@ -39,21 +39,21 @@ public:
|
|||||||
res += ")";
|
res += ")";
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
DataTypePtr clone() const
|
DataTypePtr clone() const override
|
||||||
{
|
{
|
||||||
return new DataTypeExpression(argument_types, return_type);
|
return new DataTypeExpression(argument_types, return_type);
|
||||||
}
|
}
|
||||||
|
|
||||||
const DataTypes & getArgumentTypes() const
|
const DataTypes & getArgumentTypes() const
|
||||||
{
|
{
|
||||||
return argument_types;
|
return argument_types;
|
||||||
}
|
}
|
||||||
|
|
||||||
const DataTypePtr & getReturnType() const
|
const DataTypePtr & getReturnType() const
|
||||||
{
|
{
|
||||||
return return_type;
|
return return_type;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -2,7 +2,7 @@
|
|||||||
|
|
||||||
#include <map>
|
#include <map>
|
||||||
|
|
||||||
#include <Yandex/singleton.h>
|
#include <common/singleton.h>
|
||||||
|
|
||||||
#include <Poco/RegularExpression.h>
|
#include <Poco/RegularExpression.h>
|
||||||
|
|
||||||
|
@ -25,12 +25,12 @@ public:
|
|||||||
throw Exception("FixedString size must be positive", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
throw Exception("FixedString size must be positive", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||||
}
|
}
|
||||||
|
|
||||||
std::string getName() const
|
std::string getName() const override
|
||||||
{
|
{
|
||||||
return "FixedString(" + toString(n) + ")";
|
return "FixedString(" + toString(n) + ")";
|
||||||
}
|
}
|
||||||
|
|
||||||
DataTypePtr clone() const
|
DataTypePtr clone() const override
|
||||||
{
|
{
|
||||||
return new DataTypeFixedString(n);
|
return new DataTypeFixedString(n);
|
||||||
}
|
}
|
||||||
@ -40,28 +40,28 @@ public:
|
|||||||
return n;
|
return n;
|
||||||
}
|
}
|
||||||
|
|
||||||
void serializeBinary(const Field & field, WriteBuffer & ostr) const;
|
void serializeBinary(const Field & field, WriteBuffer & ostr) const override;
|
||||||
void deserializeBinary(Field & field, ReadBuffer & istr) const;
|
void deserializeBinary(Field & field, ReadBuffer & istr) const override;
|
||||||
void serializeBinary(const IColumn & column, WriteBuffer & ostr, size_t offset = 0, size_t limit = 0) const;
|
void serializeBinary(const IColumn & column, WriteBuffer & ostr, size_t offset = 0, size_t limit = 0) const override;
|
||||||
void deserializeBinary(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const;
|
void deserializeBinary(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override;
|
||||||
|
|
||||||
void serializeText(const Field & field, WriteBuffer & ostr) const;
|
void serializeText(const Field & field, WriteBuffer & ostr) const override;
|
||||||
void deserializeText(Field & field, ReadBuffer & istr) const;
|
void deserializeText(Field & field, ReadBuffer & istr) const override;
|
||||||
|
|
||||||
void serializeTextEscaped(const Field & field, WriteBuffer & ostr) const;
|
void serializeTextEscaped(const Field & field, WriteBuffer & ostr) const override;
|
||||||
void deserializeTextEscaped(Field & field, ReadBuffer & istr) const;
|
void deserializeTextEscaped(Field & field, ReadBuffer & istr) const override;
|
||||||
|
|
||||||
void serializeTextQuoted(const Field & field, WriteBuffer & ostr) const;
|
void serializeTextQuoted(const Field & field, WriteBuffer & ostr) const override;
|
||||||
void deserializeTextQuoted(Field & field, ReadBuffer & istr) const;
|
void deserializeTextQuoted(Field & field, ReadBuffer & istr) const override;
|
||||||
|
|
||||||
void serializeTextJSON(const Field & field, WriteBuffer & ostr) const;
|
void serializeTextJSON(const Field & field, WriteBuffer & ostr) const override;
|
||||||
|
|
||||||
ColumnPtr createColumn() const;
|
ColumnPtr createColumn() const override;
|
||||||
ColumnPtr createConstColumn(size_t size, const Field & field) const;
|
ColumnPtr createConstColumn(size_t size, const Field & field) const override;
|
||||||
|
|
||||||
Field getDefault() const
|
Field getDefault() const override
|
||||||
{
|
{
|
||||||
return String("");
|
return String();
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -1,26 +1,32 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <DB/DataTypes/IDataType.h>
|
#include <DB/DataTypes/IDataTypeDummy.h>
|
||||||
#include <DB/Core/NamesAndTypes.h>
|
#include <DB/Core/NamesAndTypes.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
using Poco::SharedPtr;
|
/** Хранит набор пар (имя, тип) для вложенной структуры данных.
|
||||||
|
* Используется только при создании таблицы. Во всех остальных случаях не используется, так как раскрывается в набор отдельных столбцов с типами.
|
||||||
|
*/
|
||||||
class DataTypeNested final : public IDataType
|
class DataTypeNested final : public IDataTypeDummy
|
||||||
{
|
{
|
||||||
private:
|
private:
|
||||||
/// Имена и типы вложенных массивов.
|
/// Имена и типы вложенных массивов.
|
||||||
NamesAndTypesListPtr nested;
|
NamesAndTypesListPtr nested;
|
||||||
/// Тип смещений.
|
|
||||||
DataTypePtr offsets;
|
|
||||||
|
|
||||||
public:
|
public:
|
||||||
DataTypeNested(NamesAndTypesListPtr nested_);
|
DataTypeNested(NamesAndTypesListPtr nested_);
|
||||||
|
|
||||||
std::string getName() const;
|
std::string getName() const override;
|
||||||
|
|
||||||
|
DataTypePtr clone() const override
|
||||||
|
{
|
||||||
|
return new DataTypeNested(nested);
|
||||||
|
}
|
||||||
|
|
||||||
|
const NamesAndTypesListPtr & getNestedTypesList() const { return nested; }
|
||||||
|
|
||||||
static std::string concatenateNestedName(const std::string & nested_table_name, const std::string & nested_field_name);
|
static std::string concatenateNestedName(const std::string & nested_table_name, const std::string & nested_field_name);
|
||||||
/// Возвращает префикс имени до первой точки '.'. Или имя без изменений, если точки нет.
|
/// Возвращает префикс имени до первой точки '.'. Или имя без изменений, если точки нет.
|
||||||
@ -28,57 +34,6 @@ public:
|
|||||||
/// Возвращает суффикс имени после первой точки справа '.'. Или имя без изменений, если точки нет.
|
/// Возвращает суффикс имени после первой точки справа '.'. Или имя без изменений, если точки нет.
|
||||||
static std::string extractNestedColumnName(const std::string & nested_name);
|
static std::string extractNestedColumnName(const std::string & nested_name);
|
||||||
|
|
||||||
DataTypePtr clone() const
|
|
||||||
{
|
|
||||||
return new DataTypeNested(nested);
|
|
||||||
}
|
|
||||||
|
|
||||||
void serializeBinary(const Field & field, WriteBuffer & ostr) const;
|
|
||||||
void deserializeBinary(Field & field, ReadBuffer & istr) const;
|
|
||||||
|
|
||||||
void serializeText(const Field & field, WriteBuffer & ostr) const;
|
|
||||||
void deserializeText(Field & field, ReadBuffer & istr) const;
|
|
||||||
|
|
||||||
void serializeTextEscaped(const Field & field, WriteBuffer & ostr) const;
|
|
||||||
void deserializeTextEscaped(Field & field, ReadBuffer & istr) const;
|
|
||||||
|
|
||||||
void serializeTextQuoted(const Field & field, WriteBuffer & ostr) const;
|
|
||||||
void deserializeTextQuoted(Field & field, ReadBuffer & istr) const;
|
|
||||||
|
|
||||||
void serializeTextJSON(const Field & field, WriteBuffer & ostr) const;
|
|
||||||
|
|
||||||
/** Потоковая сериализация массивов устроена по-особенному:
|
|
||||||
* - записываются/читаются элементы, уложенные подряд, без размеров массивов;
|
|
||||||
* - размеры записываются/читаются в отдельный столбец,
|
|
||||||
* и о записи/чтении размеров должна позаботиться вызывающая сторона.
|
|
||||||
* Это нужно, так как несколько массивов имеют общие размеры.
|
|
||||||
*/
|
|
||||||
|
|
||||||
/** Записать только значения, без размеров. Вызывающая сторона также должна куда-нибудь записать смещения. */
|
|
||||||
void serializeBinary(const IColumn & column, WriteBuffer & ostr, size_t offset = 0, size_t limit = 0) const;
|
|
||||||
|
|
||||||
/** Прочитать только значения, без размеров.
|
|
||||||
* При этом, в column уже заранее должны быть считаны все размеры.
|
|
||||||
*/
|
|
||||||
void deserializeBinary(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const;
|
|
||||||
|
|
||||||
/** Записать размеры. */
|
|
||||||
void serializeOffsets(const IColumn & column, WriteBuffer & ostr, size_t offset = 0, size_t limit = 0) const;
|
|
||||||
|
|
||||||
/** Прочитать размеры. Вызывайте этот метод перед чтением значений. */
|
|
||||||
void deserializeOffsets(IColumn & column, ReadBuffer & istr, size_t limit) const;
|
|
||||||
|
|
||||||
ColumnPtr createColumn() const;
|
|
||||||
ColumnPtr createConstColumn(size_t size, const Field & field) const;
|
|
||||||
|
|
||||||
Field getDefault() const
|
|
||||||
{
|
|
||||||
throw Exception("Method getDefault is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
|
||||||
}
|
|
||||||
|
|
||||||
const NamesAndTypesListPtr & getNestedTypesList() const { return nested; }
|
|
||||||
const DataTypePtr & getOffsetsType() const { return offsets; }
|
|
||||||
|
|
||||||
/// Создает новый список в котором колонки типа Nested заменены на несколько вида имя_колонки.имя_вложенной_ячейки
|
/// Создает новый список в котором колонки типа Nested заменены на несколько вида имя_колонки.имя_вложенной_ячейки
|
||||||
static NamesAndTypesListPtr expandNestedColumns(const NamesAndTypesList & names_and_types);
|
static NamesAndTypesListPtr expandNestedColumns(const NamesAndTypesList & names_and_types);
|
||||||
};
|
};
|
||||||
|
@ -12,8 +12,8 @@ namespace DB
|
|||||||
class DataTypeSet final : public IDataTypeDummy
|
class DataTypeSet final : public IDataTypeDummy
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
std::string getName() const { return "Set"; }
|
std::string getName() const override { return "Set"; }
|
||||||
SharedPtr<IDataType> clone() const { return new DataTypeSet(); }
|
SharedPtr<IDataType> clone() const override { return new DataTypeSet(); }
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -18,36 +18,36 @@ class DataTypeString final : public IDataType
|
|||||||
public:
|
public:
|
||||||
using FieldType = String;
|
using FieldType = String;
|
||||||
|
|
||||||
std::string getName() const
|
std::string getName() const override
|
||||||
{
|
{
|
||||||
return "String";
|
return "String";
|
||||||
}
|
}
|
||||||
|
|
||||||
DataTypePtr clone() const
|
DataTypePtr clone() const override
|
||||||
{
|
{
|
||||||
return new DataTypeString;
|
return new DataTypeString;
|
||||||
}
|
}
|
||||||
|
|
||||||
void serializeBinary(const Field & field, WriteBuffer & ostr) const;
|
void serializeBinary(const Field & field, WriteBuffer & ostr) const override;
|
||||||
void deserializeBinary(Field & field, ReadBuffer & istr) const;
|
void deserializeBinary(Field & field, ReadBuffer & istr) const override;
|
||||||
void serializeBinary(const IColumn & column, WriteBuffer & ostr, size_t offset = 0, size_t limit = 0) const;
|
void serializeBinary(const IColumn & column, WriteBuffer & ostr, size_t offset = 0, size_t limit = 0) const override;
|
||||||
void deserializeBinary(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const;
|
void deserializeBinary(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override;
|
||||||
|
|
||||||
void serializeText(const Field & field, WriteBuffer & ostr) const;
|
void serializeText(const Field & field, WriteBuffer & ostr) const override;
|
||||||
void deserializeText(Field & field, ReadBuffer & istr) const;
|
void deserializeText(Field & field, ReadBuffer & istr) const override;
|
||||||
|
|
||||||
void serializeTextEscaped(const Field & field, WriteBuffer & ostr) const;
|
void serializeTextEscaped(const Field & field, WriteBuffer & ostr) const override;
|
||||||
void deserializeTextEscaped(Field & field, ReadBuffer & istr) const;
|
void deserializeTextEscaped(Field & field, ReadBuffer & istr) const override;
|
||||||
|
|
||||||
void serializeTextQuoted(const Field & field, WriteBuffer & ostr) const;
|
void serializeTextQuoted(const Field & field, WriteBuffer & ostr) const override;
|
||||||
void deserializeTextQuoted(Field & field, ReadBuffer & istr) const;
|
void deserializeTextQuoted(Field & field, ReadBuffer & istr) const override;
|
||||||
|
|
||||||
void serializeTextJSON(const Field & field, WriteBuffer & ostr) const;
|
void serializeTextJSON(const Field & field, WriteBuffer & ostr) const override;
|
||||||
|
|
||||||
ColumnPtr createColumn() const;
|
ColumnPtr createColumn() const override;
|
||||||
ColumnPtr createConstColumn(size_t size, const Field & field) const;
|
ColumnPtr createConstColumn(size_t size, const Field & field) const override;
|
||||||
|
|
||||||
Field getDefault() const
|
Field getDefault() const override
|
||||||
{
|
{
|
||||||
return String("");
|
return String("");
|
||||||
}
|
}
|
||||||
|
@ -22,7 +22,7 @@ private:
|
|||||||
public:
|
public:
|
||||||
DataTypeTuple(DataTypes elems_) : elems(elems_) {}
|
DataTypeTuple(DataTypes elems_) : elems(elems_) {}
|
||||||
|
|
||||||
std::string getName() const
|
std::string getName() const override
|
||||||
{
|
{
|
||||||
std::stringstream s;
|
std::stringstream s;
|
||||||
|
|
||||||
@ -34,16 +34,16 @@ public:
|
|||||||
return s.str();
|
return s.str();
|
||||||
}
|
}
|
||||||
|
|
||||||
SharedPtr<IDataType> clone() const { return new DataTypeTuple(elems); }
|
SharedPtr<IDataType> clone() const override { return new DataTypeTuple(elems); }
|
||||||
|
|
||||||
void serializeBinary(const Field & field, WriteBuffer & ostr) const
|
void serializeBinary(const Field & field, WriteBuffer & ostr) const override
|
||||||
{
|
{
|
||||||
const Array & arr = get<const Array &>(field);
|
const Array & arr = get<const Array &>(field);
|
||||||
for (size_t i = 0, size = elems.size(); i < size; ++i)
|
for (size_t i = 0, size = elems.size(); i < size; ++i)
|
||||||
elems[i]->serializeBinary(arr[i], ostr);
|
elems[i]->serializeBinary(arr[i], ostr);
|
||||||
}
|
}
|
||||||
|
|
||||||
void deserializeBinary(Field & field, ReadBuffer & istr) const
|
void deserializeBinary(Field & field, ReadBuffer & istr) const override
|
||||||
{
|
{
|
||||||
size_t size = elems.size();
|
size_t size = elems.size();
|
||||||
field = Array(size);
|
field = Array(size);
|
||||||
@ -52,7 +52,7 @@ public:
|
|||||||
elems[i]->deserializeBinary(arr[i], istr);
|
elems[i]->deserializeBinary(arr[i], istr);
|
||||||
}
|
}
|
||||||
|
|
||||||
void serializeText(const Field & field, WriteBuffer & ostr) const
|
void serializeText(const Field & field, WriteBuffer & ostr) const override
|
||||||
{
|
{
|
||||||
const Array & arr = get<const Array &>(field);
|
const Array & arr = get<const Array &>(field);
|
||||||
writeChar('(', ostr);
|
writeChar('(', ostr);
|
||||||
@ -65,7 +65,7 @@ public:
|
|||||||
writeChar(')', ostr);
|
writeChar(')', ostr);
|
||||||
}
|
}
|
||||||
|
|
||||||
void deserializeText(Field & field, ReadBuffer & istr) const
|
void deserializeText(Field & field, ReadBuffer & istr) const override
|
||||||
{
|
{
|
||||||
size_t size = elems.size();
|
size_t size = elems.size();
|
||||||
field = Array(size);
|
field = Array(size);
|
||||||
@ -80,27 +80,27 @@ public:
|
|||||||
assertString(")", istr);
|
assertString(")", istr);
|
||||||
}
|
}
|
||||||
|
|
||||||
void serializeTextEscaped(const Field & field, WriteBuffer & ostr) const
|
void serializeTextEscaped(const Field & field, WriteBuffer & ostr) const override
|
||||||
{
|
{
|
||||||
serializeText(field, ostr);
|
serializeText(field, ostr);
|
||||||
}
|
}
|
||||||
|
|
||||||
void deserializeTextEscaped(Field & field, ReadBuffer & istr) const
|
void deserializeTextEscaped(Field & field, ReadBuffer & istr) const override
|
||||||
{
|
{
|
||||||
deserializeText(field, istr);
|
deserializeText(field, istr);
|
||||||
}
|
}
|
||||||
|
|
||||||
void serializeTextQuoted(const Field & field, WriteBuffer & ostr) const
|
void serializeTextQuoted(const Field & field, WriteBuffer & ostr) const override
|
||||||
{
|
{
|
||||||
serializeText(field, ostr);
|
serializeText(field, ostr);
|
||||||
}
|
}
|
||||||
|
|
||||||
void deserializeTextQuoted(Field & field, ReadBuffer & istr) const
|
void deserializeTextQuoted(Field & field, ReadBuffer & istr) const override
|
||||||
{
|
{
|
||||||
deserializeText(field, istr);
|
deserializeText(field, istr);
|
||||||
}
|
}
|
||||||
|
|
||||||
void serializeTextJSON(const Field & field, WriteBuffer & ostr) const
|
void serializeTextJSON(const Field & field, WriteBuffer & ostr) const override
|
||||||
{
|
{
|
||||||
const Array & arr = get<const Array &>(field);
|
const Array & arr = get<const Array &>(field);
|
||||||
writeChar('[', ostr);
|
writeChar('[', ostr);
|
||||||
@ -113,7 +113,7 @@ public:
|
|||||||
writeChar(']', ostr);
|
writeChar(']', ostr);
|
||||||
}
|
}
|
||||||
|
|
||||||
void serializeBinary(const IColumn & column, WriteBuffer & ostr, size_t offset = 0, size_t limit = 0) const
|
void serializeBinary(const IColumn & column, WriteBuffer & ostr, size_t offset = 0, size_t limit = 0) const override
|
||||||
{
|
{
|
||||||
const ColumnTuple & real_column = static_cast<const ColumnTuple &>(column);
|
const ColumnTuple & real_column = static_cast<const ColumnTuple &>(column);
|
||||||
for (size_t i = 0, size = elems.size(); i < size; ++i)
|
for (size_t i = 0, size = elems.size(); i < size; ++i)
|
||||||
@ -124,14 +124,14 @@ public:
|
|||||||
* Именно из-за этого (невозможности читать меньший кусок записанных данных), Tuple не могут быть использованы для хранения данных в таблицах.
|
* Именно из-за этого (невозможности читать меньший кусок записанных данных), Tuple не могут быть использованы для хранения данных в таблицах.
|
||||||
* (Хотя могут быть использованы для передачи данных по сети в Native формате.)
|
* (Хотя могут быть использованы для передачи данных по сети в Native формате.)
|
||||||
*/
|
*/
|
||||||
void deserializeBinary(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const
|
void deserializeBinary(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override
|
||||||
{
|
{
|
||||||
ColumnTuple & real_column = static_cast<ColumnTuple &>(column);
|
ColumnTuple & real_column = static_cast<ColumnTuple &>(column);
|
||||||
for (size_t i = 0, size = elems.size(); i < size; ++i)
|
for (size_t i = 0, size = elems.size(); i < size; ++i)
|
||||||
NativeBlockInputStream::readData(*elems[i], *real_column.getData().getByPosition(i).column, istr, limit);
|
NativeBlockInputStream::readData(*elems[i], *real_column.getData().getByPosition(i).column, istr, limit);
|
||||||
}
|
}
|
||||||
|
|
||||||
ColumnPtr createColumn() const
|
ColumnPtr createColumn() const override
|
||||||
{
|
{
|
||||||
Block tuple_block;
|
Block tuple_block;
|
||||||
for (size_t i = 0, size = elems.size(); i < size; ++i)
|
for (size_t i = 0, size = elems.size(); i < size; ++i)
|
||||||
@ -144,12 +144,12 @@ public:
|
|||||||
return new ColumnTuple(tuple_block);
|
return new ColumnTuple(tuple_block);
|
||||||
}
|
}
|
||||||
|
|
||||||
ColumnPtr createConstColumn(size_t size, const Field & field) const
|
ColumnPtr createConstColumn(size_t size, const Field & field) const override
|
||||||
{
|
{
|
||||||
return new ColumnConstArray(size, get<const Array &>(field), new DataTypeTuple(elems));
|
return new ColumnConstArray(size, get<const Array &>(field), new DataTypeTuple(elems));
|
||||||
}
|
}
|
||||||
|
|
||||||
Field getDefault() const
|
Field getDefault() const override
|
||||||
{
|
{
|
||||||
size_t size = elems.size();
|
size_t size = elems.size();
|
||||||
Array res(size);
|
Array res(size);
|
||||||
|
@ -13,11 +13,11 @@ template <typename T>
|
|||||||
struct DataTypeFromFieldType;
|
struct DataTypeFromFieldType;
|
||||||
|
|
||||||
#define DEFINE_DATA_TYPE_NUMBER_FIXED(TYPE) \
|
#define DEFINE_DATA_TYPE_NUMBER_FIXED(TYPE) \
|
||||||
class DataType ## TYPE final : public IDataTypeNumberFixed<TYPE, Column ## TYPE> \
|
class DataType ## TYPE final : public IDataTypeNumberFixed<TYPE, Column ## TYPE> \
|
||||||
{ \
|
{ \
|
||||||
public: \
|
public: \
|
||||||
std::string getName() const { return #TYPE; } \
|
std::string getName() const override { return #TYPE; } \
|
||||||
DataTypePtr clone() const { return new DataType ## TYPE; } \
|
DataTypePtr clone() const override { return new DataType ## TYPE; } \
|
||||||
}; \
|
}; \
|
||||||
\
|
\
|
||||||
template <> struct DataTypeFromFieldType<TYPE> \
|
template <> struct DataTypeFromFieldType<TYPE> \
|
||||||
|
@ -4,7 +4,7 @@
|
|||||||
#include <DB/DataTypes/DataTypeString.h>
|
#include <DB/DataTypes/DataTypeString.h>
|
||||||
#include <DB/DataTypes/DataTypeArray.h>
|
#include <DB/DataTypes/DataTypeArray.h>
|
||||||
|
|
||||||
#include <DB/Core/Exception.h>
|
#include <DB/Common/Exception.h>
|
||||||
#include <DB/Core/ErrorCodes.h>
|
#include <DB/Core/ErrorCodes.h>
|
||||||
|
|
||||||
|
|
||||||
|
@ -20,36 +20,36 @@ private:
|
|||||||
}
|
}
|
||||||
|
|
||||||
public:
|
public:
|
||||||
void serializeBinary(const Field & field, WriteBuffer & ostr) const { throwNoSerialization(); }
|
void serializeBinary(const Field & field, WriteBuffer & ostr) const override { throwNoSerialization(); }
|
||||||
void deserializeBinary(Field & field, ReadBuffer & istr) const { throwNoSerialization(); }
|
void deserializeBinary(Field & field, ReadBuffer & istr) const override { throwNoSerialization(); }
|
||||||
|
|
||||||
void serializeBinary(const IColumn & column, WriteBuffer & ostr,
|
void serializeBinary(const IColumn & column, WriteBuffer & ostr,
|
||||||
size_t offset = 0, size_t limit = 0) const { throwNoSerialization(); }
|
size_t offset = 0, size_t limit = 0) const { throwNoSerialization(); }
|
||||||
|
|
||||||
void deserializeBinary(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const { throwNoSerialization(); }
|
void deserializeBinary(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override { throwNoSerialization(); }
|
||||||
|
|
||||||
void serializeText(const Field & field, WriteBuffer & ostr) const { throwNoSerialization(); }
|
void serializeText(const Field & field, WriteBuffer & ostr) const override { throwNoSerialization(); }
|
||||||
void deserializeText(Field & field, ReadBuffer & istr) const { throwNoSerialization(); }
|
void deserializeText(Field & field, ReadBuffer & istr) const override { throwNoSerialization(); }
|
||||||
|
|
||||||
void serializeTextEscaped(const Field & field, WriteBuffer & ostr) const { throwNoSerialization(); }
|
void serializeTextEscaped(const Field & field, WriteBuffer & ostr) const override { throwNoSerialization(); }
|
||||||
void deserializeTextEscaped(Field & field, ReadBuffer & istr) const { throwNoSerialization(); }
|
void deserializeTextEscaped(Field & field, ReadBuffer & istr) const override { throwNoSerialization(); }
|
||||||
|
|
||||||
void serializeTextQuoted(const Field & field, WriteBuffer & ostr) const { throwNoSerialization(); }
|
void serializeTextQuoted(const Field & field, WriteBuffer & ostr) const override { throwNoSerialization(); }
|
||||||
void deserializeTextQuoted(Field & field, ReadBuffer & istr) const { throwNoSerialization(); }
|
void deserializeTextQuoted(Field & field, ReadBuffer & istr) const override { throwNoSerialization(); }
|
||||||
|
|
||||||
void serializeTextJSON(const Field & field, WriteBuffer & ostr) const { throwNoSerialization(); }
|
void serializeTextJSON(const Field & field, WriteBuffer & ostr) const override { throwNoSerialization(); }
|
||||||
|
|
||||||
SharedPtr<IColumn> createColumn() const
|
SharedPtr<IColumn> createColumn() const override
|
||||||
{
|
{
|
||||||
throw Exception("Method createColumn() is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
throw Exception("Method createColumn() is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||||
}
|
}
|
||||||
|
|
||||||
SharedPtr<IColumn> createConstColumn(size_t size, const Field & field) const
|
SharedPtr<IColumn> createConstColumn(size_t size, const Field & field) const override
|
||||||
{
|
{
|
||||||
throw Exception("Method createConstColumn() is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
throw Exception("Method createConstColumn() is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||||
}
|
}
|
||||||
|
|
||||||
Field getDefault() const
|
Field getDefault() const override
|
||||||
{
|
{
|
||||||
throw Exception("Method getDefault() is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
throw Exception("Method getDefault() is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||||
}
|
}
|
||||||
|
@ -21,42 +21,42 @@ class IDataTypeNumber : public IDataType
|
|||||||
public:
|
public:
|
||||||
typedef FType FieldType;
|
typedef FType FieldType;
|
||||||
|
|
||||||
bool isNumeric() const { return true; }
|
bool isNumeric() const override { return true; }
|
||||||
|
|
||||||
bool behavesAsNumber() const { return true; }
|
bool behavesAsNumber() const override { return true; }
|
||||||
|
|
||||||
void serializeText(const Field & field, WriteBuffer & ostr) const
|
void serializeText(const Field & field, WriteBuffer & ostr) const override
|
||||||
{
|
{
|
||||||
writeText(static_cast<FieldType>(get<typename NearestFieldType<FieldType>::Type>(field)), ostr);
|
writeText(static_cast<FieldType>(get<typename NearestFieldType<FieldType>::Type>(field)), ostr);
|
||||||
}
|
}
|
||||||
|
|
||||||
inline void deserializeText(Field & field, ReadBuffer & istr) const;
|
|
||||||
|
|
||||||
void serializeTextEscaped(const Field & field, WriteBuffer & ostr) const
|
inline void deserializeText(Field & field, ReadBuffer & istr) const override;
|
||||||
|
|
||||||
|
void serializeTextEscaped(const Field & field, WriteBuffer & ostr) const override
|
||||||
{
|
{
|
||||||
serializeText(field, ostr);
|
serializeText(field, ostr);
|
||||||
}
|
}
|
||||||
|
|
||||||
void deserializeTextEscaped(Field & field, ReadBuffer & istr) const
|
void deserializeTextEscaped(Field & field, ReadBuffer & istr) const override
|
||||||
{
|
{
|
||||||
deserializeText(field, istr);
|
deserializeText(field, istr);
|
||||||
}
|
}
|
||||||
|
|
||||||
void serializeTextQuoted(const Field & field, WriteBuffer & ostr) const
|
void serializeTextQuoted(const Field & field, WriteBuffer & ostr) const override
|
||||||
{
|
{
|
||||||
serializeText(field, ostr);
|
serializeText(field, ostr);
|
||||||
}
|
}
|
||||||
|
|
||||||
void deserializeTextQuoted(Field & field, ReadBuffer & istr) const
|
void deserializeTextQuoted(Field & field, ReadBuffer & istr) const override
|
||||||
{
|
{
|
||||||
deserializeText(field, istr);
|
deserializeText(field, istr);
|
||||||
}
|
}
|
||||||
|
|
||||||
inline void serializeTextJSON(const Field & field, WriteBuffer & ostr) const;
|
|
||||||
|
|
||||||
size_t getSizeOfField() const { return sizeof(FieldType); }
|
inline void serializeTextJSON(const Field & field, WriteBuffer & ostr) const override;
|
||||||
|
|
||||||
Field getDefault() const
|
size_t getSizeOfField() const override { return sizeof(FieldType); }
|
||||||
|
|
||||||
|
Field getDefault() const override
|
||||||
{
|
{
|
||||||
return typename NearestFieldType<FieldType>::Type();
|
return typename NearestFieldType<FieldType>::Type();
|
||||||
}
|
}
|
||||||
|
@ -25,21 +25,21 @@ public:
|
|||||||
/** Формат платформозависимый (зависит от представления данных в памяти).
|
/** Формат платформозависимый (зависит от представления данных в памяти).
|
||||||
*/
|
*/
|
||||||
|
|
||||||
void serializeBinary(const Field & field, WriteBuffer & ostr) const
|
void serializeBinary(const Field & field, WriteBuffer & ostr) const override
|
||||||
{
|
{
|
||||||
/// ColumnType::value_type - более узкий тип. Например, UInt8, когда тип Field - UInt64
|
/// ColumnType::value_type - более узкий тип. Например, UInt8, когда тип Field - UInt64
|
||||||
typename ColumnType::value_type x = get<typename NearestFieldType<FieldType>::Type>(field);
|
typename ColumnType::value_type x = get<typename NearestFieldType<FieldType>::Type>(field);
|
||||||
writeBinary(x, ostr);
|
writeBinary(x, ostr);
|
||||||
}
|
}
|
||||||
|
|
||||||
void deserializeBinary(Field & field, ReadBuffer & istr) const
|
void deserializeBinary(Field & field, ReadBuffer & istr) const override
|
||||||
{
|
{
|
||||||
typename ColumnType::value_type x;
|
typename ColumnType::value_type x;
|
||||||
readBinary(x, istr);
|
readBinary(x, istr);
|
||||||
field = typename NearestFieldType<FieldType>::Type(x);
|
field = typename NearestFieldType<FieldType>::Type(x);
|
||||||
}
|
}
|
||||||
|
|
||||||
void serializeBinary(const IColumn & column, WriteBuffer & ostr, size_t offset = 0, size_t limit = 0) const
|
void serializeBinary(const IColumn & column, WriteBuffer & ostr, size_t offset = 0, size_t limit = 0) const override
|
||||||
{
|
{
|
||||||
const typename ColumnType::Container_t & x = typeid_cast<const ColumnType &>(column).getData();
|
const typename ColumnType::Container_t & x = typeid_cast<const ColumnType &>(column).getData();
|
||||||
|
|
||||||
@ -51,7 +51,7 @@ public:
|
|||||||
ostr.write(reinterpret_cast<const char *>(&x[offset]), sizeof(typename ColumnType::value_type) * limit);
|
ostr.write(reinterpret_cast<const char *>(&x[offset]), sizeof(typename ColumnType::value_type) * limit);
|
||||||
}
|
}
|
||||||
|
|
||||||
void deserializeBinary(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const
|
void deserializeBinary(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override
|
||||||
{
|
{
|
||||||
typename ColumnType::Container_t & x = typeid_cast<ColumnType &>(column).getData();
|
typename ColumnType::Container_t & x = typeid_cast<ColumnType &>(column).getData();
|
||||||
size_t initial_size = x.size();
|
size_t initial_size = x.size();
|
||||||
@ -60,12 +60,12 @@ public:
|
|||||||
x.resize(initial_size + size / sizeof(typename ColumnType::value_type));
|
x.resize(initial_size + size / sizeof(typename ColumnType::value_type));
|
||||||
}
|
}
|
||||||
|
|
||||||
ColumnPtr createColumn() const
|
ColumnPtr createColumn() const override
|
||||||
{
|
{
|
||||||
return new ColumnType;
|
return new ColumnType;
|
||||||
}
|
}
|
||||||
|
|
||||||
ColumnPtr createConstColumn(size_t size, const Field & field) const
|
ColumnPtr createConstColumn(size_t size, const Field & field) const override
|
||||||
{
|
{
|
||||||
return new ColumnConst<FieldType>(size, get<typename NearestFieldType<FieldType>::Type>(field));
|
return new ColumnConst<FieldType>(size, get<typename NearestFieldType<FieldType>::Type>(field));
|
||||||
}
|
}
|
||||||
|
@ -6,7 +6,7 @@
|
|||||||
#include <DB/Common/HashTable/HashMap.h>
|
#include <DB/Common/HashTable/HashMap.h>
|
||||||
#include <DB/Columns/ColumnString.h>
|
#include <DB/Columns/ColumnString.h>
|
||||||
#include <DB/Common/HashTable/HashMap.h>
|
#include <DB/Common/HashTable/HashMap.h>
|
||||||
#include <statdaemons/ext/scope_guard.hpp>
|
#include <ext/scope_guard.hpp>
|
||||||
#include <Poco/RWLock.h>
|
#include <Poco/RWLock.h>
|
||||||
#include <cmath>
|
#include <cmath>
|
||||||
#include <atomic>
|
#include <atomic>
|
||||||
|
@ -5,7 +5,7 @@
|
|||||||
#include <DB/DataStreams/RemoteBlockInputStream.h>
|
#include <DB/DataStreams/RemoteBlockInputStream.h>
|
||||||
#include <DB/Interpreters/executeQuery.h>
|
#include <DB/Interpreters/executeQuery.h>
|
||||||
#include <DB/Common/isLocalAddress.h>
|
#include <DB/Common/isLocalAddress.h>
|
||||||
#include <statdaemons/ext/range.hpp>
|
#include <ext/range.hpp>
|
||||||
#include <Poco/Util/AbstractConfiguration.h>
|
#include <Poco/Util/AbstractConfiguration.h>
|
||||||
#include "writeParenthesisedString.h"
|
#include "writeParenthesisedString.h"
|
||||||
|
|
||||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user