mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-18 13:42:02 +00:00
Merge
This commit is contained in:
commit
7423bd1224
@ -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
|
||||||
|
@ -2,6 +2,7 @@
|
|||||||
|
|
||||||
#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 <statdaemons/ext/range.hpp>
|
||||||
#include <boost/range/iterator_range_core.hpp>
|
#include <boost/range/iterator_range_core.hpp>
|
||||||
|
@ -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>
|
||||||
|
|
||||||
|
|
||||||
|
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);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -7,6 +7,7 @@
|
|||||||
#include <sstream>
|
#include <sstream>
|
||||||
#include <stats/ReservoirSampler.h>
|
#include <stats/ReservoirSampler.h>
|
||||||
#include <Yandex/Common.h>
|
#include <Yandex/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>
|
||||||
|
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);
|
@ -5,6 +5,7 @@
|
|||||||
#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
|
||||||
@ -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
|
||||||
|
@ -9,6 +9,7 @@
|
|||||||
#include <Yandex/logger_useful.h>
|
#include <Yandex/logger_useful.h>
|
||||||
|
|
||||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||||
|
#include <DB/Common/setThreadName.h>
|
||||||
|
|
||||||
|
|
||||||
/** Позволяет обработать множество источников блоков параллельно, используя указанное количество потоков.
|
/** Позволяет обработать множество источников блоков параллельно, используя указанное количество потоков.
|
||||||
@ -141,6 +142,8 @@ private:
|
|||||||
current_memory_tracker = memory_tracker;
|
current_memory_tracker = memory_tracker;
|
||||||
ExceptionPtr exception;
|
ExceptionPtr exception;
|
||||||
|
|
||||||
|
setThreadName("ParalInputsProc");
|
||||||
|
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
loop(thread_num);
|
loop(thread_num);
|
||||||
|
@ -344,16 +344,56 @@ struct LowerUpperUTF8Impl
|
|||||||
reinterpret_cast<UInt8 *>(&res_data[0]));
|
reinterpret_cast<UInt8 *>(&res_data[0]));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** Converts a single code point starting at `src` to desired case, storing result starting at `dst`.
|
||||||
|
* `src` and `dst` are incremented by corresponding sequence lengths. */
|
||||||
|
static void toCase(const UInt8 * & src, const UInt8 * const src_end, UInt8 * & dst)
|
||||||
|
{
|
||||||
|
if (src[0] <= ascii_upper_bound)
|
||||||
|
{
|
||||||
|
if (*src >= not_case_lower_bound && *src <= not_case_upper_bound)
|
||||||
|
*dst++ = *src++ ^ flip_case_mask;
|
||||||
|
else
|
||||||
|
*dst++ = *src++;
|
||||||
|
}
|
||||||
|
else if (src + 1 < src_end &&
|
||||||
|
((src[0] == 0xD0u && (src[1] >= 0x80u && src[1] <= 0xBFu)) ||
|
||||||
|
(src[0] == 0xD1u && (src[1] >= 0x80u && src[1] <= 0x9Fu))))
|
||||||
|
{
|
||||||
|
cyrillic_to_case(src, src_end, dst);
|
||||||
|
}
|
||||||
|
else if (src + 1 < src_end && src[0] == 0xC2u)
|
||||||
|
{
|
||||||
|
/// Пунктуация U+0080 - U+00BF, UTF-8: C2 80 - C2 BF
|
||||||
|
*dst++ = *src++;
|
||||||
|
*dst++ = *src++;
|
||||||
|
}
|
||||||
|
else if (src + 2 < src_end && src[0] == 0xE2u)
|
||||||
|
{
|
||||||
|
/// Символы U+2000 - U+2FFF, UTF-8: E2 80 80 - E2 BF BF
|
||||||
|
*dst++ = *src++;
|
||||||
|
*dst++ = *src++;
|
||||||
|
*dst++ = *src++;
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
static const Poco::UTF8Encoding utf8;
|
||||||
|
|
||||||
|
if (const auto chars = utf8.convert(to_case(utf8.convert(src)), dst, src_end - src))
|
||||||
|
src += chars, dst += chars;
|
||||||
|
else
|
||||||
|
++src, ++dst;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private:
|
private:
|
||||||
|
static constexpr auto ascii_upper_bound = '\x7f';
|
||||||
|
static constexpr auto flip_case_mask = 'A' ^ 'a';
|
||||||
|
|
||||||
static void array(const UInt8 * src, const UInt8 * src_end, UInt8 * dst)
|
static void array(const UInt8 * src, const UInt8 * src_end, UInt8 * dst)
|
||||||
{
|
{
|
||||||
static const Poco::UTF8Encoding utf8;
|
|
||||||
|
|
||||||
const auto bytes_sse = sizeof(__m128i);
|
const auto bytes_sse = sizeof(__m128i);
|
||||||
auto src_end_sse = src + (src_end - src) / bytes_sse * bytes_sse;
|
auto src_end_sse = src + (src_end - src) / bytes_sse * bytes_sse;
|
||||||
|
|
||||||
const auto flip_case_mask = 'A' ^ 'a';
|
|
||||||
const auto ascii_upper_bound = '\x7f';
|
|
||||||
/// SSE2 packed comparison operate on signed types, hence compare (c < 0) instead of (c > 0x7f)
|
/// SSE2 packed comparison operate on signed types, hence compare (c < 0) instead of (c > 0x7f)
|
||||||
const auto v_zero = _mm_setzero_si128();
|
const auto v_zero = _mm_setzero_si128();
|
||||||
const auto v_not_case_lower_bound = _mm_set1_epi8(not_case_lower_bound - 1);
|
const auto v_not_case_lower_bound = _mm_set1_epi8(not_case_lower_bound - 1);
|
||||||
@ -399,41 +439,7 @@ private:
|
|||||||
const auto expected_end = src + bytes_sse;
|
const auto expected_end = src + bytes_sse;
|
||||||
|
|
||||||
while (src < expected_end)
|
while (src < expected_end)
|
||||||
{
|
toCase(src, src_end, dst);
|
||||||
if (src[0] <= ascii_upper_bound)
|
|
||||||
{
|
|
||||||
if (*src >= not_case_lower_bound && *src <= not_case_upper_bound)
|
|
||||||
*dst++ = *src++ ^ flip_case_mask;
|
|
||||||
else
|
|
||||||
*dst++ = *src++;
|
|
||||||
}
|
|
||||||
else if (src + 1 < src_end &&
|
|
||||||
((src[0] == 0xD0u && (src[1] >= 0x80u && src[1] <= 0xBFu)) ||
|
|
||||||
(src[0] == 0xD1u && (src[1] >= 0x80u && src[1] <= 0x9Fu))))
|
|
||||||
{
|
|
||||||
cyrillic_to_case(src, src_end, dst);
|
|
||||||
}
|
|
||||||
else if (src + 1 < src_end && src[0] == 0xC2u)
|
|
||||||
{
|
|
||||||
/// Пунктуация U+0080 - U+00BF, UTF-8: C2 80 - C2 BF
|
|
||||||
*dst++ = *src++;
|
|
||||||
*dst++ = *src++;
|
|
||||||
}
|
|
||||||
else if (src + 2 < src_end && src[0] == 0xE2u)
|
|
||||||
{
|
|
||||||
/// Символы U+2000 - U+2FFF, UTF-8: E2 80 80 - E2 BF BF
|
|
||||||
*dst++ = *src++;
|
|
||||||
*dst++ = *src++;
|
|
||||||
*dst++ = *src++;
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
if (const auto chars = utf8.convert(to_case(utf8.convert(src)), dst, src_end - src))
|
|
||||||
src += chars, dst += chars;
|
|
||||||
else
|
|
||||||
++src, ++dst;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/// adjust src_end_sse by pushing it forward or backward
|
/// adjust src_end_sse by pushing it forward or backward
|
||||||
const auto diff = src - expected_end;
|
const auto diff = src - expected_end;
|
||||||
@ -449,10 +455,7 @@ private:
|
|||||||
|
|
||||||
/// handle remaining symbols
|
/// handle remaining symbols
|
||||||
while (src < src_end)
|
while (src < src_end)
|
||||||
if (const auto chars = utf8.convert(to_case(utf8.convert(src)), dst, src_end - src))
|
toCase(src, src_end, dst);
|
||||||
src += chars, dst += chars;
|
|
||||||
else
|
|
||||||
++src, ++dst;
|
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -15,10 +15,12 @@
|
|||||||
#include <DB/Functions/IFunction.h>
|
#include <DB/Functions/IFunction.h>
|
||||||
#include <re2/re2.h>
|
#include <re2/re2.h>
|
||||||
#include <re2/stringpiece.h>
|
#include <re2/stringpiece.h>
|
||||||
|
#include <Poco/UTF8Encoding.h>
|
||||||
|
|
||||||
#include <mutex>
|
#include <mutex>
|
||||||
#include <stack>
|
#include <stack>
|
||||||
#include <statdaemons/ext/range.hpp>
|
#include <statdaemons/ext/range.hpp>
|
||||||
|
#include <Poco/Unicode.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -103,6 +105,43 @@ struct PositionImpl
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
|
||||||
|
|
||||||
|
const UInt8 utf8_continuation_octet_mask = 0b11000000u;
|
||||||
|
const UInt8 utf8_continuation_octet = 0b10000000u;
|
||||||
|
|
||||||
|
|
||||||
|
/// return true if `octet` binary repr starts with 10 (octet is a UTF-8 sequence continuation)
|
||||||
|
bool utf8_is_continuation_octet(const UInt8 octet)
|
||||||
|
{
|
||||||
|
return (octet & utf8_continuation_octet_mask) == utf8_continuation_octet;
|
||||||
|
}
|
||||||
|
|
||||||
|
/// moves `s` forward until either first non-continuation octet or string end is met
|
||||||
|
void utf8_sync_forward(const UInt8 * & s, const UInt8 * const end = nullptr)
|
||||||
|
{
|
||||||
|
while (s < end && utf8_is_continuation_octet(*s))
|
||||||
|
++s;
|
||||||
|
}
|
||||||
|
|
||||||
|
/// returns UTF-8 code point sequence length judging by it's first octet
|
||||||
|
std::size_t utf8_seq_length(const UInt8 first_octet)
|
||||||
|
{
|
||||||
|
if (first_octet < 0x80u)
|
||||||
|
return 1;
|
||||||
|
|
||||||
|
const std::size_t bits = 8;
|
||||||
|
const auto first_zero = _bit_scan_reverse(static_cast<UInt8>(~first_octet));
|
||||||
|
|
||||||
|
return bits - 1 - first_zero;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
struct PositionUTF8Impl
|
struct PositionUTF8Impl
|
||||||
{
|
{
|
||||||
typedef UInt64 ResultType;
|
typedef UInt64 ResultType;
|
||||||
@ -136,7 +175,7 @@ struct PositionUTF8Impl
|
|||||||
/// А теперь надо найти, сколько кодовых точек находится перед pos.
|
/// А теперь надо найти, сколько кодовых точек находится перед pos.
|
||||||
res[i] = 1;
|
res[i] = 1;
|
||||||
for (const UInt8 * c = begin + (i != 0 ? offsets[i - 1] : 0); c < pos; ++c)
|
for (const UInt8 * c = begin + (i != 0 ? offsets[i - 1] : 0); c < pos; ++c)
|
||||||
if (*c <= 0x7F || *c >= 0xC0)
|
if (!utf8_is_continuation_octet(*c))
|
||||||
++res[i];
|
++res[i];
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
@ -151,63 +190,85 @@ struct PositionUTF8Impl
|
|||||||
|
|
||||||
static void constant(const std::string & data, const std::string & needle, UInt64 & res)
|
static void constant(const std::string & data, const std::string & needle, UInt64 & res)
|
||||||
{
|
{
|
||||||
res = data.find(needle);
|
const auto pos = data.find(needle);
|
||||||
if (res == std::string::npos)
|
if (pos != std::string::npos)
|
||||||
res = 0;
|
{
|
||||||
|
/// А теперь надо найти, сколько кодовых точек находится перед pos.
|
||||||
|
res = 1;
|
||||||
|
for (const auto i : ext::range(0, pos))
|
||||||
|
if (!utf8_is_continuation_octet(static_cast<UInt8>(data[i])))
|
||||||
|
++res;
|
||||||
|
}
|
||||||
else
|
else
|
||||||
++res;
|
res = 0;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
struct PositionCaseInsensitiveImpl
|
struct PositionCaseInsensitiveImpl
|
||||||
{
|
{
|
||||||
using ResultType = UInt64;
|
private:
|
||||||
|
class CaseInsensitiveSearcher
|
||||||
static void vector(
|
|
||||||
const ColumnString::Chars_t & data, const ColumnString::Offsets_t & offsets, const std::string & needle,
|
|
||||||
PODArray<UInt64> & res)
|
|
||||||
{
|
{
|
||||||
|
static constexpr auto n = sizeof(__m128i);
|
||||||
|
|
||||||
|
const int page_size = getpagesize();
|
||||||
|
|
||||||
|
/// string to be searched for
|
||||||
|
const std::string & needle;
|
||||||
/// lower and uppercase variants of the first character in `needle`
|
/// lower and uppercase variants of the first character in `needle`
|
||||||
const auto l = std::tolower(needle.front());
|
UInt8 l{};
|
||||||
const auto u = std::toupper(needle.front());
|
UInt8 u{};
|
||||||
/// for detecting leftmost position of the first symbol
|
/// vectors filled with `l` and `u`, for determining leftmost position of the first symbol
|
||||||
const auto patl = _mm_set1_epi8(l);
|
__m128i patl, patu;
|
||||||
const auto patu = _mm_set1_epi8(u);
|
|
||||||
/// lower and uppercase vectors of first 16 characters of `needle`
|
/// lower and uppercase vectors of first 16 characters of `needle`
|
||||||
auto cachel = _mm_setzero_si128();
|
__m128i cachel = _mm_setzero_si128(), cacheu = _mm_setzero_si128();
|
||||||
auto cacheu = _mm_setzero_si128();
|
int cachemask{};
|
||||||
int cachemask = 0;
|
|
||||||
|
|
||||||
const auto n = sizeof(cachel);
|
bool page_safe(const void * const ptr) const
|
||||||
const auto needle_begin = needle.data();
|
|
||||||
const auto needle_end = needle_begin + needle.size();
|
|
||||||
auto needle_pos = needle_begin;
|
|
||||||
|
|
||||||
for (const auto i : ext::range(0, n))
|
|
||||||
{
|
{
|
||||||
cachel = _mm_srli_si128(cachel, 1);
|
return ((page_size - 1) & reinterpret_cast<std::uintptr_t>(ptr)) <= page_size - n;
|
||||||
cacheu = _mm_srli_si128(cacheu, 1);
|
}
|
||||||
|
|
||||||
cachel = _mm_insert_epi8(cachel, std::tolower(*needle_pos), n - 1);
|
public:
|
||||||
cacheu = _mm_insert_epi8(cacheu, std::toupper(*needle_pos), n - 1);
|
CaseInsensitiveSearcher(const std::string & needle) : needle(needle)
|
||||||
|
{
|
||||||
|
if (needle.empty())
|
||||||
|
return;
|
||||||
|
|
||||||
if (needle_pos != needle_end)
|
auto needle_pos = needle.data();
|
||||||
|
|
||||||
|
l = std::tolower(*needle_pos);
|
||||||
|
u = std::toupper(*needle_pos);
|
||||||
|
|
||||||
|
patl = _mm_set1_epi8(l);
|
||||||
|
patu = _mm_set1_epi8(u);
|
||||||
|
|
||||||
|
const auto needle_end = needle_pos + needle.size();
|
||||||
|
|
||||||
|
for (const auto i : ext::range(0, n))
|
||||||
{
|
{
|
||||||
cachemask |= 1 << i;
|
cachel = _mm_srli_si128(cachel, 1);
|
||||||
++needle_pos;
|
cacheu = _mm_srli_si128(cacheu, 1);
|
||||||
|
|
||||||
|
if (needle_pos != needle_end)
|
||||||
|
{
|
||||||
|
cachel = _mm_insert_epi8(cachel, std::tolower(*needle_pos), n - 1);
|
||||||
|
cacheu = _mm_insert_epi8(cacheu, std::toupper(*needle_pos), n - 1);
|
||||||
|
cachemask |= 1 << i;
|
||||||
|
++needle_pos;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
const auto page_size = getpagesize();
|
const UInt8 * find(const UInt8 * haystack, const UInt8 * const haystack_end) const
|
||||||
const auto page_safe = [&] (const void * const ptr) {
|
{
|
||||||
return ((page_size - 1) & reinterpret_cast<std::uintptr_t>(ptr)) <= page_size - n;
|
if (needle.empty())
|
||||||
};
|
|
||||||
|
|
||||||
const auto find_ci = [&] (const UInt8 * haystack, const UInt8 * const haystack_end) -> const UInt8 * {
|
|
||||||
if (needle_begin == needle_end)
|
|
||||||
return haystack;
|
return haystack;
|
||||||
|
|
||||||
|
const auto needle_begin = reinterpret_cast<const UInt8 *>(needle.data());
|
||||||
|
const auto needle_end = needle_begin + needle.size();
|
||||||
|
|
||||||
while (haystack < haystack_end)
|
while (haystack < haystack_end)
|
||||||
{
|
{
|
||||||
/// @todo supposedly for long strings spanning across multiple pages. Why don't we use this technique in other places?
|
/// @todo supposedly for long strings spanning across multiple pages. Why don't we use this technique in other places?
|
||||||
@ -241,13 +302,14 @@ struct PositionCaseInsensitiveImpl
|
|||||||
{
|
{
|
||||||
if (mask == cachemask)
|
if (mask == cachemask)
|
||||||
{
|
{
|
||||||
auto s1 = haystack + n;
|
auto haystack_pos = haystack + n;
|
||||||
auto s2 = needle_begin + n;
|
auto needle_pos = needle_begin + n;
|
||||||
|
|
||||||
while (s1 < haystack_end && s2 < needle_end && std::tolower(*s1) == std::tolower(*s2))
|
while (haystack_pos < haystack_end && needle_pos < needle_end &&
|
||||||
++s1, ++s2;
|
std::tolower(*haystack_pos) == std::tolower(*needle_pos))
|
||||||
|
++haystack_pos, ++needle_pos;
|
||||||
|
|
||||||
if (s2 == needle_end)
|
if (needle_pos == needle_end)
|
||||||
return haystack;
|
return haystack;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -260,17 +322,18 @@ struct PositionCaseInsensitiveImpl
|
|||||||
}
|
}
|
||||||
|
|
||||||
if (haystack == haystack_end)
|
if (haystack == haystack_end)
|
||||||
return haystack;
|
return haystack_end;
|
||||||
|
|
||||||
if (*haystack == l || *haystack == u)
|
if (*haystack == l || *haystack == u)
|
||||||
{
|
{
|
||||||
auto s1 = haystack + 1;
|
auto haystack_pos = haystack + 1;
|
||||||
auto s2 = needle_begin + 1;
|
auto needle_pos = needle_begin + 1;
|
||||||
|
|
||||||
while (s1 < haystack_end && s2 < needle_end && std::tolower(*s1) == std::tolower(*s2))
|
while (haystack_pos < haystack_end && needle_pos < needle_end &&
|
||||||
++s1, ++s2;
|
std::tolower(*haystack_pos) == std::tolower(*needle_pos))
|
||||||
|
++haystack_pos, ++needle_pos;
|
||||||
|
|
||||||
if (s2 == needle_end)
|
if (needle_pos == needle_end)
|
||||||
return haystack;
|
return haystack;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -278,7 +341,17 @@ struct PositionCaseInsensitiveImpl
|
|||||||
}
|
}
|
||||||
|
|
||||||
return haystack_end;
|
return haystack_end;
|
||||||
};
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
public:
|
||||||
|
using ResultType = UInt64;
|
||||||
|
|
||||||
|
static void vector(
|
||||||
|
const ColumnString::Chars_t & data, const ColumnString::Offsets_t & offsets, const std::string & needle,
|
||||||
|
PODArray<UInt64> & res)
|
||||||
|
{
|
||||||
|
const CaseInsensitiveSearcher searcher{needle};
|
||||||
|
|
||||||
const UInt8 * begin = &data[0];
|
const UInt8 * begin = &data[0];
|
||||||
const UInt8 * pos = begin;
|
const UInt8 * pos = begin;
|
||||||
@ -288,7 +361,7 @@ struct PositionCaseInsensitiveImpl
|
|||||||
size_t i = 0;
|
size_t i = 0;
|
||||||
|
|
||||||
/// Искать будем следующее вхождение сразу во всех строках.
|
/// Искать будем следующее вхождение сразу во всех строках.
|
||||||
while (pos < end && end != (pos = find_ci(pos, end)))
|
while (pos < end && end != (pos = searcher.find(pos, end)))
|
||||||
{
|
{
|
||||||
/// Определим, к какому индексу оно относится.
|
/// Определим, к какому индексу оно относится.
|
||||||
while (begin + offsets[i] < pos)
|
while (begin + offsets[i] < pos)
|
||||||
@ -326,24 +399,290 @@ struct PositionCaseInsensitiveImpl
|
|||||||
|
|
||||||
struct PositionCaseInsensitiveUTF8Impl
|
struct PositionCaseInsensitiveUTF8Impl
|
||||||
{
|
{
|
||||||
|
private:
|
||||||
|
class CaseInsensitiveSearcher
|
||||||
|
{
|
||||||
|
using UTF8SequenceBuffer = UInt8[6];
|
||||||
|
|
||||||
|
static constexpr auto n = sizeof(__m128i);
|
||||||
|
|
||||||
|
const int page_size = getpagesize();
|
||||||
|
|
||||||
|
/// string to be searched for
|
||||||
|
const std::string & needle;
|
||||||
|
bool first_needle_symbol_is_ascii{};
|
||||||
|
/// lower and uppercase variants of the first octet of the first character in `needle`
|
||||||
|
UInt8 l{};
|
||||||
|
UInt8 u{};
|
||||||
|
/// vectors filled with `l` and `u`, for determining leftmost position of the first symbol
|
||||||
|
__m128i patl, patu;
|
||||||
|
/// lower and uppercase vectors of first 16 characters of `needle`
|
||||||
|
__m128i cachel = _mm_setzero_si128(), cacheu = _mm_setzero_si128();
|
||||||
|
int cachemask{};
|
||||||
|
std::size_t cache_valid_len{};
|
||||||
|
std::size_t cache_actual_len{};
|
||||||
|
|
||||||
|
bool page_safe(const void * const ptr) const
|
||||||
|
{
|
||||||
|
return ((page_size - 1) & reinterpret_cast<std::uintptr_t>(ptr)) <= page_size - n;
|
||||||
|
}
|
||||||
|
|
||||||
|
public:
|
||||||
|
CaseInsensitiveSearcher(const std::string & needle) : needle(needle)
|
||||||
|
{
|
||||||
|
if (needle.empty())
|
||||||
|
return;
|
||||||
|
|
||||||
|
static const Poco::UTF8Encoding utf8;
|
||||||
|
UTF8SequenceBuffer l_seq, u_seq;
|
||||||
|
|
||||||
|
auto needle_pos = reinterpret_cast<const UInt8 *>(needle.data());
|
||||||
|
if (*needle_pos < 0x80u)
|
||||||
|
{
|
||||||
|
first_needle_symbol_is_ascii = true;
|
||||||
|
l = std::tolower(*needle_pos);
|
||||||
|
u = std::toupper(*needle_pos);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
const auto first_u32 = utf8.convert(needle_pos);
|
||||||
|
const auto first_l_u32 = Poco::Unicode::toLower(first_u32);
|
||||||
|
const auto first_u_u32 = Poco::Unicode::toUpper(first_u32);
|
||||||
|
|
||||||
|
/// lower and uppercase variants of the first octet of the first character in `needle`
|
||||||
|
utf8.convert(first_l_u32, l_seq, sizeof(l_seq));
|
||||||
|
l = l_seq[0];
|
||||||
|
utf8.convert(first_u_u32, u_seq, sizeof(u_seq));
|
||||||
|
u = u_seq[0];
|
||||||
|
}
|
||||||
|
|
||||||
|
/// for detecting leftmost position of the first symbol
|
||||||
|
patl = _mm_set1_epi8(l);
|
||||||
|
patu = _mm_set1_epi8(u);
|
||||||
|
/// lower and uppercase vectors of first 16 octets of `needle`
|
||||||
|
|
||||||
|
const auto needle_end = needle_pos + needle.size();
|
||||||
|
|
||||||
|
for (std::size_t i = 0; i < n;)
|
||||||
|
{
|
||||||
|
if (needle_pos == needle_end)
|
||||||
|
{
|
||||||
|
cachel = _mm_srli_si128(cachel, 1);
|
||||||
|
cacheu = _mm_srli_si128(cacheu, 1);
|
||||||
|
++i;
|
||||||
|
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
const auto src_len = utf8_seq_length(*needle_pos);
|
||||||
|
const auto c_u32 = utf8.convert(needle_pos);
|
||||||
|
|
||||||
|
const auto c_l_u32 = Poco::Unicode::toLower(c_u32);
|
||||||
|
const auto c_u_u32 = Poco::Unicode::toUpper(c_u32);
|
||||||
|
|
||||||
|
const auto dst_l_len = static_cast<UInt8>(utf8.convert(c_l_u32, l_seq, sizeof(l_seq)));
|
||||||
|
const auto dst_u_len = static_cast<UInt8>(utf8.convert(c_u_u32, u_seq, sizeof(u_seq)));
|
||||||
|
|
||||||
|
/// @note Unicode standard states it is a rare but possible occasion
|
||||||
|
if (!(dst_l_len == dst_u_len && dst_u_len == src_len))
|
||||||
|
throw Exception{
|
||||||
|
"UTF8 sequences with different lowercase and uppercase lengths are not supported",
|
||||||
|
ErrorCodes::UNSUPPORTED_PARAMETER
|
||||||
|
};
|
||||||
|
|
||||||
|
cache_actual_len += src_len;
|
||||||
|
if (cache_actual_len < n)
|
||||||
|
cache_valid_len += src_len;
|
||||||
|
|
||||||
|
for (std::size_t j = 0; j < src_len && i < n; ++j, ++i)
|
||||||
|
{
|
||||||
|
cachel = _mm_srli_si128(cachel, 1);
|
||||||
|
cacheu = _mm_srli_si128(cacheu, 1);
|
||||||
|
|
||||||
|
if (needle_pos != needle_end)
|
||||||
|
{
|
||||||
|
cachel = _mm_insert_epi8(cachel, l_seq[j], n - 1);
|
||||||
|
cacheu = _mm_insert_epi8(cacheu, u_seq[j], n - 1);
|
||||||
|
|
||||||
|
cachemask |= 1 << i;
|
||||||
|
++needle_pos;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
const UInt8 * find(const UInt8 * haystack, const UInt8 * const haystack_end) const
|
||||||
|
{
|
||||||
|
if (needle.empty())
|
||||||
|
return haystack;
|
||||||
|
|
||||||
|
static const Poco::UTF8Encoding utf8;
|
||||||
|
|
||||||
|
const auto needle_begin = reinterpret_cast<const UInt8 *>(needle.data());
|
||||||
|
const auto needle_end = needle_begin + needle.size();
|
||||||
|
|
||||||
|
while (haystack < haystack_end)
|
||||||
|
{
|
||||||
|
if (haystack + n <= haystack_end && page_safe(haystack))
|
||||||
|
{
|
||||||
|
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
|
||||||
|
const auto v_against_l = _mm_cmpeq_epi8(v_haystack, patl);
|
||||||
|
const auto v_against_u = _mm_cmpeq_epi8(v_haystack, patu);
|
||||||
|
const auto v_against_l_or_u = _mm_or_si128(v_against_l, v_against_u);
|
||||||
|
|
||||||
|
const auto mask = _mm_movemask_epi8(v_against_l_or_u);
|
||||||
|
|
||||||
|
if (mask == 0)
|
||||||
|
{
|
||||||
|
haystack += n;
|
||||||
|
utf8_sync_forward(haystack, haystack_end);
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
const auto offset = _bit_scan_forward(mask);
|
||||||
|
haystack += offset;
|
||||||
|
|
||||||
|
if (haystack < haystack_end && haystack + n <= haystack_end && page_safe(haystack))
|
||||||
|
{
|
||||||
|
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
|
||||||
|
const auto v_against_l = _mm_cmpeq_epi8(v_haystack, cachel);
|
||||||
|
const auto v_against_u = _mm_cmpeq_epi8(v_haystack, cacheu);
|
||||||
|
const auto v_against_l_or_u = _mm_or_si128(v_against_l, v_against_u);
|
||||||
|
const auto mask = _mm_movemask_epi8(v_against_l_or_u);
|
||||||
|
|
||||||
|
if (0xffff == cachemask)
|
||||||
|
{
|
||||||
|
if (mask == cachemask)
|
||||||
|
{
|
||||||
|
auto haystack_pos = haystack + cache_valid_len;
|
||||||
|
auto needle_pos = needle_begin + cache_valid_len;
|
||||||
|
|
||||||
|
while (haystack_pos < haystack_end && needle_pos < needle_end &&
|
||||||
|
Poco::Unicode::toLower(utf8.convert(haystack_pos)) ==
|
||||||
|
Poco::Unicode::toLower(utf8.convert(needle_pos)))
|
||||||
|
{
|
||||||
|
/// @note assuming sequences for lowercase and uppercase have exact same length
|
||||||
|
const auto len = utf8_seq_length(*haystack_pos);
|
||||||
|
haystack_pos += len, needle_pos += len;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (needle_pos == needle_end)
|
||||||
|
return haystack;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
else if ((mask & cachemask) == cachemask)
|
||||||
|
return haystack;
|
||||||
|
|
||||||
|
/// first octet was ok, but not the first 16, move to start of next sequence and reapply
|
||||||
|
haystack += utf8_seq_length(*haystack);
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (haystack == haystack_end)
|
||||||
|
return haystack_end;
|
||||||
|
|
||||||
|
if (*haystack == l || *haystack == u)
|
||||||
|
{
|
||||||
|
auto haystack_pos = haystack + first_needle_symbol_is_ascii;
|
||||||
|
auto needle_pos = needle_begin + first_needle_symbol_is_ascii;
|
||||||
|
|
||||||
|
while (haystack_pos < haystack_end && needle_pos < needle_end &&
|
||||||
|
Poco::Unicode::toLower(utf8.convert(haystack_pos)) ==
|
||||||
|
Poco::Unicode::toLower(utf8.convert(needle_pos)))
|
||||||
|
{
|
||||||
|
const auto len = utf8_seq_length(*haystack_pos);
|
||||||
|
haystack_pos += len, needle_pos += len;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (needle_pos == needle_end)
|
||||||
|
return haystack;
|
||||||
|
}
|
||||||
|
|
||||||
|
/// advance to the start of the next sequence
|
||||||
|
haystack += utf8_seq_length(*haystack);
|
||||||
|
}
|
||||||
|
|
||||||
|
return haystack_end;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
public:
|
||||||
using ResultType = UInt64;
|
using ResultType = UInt64;
|
||||||
|
|
||||||
static void vector(
|
static void vector(
|
||||||
const ColumnString::Chars_t & data, const ColumnString::Offsets_t & offsets, const std::string & needle,
|
const ColumnString::Chars_t & data, const ColumnString::Offsets_t & offsets, const std::string & needle,
|
||||||
PODArray<UInt64> & res)
|
PODArray<UInt64> & res)
|
||||||
{
|
{
|
||||||
throw Exception{
|
const CaseInsensitiveSearcher searcher{needle};
|
||||||
"Not yet implemented",
|
|
||||||
ErrorCodes::NOT_IMPLEMENTED
|
const UInt8 * begin = &data[0];
|
||||||
};
|
const UInt8 * pos = begin;
|
||||||
|
const UInt8 * end = pos + data.size();
|
||||||
|
|
||||||
|
/// Текущий индекс в массиве строк.
|
||||||
|
size_t i = 0;
|
||||||
|
|
||||||
|
/// Искать будем следующее вхождение сразу во всех строках.
|
||||||
|
while (pos < end && end != (pos = searcher.find(pos, end)))
|
||||||
|
{
|
||||||
|
/// Определим, к какому индексу оно относится.
|
||||||
|
while (begin + offsets[i] < pos)
|
||||||
|
{
|
||||||
|
res[i] = 0;
|
||||||
|
++i;
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Проверяем, что вхождение не переходит через границы строк.
|
||||||
|
if (pos + needle.size() < begin + offsets[i])
|
||||||
|
{
|
||||||
|
/// А теперь надо найти, сколько кодовых точек находится перед pos.
|
||||||
|
res[i] = 1;
|
||||||
|
for (const UInt8 * c = begin + (i != 0 ? offsets[i - 1] : 0); c < pos; ++c)
|
||||||
|
if (!utf8_is_continuation_octet(*c))
|
||||||
|
++res[i];
|
||||||
|
}
|
||||||
|
else
|
||||||
|
res[i] = 0;
|
||||||
|
|
||||||
|
pos = begin + offsets[i];
|
||||||
|
++i;
|
||||||
|
}
|
||||||
|
|
||||||
|
memset(&res[i], 0, (res.size() - i) * sizeof(res[0]));
|
||||||
}
|
}
|
||||||
|
|
||||||
static void constant(const std::string & data, const std::string & needle, UInt64 & res)
|
static void constant(std::string data, std::string needle, UInt64 & res)
|
||||||
{
|
{
|
||||||
throw Exception{
|
static const Poco::UTF8Encoding utf8;
|
||||||
"Not yet implemented",
|
|
||||||
ErrorCodes::NOT_IMPLEMENTED
|
auto data_pos = reinterpret_cast<UInt8 *>(&data[0]);
|
||||||
};
|
const auto data_end = data_pos + data.size();
|
||||||
|
while (data_pos < data_end)
|
||||||
|
{
|
||||||
|
const auto len = utf8.convert(Poco::Unicode::toLower(utf8.convert(data_pos)), data_pos, data_end - data_pos);
|
||||||
|
data_pos += len;
|
||||||
|
}
|
||||||
|
|
||||||
|
auto needle_pos = reinterpret_cast<UInt8 *>(&needle[0]);
|
||||||
|
const auto needle_end = needle_pos + needle.size();
|
||||||
|
while (needle_pos < needle_end)
|
||||||
|
{
|
||||||
|
const auto len = utf8.convert(Poco::Unicode::toLower(utf8.convert(needle_pos)), needle_pos, needle_end - needle_pos);
|
||||||
|
needle_pos += len;
|
||||||
|
}
|
||||||
|
|
||||||
|
const auto pos = data.find(needle);
|
||||||
|
if (pos != std::string::npos)
|
||||||
|
{
|
||||||
|
/// А теперь надо найти, сколько кодовых точек находится перед pos.
|
||||||
|
res = 1;
|
||||||
|
for (const auto i : ext::range(0, pos))
|
||||||
|
if (!utf8_is_continuation_octet(static_cast<UInt8>(data[i])))
|
||||||
|
++res;
|
||||||
|
}
|
||||||
|
else
|
||||||
|
res = 0;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -1394,7 +1733,7 @@ public:
|
|||||||
struct NamePosition { static constexpr auto name = "position"; };
|
struct NamePosition { static constexpr auto name = "position"; };
|
||||||
struct NamePositionUTF8 { static constexpr auto name = "positionUTF8"; };
|
struct NamePositionUTF8 { static constexpr auto name = "positionUTF8"; };
|
||||||
struct NamePositionCaseInsensitive { static constexpr auto name = "positionCaseInsensitive"; };
|
struct NamePositionCaseInsensitive { static constexpr auto name = "positionCaseInsensitive"; };
|
||||||
struct NamePositionCaseInsenseitiveUTF8 { static constexpr auto name = "positionCaseInsensitiveUTF8"; };
|
struct NamePositionCaseInsensitiveUTF8 { static constexpr auto name = "positionCaseInsensitiveUTF8"; };
|
||||||
struct NameMatch { static constexpr auto name = "match"; };
|
struct NameMatch { static constexpr auto name = "match"; };
|
||||||
struct NameLike { static constexpr auto name = "like"; };
|
struct NameLike { static constexpr auto name = "like"; };
|
||||||
struct NameNotLike { static constexpr auto name = "notLike"; };
|
struct NameNotLike { static constexpr auto name = "notLike"; };
|
||||||
@ -1407,7 +1746,7 @@ struct NameReplaceRegexpAll { static constexpr auto name = "replaceRegexpAll"
|
|||||||
typedef FunctionsStringSearch<PositionImpl, NamePosition> FunctionPosition;
|
typedef FunctionsStringSearch<PositionImpl, NamePosition> FunctionPosition;
|
||||||
typedef FunctionsStringSearch<PositionUTF8Impl, NamePositionUTF8> FunctionPositionUTF8;
|
typedef FunctionsStringSearch<PositionUTF8Impl, NamePositionUTF8> FunctionPositionUTF8;
|
||||||
typedef FunctionsStringSearch<PositionCaseInsensitiveImpl, NamePositionCaseInsensitive> FunctionPositionCaseInsensitive;
|
typedef FunctionsStringSearch<PositionCaseInsensitiveImpl, NamePositionCaseInsensitive> FunctionPositionCaseInsensitive;
|
||||||
typedef FunctionsStringSearch<PositionCaseInsensitiveUTF8Impl, NamePositionCaseInsenseitiveUTF8> FunctionPositionCaseInsensitiveUTF8;
|
typedef FunctionsStringSearch<PositionCaseInsensitiveUTF8Impl, NamePositionCaseInsensitiveUTF8> FunctionPositionCaseInsensitiveUTF8;
|
||||||
typedef FunctionsStringSearch<MatchImpl<false>, NameMatch> FunctionMatch;
|
typedef FunctionsStringSearch<MatchImpl<false>, NameMatch> FunctionMatch;
|
||||||
typedef FunctionsStringSearch<MatchImpl<true>, NameLike> FunctionLike;
|
typedef FunctionsStringSearch<MatchImpl<true>, NameLike> FunctionLike;
|
||||||
typedef FunctionsStringSearch<MatchImpl<true, true>, NameNotLike> FunctionNotLike;
|
typedef FunctionsStringSearch<MatchImpl<true, true>, NameNotLike> FunctionNotLike;
|
||||||
|
@ -190,6 +190,8 @@ public:
|
|||||||
/// Как другие серверы могут обратиться к этому для скачивания реплицируемых данных.
|
/// Как другие серверы могут обратиться к этому для скачивания реплицируемых данных.
|
||||||
void setInterserverIOAddress(const String & host, UInt16 port);
|
void setInterserverIOAddress(const String & host, UInt16 port);
|
||||||
std::pair<String, UInt16> getInterserverIOAddress() const;
|
std::pair<String, UInt16> getInterserverIOAddress() const;
|
||||||
|
/// Порт, который сервер слушает для выполнения SQL-запросов.
|
||||||
|
UInt16 getTCPPort() const;
|
||||||
|
|
||||||
/// Получить запрос на CREATE таблицы.
|
/// Получить запрос на CREATE таблицы.
|
||||||
ASTPtr getCreateQuery(const String & database_name, const String & table_name) const;
|
ASTPtr getCreateQuery(const String & database_name, const String & table_name) const;
|
||||||
|
@ -6,6 +6,7 @@
|
|||||||
#include <statdaemons/RegionsHierarchies.h>
|
#include <statdaemons/RegionsHierarchies.h>
|
||||||
#include <statdaemons/TechDataHierarchy.h>
|
#include <statdaemons/TechDataHierarchy.h>
|
||||||
#include <statdaemons/RegionsNames.h>
|
#include <statdaemons/RegionsNames.h>
|
||||||
|
#include <DB/Common/setThreadName.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -123,6 +124,8 @@ private:
|
|||||||
/// Обновляет каждые reload_period секунд.
|
/// Обновляет каждые reload_period секунд.
|
||||||
void reloadPeriodically()
|
void reloadPeriodically()
|
||||||
{
|
{
|
||||||
|
setThreadName("DictReload");
|
||||||
|
|
||||||
while (true)
|
while (true)
|
||||||
{
|
{
|
||||||
if (destroy.tryWait(reload_period * 1000))
|
if (destroy.tryWait(reload_period * 1000))
|
||||||
|
@ -3,6 +3,7 @@
|
|||||||
#include <DB/Dictionaries/IDictionary.h>
|
#include <DB/Dictionaries/IDictionary.h>
|
||||||
#include <DB/Core/Exception.h>
|
#include <DB/Core/Exception.h>
|
||||||
#include <DB/Core/ErrorCodes.h>
|
#include <DB/Core/ErrorCodes.h>
|
||||||
|
#include <DB/Common/setThreadName.h>
|
||||||
#include <Yandex/MultiVersion.h>
|
#include <Yandex/MultiVersion.h>
|
||||||
#include <Yandex/logger_useful.h>
|
#include <Yandex/logger_useful.h>
|
||||||
#include <Poco/Event.h>
|
#include <Poco/Event.h>
|
||||||
@ -86,6 +87,8 @@ private:
|
|||||||
|
|
||||||
void reloadPeriodically()
|
void reloadPeriodically()
|
||||||
{
|
{
|
||||||
|
setThreadName("ExterDictReload");
|
||||||
|
|
||||||
while (true)
|
while (true)
|
||||||
{
|
{
|
||||||
if (destroy.tryWait(check_period_sec * 1000))
|
if (destroy.tryWait(check_period_sec * 1000))
|
||||||
|
@ -8,11 +8,16 @@
|
|||||||
#include <Poco/Net/DNS.h>
|
#include <Poco/Net/DNS.h>
|
||||||
#include <Poco/Util/Application.h>
|
#include <Poco/Util/Application.h>
|
||||||
#include <Poco/Util/AbstractConfiguration.h>
|
#include <Poco/Util/AbstractConfiguration.h>
|
||||||
|
#include <Poco/String.h>
|
||||||
|
|
||||||
#include <DB/Core/Types.h>
|
#include <DB/Core/Types.h>
|
||||||
#include <DB/Core/Exception.h>
|
#include <DB/Core/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/HexWriteBuffer.h>
|
||||||
|
#include <DB/IO/WriteBufferFromString.h>
|
||||||
|
|
||||||
|
#include <openssl/sha.h>
|
||||||
|
|
||||||
#include <Yandex/logger_useful.h>
|
#include <Yandex/logger_useful.h>
|
||||||
|
|
||||||
@ -64,7 +69,7 @@ public:
|
|||||||
construct(Poco::Net::IPAddress(addr), prefix_bits_);
|
construct(Poco::Net::IPAddress(addr), prefix_bits_);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
bool contains(const Poco::Net::IPAddress & addr) const
|
bool contains(const Poco::Net::IPAddress & addr) const
|
||||||
{
|
{
|
||||||
return prefixBitsEquals(reinterpret_cast<const char *>(toIPv6(addr).addr()), reinterpret_cast<const char *>(mask_address.addr()), prefix_bits);
|
return prefixBitsEquals(reinterpret_cast<const char *>(toIPv6(addr).addr()), reinterpret_cast<const char *>(mask_address.addr()), prefix_bits);
|
||||||
@ -168,7 +173,7 @@ private:
|
|||||||
|
|
||||||
public:
|
public:
|
||||||
HostRegexpPattern(const String & host_regexp_) : host_regexp(host_regexp_) {}
|
HostRegexpPattern(const String & host_regexp_) : host_regexp(host_regexp_) {}
|
||||||
|
|
||||||
bool contains(const Poco::Net::IPAddress & addr) const
|
bool contains(const Poco::Net::IPAddress & addr) const
|
||||||
{
|
{
|
||||||
Poco::Net::SocketAddress sock_addr(addr, 0);
|
Poco::Net::SocketAddress sock_addr(addr, 0);
|
||||||
@ -243,7 +248,7 @@ public:
|
|||||||
pattern = new HostExactPattern(value);
|
pattern = new HostExactPattern(value);
|
||||||
else
|
else
|
||||||
throw Exception("Unknown address pattern type: " + *it, ErrorCodes::UNKNOWN_ADDRESS_PATTERN_TYPE);
|
throw Exception("Unknown address pattern type: " + *it, ErrorCodes::UNKNOWN_ADDRESS_PATTERN_TYPE);
|
||||||
|
|
||||||
patterns.push_back(pattern);
|
patterns.push_back(pattern);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -256,8 +261,9 @@ struct User
|
|||||||
{
|
{
|
||||||
String name;
|
String name;
|
||||||
|
|
||||||
/// Требуемый пароль. Хранится в открытом виде.
|
/// Требуемый пароль. Может храниться либо в открытом виде, либо в виде SHA256.
|
||||||
String password;
|
String password;
|
||||||
|
String password_sha256_hex;
|
||||||
|
|
||||||
String profile;
|
String profile;
|
||||||
String quota;
|
String quota;
|
||||||
@ -267,7 +273,26 @@ struct User
|
|||||||
User(const String & name_, const String & config_elem, Poco::Util::AbstractConfiguration & config)
|
User(const String & name_, const String & config_elem, Poco::Util::AbstractConfiguration & config)
|
||||||
: name(name_)
|
: name(name_)
|
||||||
{
|
{
|
||||||
password = config.getString(config_elem + ".password");
|
bool has_password = config.has(config_elem + ".password");
|
||||||
|
bool has_password_sha256_hex = config.has(config_elem + ".password_sha256_hex");
|
||||||
|
|
||||||
|
if (has_password && has_password_sha256_hex)
|
||||||
|
throw Exception("Both fields 'password' and 'password_sha256_hex' are specified for user " + name + ". Must be only one of them.", ErrorCodes::BAD_ARGUMENTS);
|
||||||
|
|
||||||
|
if (!has_password && !has_password_sha256_hex)
|
||||||
|
throw Exception("Either 'password' or 'password_sha256_hex' must be specified for user " + name + ".", ErrorCodes::BAD_ARGUMENTS);
|
||||||
|
|
||||||
|
if (has_password)
|
||||||
|
password = config.getString(config_elem + ".password");
|
||||||
|
|
||||||
|
if (has_password_sha256_hex)
|
||||||
|
{
|
||||||
|
password_sha256_hex = Poco::toLower(config.getString(config_elem + ".password_sha256_hex"));
|
||||||
|
|
||||||
|
if (password_sha256_hex.size() != 64)
|
||||||
|
throw Exception("password_sha256_hex for user " + name + " has length " + toString(password_sha256_hex.size()) + " but must be exactly 64 symbols.", ErrorCodes::BAD_ARGUMENTS);
|
||||||
|
}
|
||||||
|
|
||||||
profile = config.getString(config_elem + ".profile");
|
profile = config.getString(config_elem + ".profile");
|
||||||
quota = config.getString(config_elem + ".quota");
|
quota = config.getString(config_elem + ".quota");
|
||||||
|
|
||||||
@ -285,7 +310,7 @@ class Users
|
|||||||
private:
|
private:
|
||||||
typedef std::map<String, User> Container;
|
typedef std::map<String, User> Container;
|
||||||
Container cont;
|
Container cont;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
void loadFromConfig(Poco::Util::AbstractConfiguration & config)
|
void loadFromConfig(Poco::Util::AbstractConfiguration & config)
|
||||||
{
|
{
|
||||||
@ -308,12 +333,38 @@ public:
|
|||||||
if (!it->second.addresses.contains(address))
|
if (!it->second.addresses.contains(address))
|
||||||
throw Exception("User " + name + " is not allowed to connect from address " + address.toString(), ErrorCodes::IP_ADDRESS_NOT_ALLOWED);
|
throw Exception("User " + name + " is not allowed to connect from address " + address.toString(), ErrorCodes::IP_ADDRESS_NOT_ALLOWED);
|
||||||
|
|
||||||
if (password != it->second.password)
|
auto on_wrong_password = [&]()
|
||||||
{
|
{
|
||||||
if (password.empty())
|
if (password.empty())
|
||||||
throw Exception("Password required for user " + name, ErrorCodes::REQUIRED_PASSWORD);
|
throw Exception("Password required for user " + name, ErrorCodes::REQUIRED_PASSWORD);
|
||||||
else
|
else
|
||||||
throw Exception("Wrong password for user " + name, ErrorCodes::WRONG_PASSWORD);
|
throw Exception("Wrong password for user " + name, ErrorCodes::WRONG_PASSWORD);
|
||||||
|
};
|
||||||
|
|
||||||
|
if (!it->second.password_sha256_hex.empty())
|
||||||
|
{
|
||||||
|
unsigned char hash[32];
|
||||||
|
|
||||||
|
SHA256_CTX ctx;
|
||||||
|
SHA256_Init(&ctx);
|
||||||
|
SHA256_Update(&ctx, reinterpret_cast<const unsigned char *>(password.data()), password.size());
|
||||||
|
SHA256_Final(hash, &ctx);
|
||||||
|
|
||||||
|
String hash_hex;
|
||||||
|
{
|
||||||
|
WriteBufferFromString buf(hash_hex);
|
||||||
|
HexWriteBuffer hex_buf(buf);
|
||||||
|
hex_buf.write(reinterpret_cast<const char *>(hash), sizeof(hash));
|
||||||
|
}
|
||||||
|
|
||||||
|
Poco::toLowerInPlace(hash_hex);
|
||||||
|
|
||||||
|
if (hash_hex != it->second.password_sha256_hex)
|
||||||
|
on_wrong_password();
|
||||||
|
}
|
||||||
|
else if (password != it->second.password)
|
||||||
|
{
|
||||||
|
on_wrong_password();
|
||||||
}
|
}
|
||||||
|
|
||||||
return it->second;
|
return it->second;
|
||||||
|
@ -2,6 +2,7 @@
|
|||||||
|
|
||||||
#include <DB/DataStreams/RemoteBlockOutputStream.h>
|
#include <DB/DataStreams/RemoteBlockOutputStream.h>
|
||||||
#include <DB/Common/escapeForFileName.h>
|
#include <DB/Common/escapeForFileName.h>
|
||||||
|
#include <DB/Common/setThreadName.h>
|
||||||
#include <DB/Storages/StorageDistributed.h>
|
#include <DB/Storages/StorageDistributed.h>
|
||||||
#include <DB/IO/ReadBufferFromFile.h>
|
#include <DB/IO/ReadBufferFromFile.h>
|
||||||
|
|
||||||
@ -86,6 +87,8 @@ public:
|
|||||||
private:
|
private:
|
||||||
void run()
|
void run()
|
||||||
{
|
{
|
||||||
|
setThreadName("DistrDirMonitor");
|
||||||
|
|
||||||
std::unique_lock<std::mutex> lock{mutex};
|
std::unique_lock<std::mutex> lock{mutex};
|
||||||
|
|
||||||
const auto quit_requested = [this] { return quit; };
|
const auto quit_requested = [this] { return quit; };
|
||||||
|
@ -8,11 +8,8 @@
|
|||||||
#include <Poco/Mutex.h>
|
#include <Poco/Mutex.h>
|
||||||
#include <Poco/RWLock.h>
|
#include <Poco/RWLock.h>
|
||||||
#include <Poco/Event.h>
|
#include <Poco/Event.h>
|
||||||
|
#include <Poco/SharedPtr.h>
|
||||||
#include <DB/Core/Types.h>
|
#include <DB/Core/Types.h>
|
||||||
#include <DB/Core/Exception.h>
|
|
||||||
#include <DB/Core/ErrorCodes.h>
|
|
||||||
#include <DB/IO/WriteHelpers.h>
|
|
||||||
#include <Yandex/logger_useful.h>
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -63,22 +60,7 @@ public:
|
|||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
/// Разбудить какой-нибудь поток.
|
/// Разбудить какой-нибудь поток.
|
||||||
void wake()
|
void wake();
|
||||||
{
|
|
||||||
if (removed)
|
|
||||||
return;
|
|
||||||
|
|
||||||
std::unique_lock<std::mutex> lock(pool.mutex);
|
|
||||||
pool.tasks.splice(pool.tasks.begin(), pool.tasks, iterator);
|
|
||||||
|
|
||||||
/// Если эта задача в прошлый раз ничего не сделала, и ей было назначено спать, то отменим время сна.
|
|
||||||
time_t current_time = time(0);
|
|
||||||
if (next_time_to_execute > current_time)
|
|
||||||
next_time_to_execute = current_time;
|
|
||||||
|
|
||||||
/// Если все потоки сейчас выполняют работу, этот вызов никого не разбудит.
|
|
||||||
pool.wake_event.notify_one();
|
|
||||||
}
|
|
||||||
|
|
||||||
private:
|
private:
|
||||||
friend class BackgroundProcessingPool;
|
friend class BackgroundProcessingPool;
|
||||||
@ -99,77 +81,27 @@ public:
|
|||||||
typedef std::shared_ptr<TaskInfo> TaskHandle;
|
typedef std::shared_ptr<TaskInfo> TaskHandle;
|
||||||
|
|
||||||
|
|
||||||
BackgroundProcessingPool(int size_) : size(size_)
|
BackgroundProcessingPool(int size_);
|
||||||
{
|
|
||||||
LOG_INFO(&Logger::get("BackgroundProcessingPool"), "Create BackgroundProcessingPool with " << size << " threads");
|
|
||||||
|
|
||||||
threads.resize(size);
|
|
||||||
for (auto & thread : threads)
|
|
||||||
thread = std::thread([this] { threadFunction(); });
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
size_t getNumberOfThreads() const
|
size_t getNumberOfThreads() const
|
||||||
{
|
{
|
||||||
return size;
|
return size;
|
||||||
}
|
}
|
||||||
|
|
||||||
int getCounter(const String & name)
|
int getCounter(const String & name);
|
||||||
{
|
|
||||||
std::unique_lock<std::mutex> lock(mutex);
|
|
||||||
return counters[name];
|
|
||||||
}
|
|
||||||
|
|
||||||
TaskHandle addTask(const Task & task)
|
TaskHandle addTask(const Task & task);
|
||||||
{
|
void removeTask(const TaskHandle & task);
|
||||||
TaskHandle res(new TaskInfo(*this, task));
|
|
||||||
|
|
||||||
{
|
~BackgroundProcessingPool();
|
||||||
std::unique_lock<std::mutex> lock(mutex);
|
|
||||||
res->iterator = tasks.insert(tasks.begin(), res);
|
|
||||||
}
|
|
||||||
|
|
||||||
wake_event.notify_all();
|
|
||||||
|
|
||||||
return res;
|
|
||||||
}
|
|
||||||
|
|
||||||
void removeTask(const TaskHandle & task)
|
|
||||||
{
|
|
||||||
task->removed = true;
|
|
||||||
|
|
||||||
/// Дождёмся завершения всех выполнений этой задачи.
|
|
||||||
{
|
|
||||||
Poco::ScopedWriteRWLock wlock(task->rwlock);
|
|
||||||
}
|
|
||||||
|
|
||||||
{
|
|
||||||
std::unique_lock<std::mutex> lock(mutex);
|
|
||||||
tasks.erase(task->iterator);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
~BackgroundProcessingPool()
|
|
||||||
{
|
|
||||||
try
|
|
||||||
{
|
|
||||||
shutdown = true;
|
|
||||||
wake_event.notify_all();
|
|
||||||
for (std::thread & thread : threads)
|
|
||||||
thread.join();
|
|
||||||
}
|
|
||||||
catch (...)
|
|
||||||
{
|
|
||||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private:
|
private:
|
||||||
typedef std::list<TaskHandle> Tasks;
|
typedef std::list<TaskHandle> Tasks;
|
||||||
typedef std::vector<std::thread> Threads;
|
typedef std::vector<std::thread> Threads;
|
||||||
|
|
||||||
const size_t size;
|
const size_t size;
|
||||||
enum { sleep_seconds = 10 };
|
static constexpr double sleep_seconds = 10;
|
||||||
|
static constexpr double sleep_seconds_random_part = 1.0;
|
||||||
|
|
||||||
Tasks tasks; /// Задачи в порядке, в котором мы планируем их выполнять.
|
Tasks tasks; /// Задачи в порядке, в котором мы планируем их выполнять.
|
||||||
Counters counters;
|
Counters counters;
|
||||||
@ -181,97 +113,7 @@ private:
|
|||||||
std::condition_variable wake_event;
|
std::condition_variable wake_event;
|
||||||
|
|
||||||
|
|
||||||
void threadFunction()
|
void threadFunction();
|
||||||
{
|
|
||||||
while (!shutdown)
|
|
||||||
{
|
|
||||||
Counters counters_diff;
|
|
||||||
bool has_exception = false;
|
|
||||||
|
|
||||||
try
|
|
||||||
{
|
|
||||||
TaskHandle task;
|
|
||||||
time_t min_time = std::numeric_limits<time_t>::max();
|
|
||||||
|
|
||||||
{
|
|
||||||
std::unique_lock<std::mutex> lock(mutex);
|
|
||||||
|
|
||||||
if (!tasks.empty())
|
|
||||||
{
|
|
||||||
/// O(n), n - число задач. По сути, количество таблиц. Обычно их мало.
|
|
||||||
for (const auto & handle : tasks)
|
|
||||||
{
|
|
||||||
time_t next_time_to_execute = handle->next_time_to_execute;
|
|
||||||
|
|
||||||
if (next_time_to_execute < min_time)
|
|
||||||
{
|
|
||||||
min_time = next_time_to_execute;
|
|
||||||
task = handle;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (task) /// Переложим в конец очереди (уменьшим приоритет среди задач с одинаковым next_time_to_execute).
|
|
||||||
tasks.splice(tasks.end(), tasks, task->iterator);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (shutdown)
|
|
||||||
break;
|
|
||||||
|
|
||||||
if (!task)
|
|
||||||
{
|
|
||||||
std::unique_lock<std::mutex> lock(mutex);
|
|
||||||
wake_event.wait_for(lock, std::chrono::duration<double>(sleep_seconds));
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (task->removed)
|
|
||||||
continue;
|
|
||||||
|
|
||||||
/// Лучшей задачи не нашлось, а эта задача в прошлый раз ничего не сделала, и поэтому ей назначено некоторое время спать.
|
|
||||||
time_t current_time = time(0);
|
|
||||||
if (min_time > current_time)
|
|
||||||
{
|
|
||||||
std::unique_lock<std::mutex> lock(mutex);
|
|
||||||
wake_event.wait_for(lock, std::chrono::duration<double>(min_time - current_time));
|
|
||||||
}
|
|
||||||
|
|
||||||
Poco::ScopedReadRWLock rlock(task->rwlock);
|
|
||||||
|
|
||||||
if (task->removed)
|
|
||||||
continue;
|
|
||||||
|
|
||||||
Context context(*this, counters_diff);
|
|
||||||
bool done_work = task->function(context);
|
|
||||||
|
|
||||||
/// Если задача сделала полезную работу, то она сможет выполняться в следующий раз хоть сразу.
|
|
||||||
/// Если нет - добавляем задержку перед повторным исполнением.
|
|
||||||
task->next_time_to_execute = time(0) + (done_work ? 0 : sleep_seconds);
|
|
||||||
}
|
|
||||||
catch (...)
|
|
||||||
{
|
|
||||||
has_exception = true;
|
|
||||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Вычтем все счётчики обратно.
|
|
||||||
if (!counters_diff.empty())
|
|
||||||
{
|
|
||||||
std::unique_lock<std::mutex> lock(mutex);
|
|
||||||
for (const auto & it : counters_diff)
|
|
||||||
counters[it.first] -= it.second;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (shutdown)
|
|
||||||
break;
|
|
||||||
|
|
||||||
if (has_exception)
|
|
||||||
{
|
|
||||||
std::unique_lock<std::mutex> lock(mutex);
|
|
||||||
wake_event.wait_for(lock, std::chrono::duration<double>(sleep_seconds));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
};
|
};
|
||||||
|
|
||||||
typedef Poco::SharedPtr<BackgroundProcessingPool> BackgroundProcessingPoolPtr;
|
typedef Poco::SharedPtr<BackgroundProcessingPool> BackgroundProcessingPoolPtr;
|
||||||
|
@ -131,10 +131,8 @@ public:
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// Контрольная сумма от множества контрольных сумм .bin файлов.
|
/// Контрольная сумма от множества контрольных сумм .bin файлов.
|
||||||
String summaryDataChecksum() const
|
void summaryDataChecksum(SipHash & hash) const
|
||||||
{
|
{
|
||||||
SipHash hash;
|
|
||||||
|
|
||||||
/// Пользуемся тем, что итерирование в детерминированном (лексикографическом) порядке.
|
/// Пользуемся тем, что итерирование в детерминированном (лексикографическом) порядке.
|
||||||
for (const auto & it : files)
|
for (const auto & it : files)
|
||||||
{
|
{
|
||||||
@ -148,10 +146,6 @@ public:
|
|||||||
hash.update(reinterpret_cast<const char *>(&sum.uncompressed_size), sizeof(sum.uncompressed_size));
|
hash.update(reinterpret_cast<const char *>(&sum.uncompressed_size), sizeof(sum.uncompressed_size));
|
||||||
hash.update(reinterpret_cast<const char *>(&sum.uncompressed_hash), sizeof(sum.uncompressed_hash));
|
hash.update(reinterpret_cast<const char *>(&sum.uncompressed_hash), sizeof(sum.uncompressed_hash));
|
||||||
}
|
}
|
||||||
|
|
||||||
UInt64 lo, hi;
|
|
||||||
hash.get128(lo, hi);
|
|
||||||
return DB::toString(lo) + "_" + DB::toString(hi);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
String toString() const
|
String toString() const
|
||||||
|
@ -82,12 +82,24 @@ public:
|
|||||||
MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block, part_number);
|
MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block, part_number);
|
||||||
String part_name = ActiveDataPartSet::getPartName(part->left_date, part->right_date, part->left, part->right, part->level);
|
String part_name = ActiveDataPartSet::getPartName(part->left_date, part->right_date, part->left, part->right, part->level);
|
||||||
|
|
||||||
|
/// Хэш от данных.
|
||||||
|
SipHash hash;
|
||||||
|
part->checksums.summaryDataChecksum(hash);
|
||||||
|
union
|
||||||
|
{
|
||||||
|
char bytes[16];
|
||||||
|
UInt64 lo, hi;
|
||||||
|
} hash_value;
|
||||||
|
hash.get128(hash_value.bytes);
|
||||||
|
|
||||||
|
String checksum(hash_value.bytes, 16);
|
||||||
|
|
||||||
/// Если в запросе не указан ID, возьмем в качестве ID хеш от данных. То есть, не вставляем одинаковые данные дважды.
|
/// Если в запросе не указан ID, возьмем в качестве ID хеш от данных. То есть, не вставляем одинаковые данные дважды.
|
||||||
/// NOTE: Если такая дедупликация не нужна, можно вместо этого оставлять block_id пустым.
|
/// NOTE: Если такая дедупликация не нужна, можно вместо этого оставлять block_id пустым.
|
||||||
/// Можно для этого сделать настройку или синтаксис в запросе (например, ID=null).
|
/// Можно для этого сделать настройку или синтаксис в запросе (например, ID=null).
|
||||||
if (block_id.empty())
|
if (block_id.empty())
|
||||||
{
|
{
|
||||||
block_id = part->checksums.summaryDataChecksum();
|
block_id = toString(hash_value.lo) + "_" + toString(hash_value.hi);
|
||||||
|
|
||||||
if (block_id.empty())
|
if (block_id.empty())
|
||||||
throw Exception("Logical error: block_id is empty.", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Logical error: block_id is empty.", ErrorCodes::LOGICAL_ERROR);
|
||||||
@ -105,30 +117,25 @@ public:
|
|||||||
|
|
||||||
/// Информация о блоке.
|
/// Информация о блоке.
|
||||||
zkutil::Ops ops;
|
zkutil::Ops ops;
|
||||||
|
auto acl = zookeeper->getDefaultACL();
|
||||||
|
|
||||||
ops.push_back(
|
ops.push_back(
|
||||||
new zkutil::Op::Create(
|
new zkutil::Op::Create(
|
||||||
storage.zookeeper_path + "/blocks/" + block_id,
|
storage.zookeeper_path + "/blocks/" + block_id,
|
||||||
"",
|
"",
|
||||||
zookeeper->getDefaultACL(),
|
acl,
|
||||||
zkutil::CreateMode::Persistent));
|
zkutil::CreateMode::Persistent));
|
||||||
ops.push_back(
|
ops.push_back(
|
||||||
new zkutil::Op::Create(
|
new zkutil::Op::Create(
|
||||||
storage.zookeeper_path + "/blocks/" + block_id + "/columns",
|
storage.zookeeper_path + "/blocks/" + block_id + "/checksum",
|
||||||
part->columns.toString(),
|
checksum,
|
||||||
zookeeper->getDefaultACL(),
|
acl,
|
||||||
zkutil::CreateMode::Persistent));
|
|
||||||
ops.push_back(
|
|
||||||
new zkutil::Op::Create(
|
|
||||||
storage.zookeeper_path + "/blocks/" + block_id + "/checksums",
|
|
||||||
part->checksums.toString(),
|
|
||||||
zookeeper->getDefaultACL(),
|
|
||||||
zkutil::CreateMode::Persistent));
|
zkutil::CreateMode::Persistent));
|
||||||
ops.push_back(
|
ops.push_back(
|
||||||
new zkutil::Op::Create(
|
new zkutil::Op::Create(
|
||||||
storage.zookeeper_path + "/blocks/" + block_id + "/number",
|
storage.zookeeper_path + "/blocks/" + block_id + "/number",
|
||||||
toString(part_number),
|
toString(part_number),
|
||||||
zookeeper->getDefaultACL(),
|
acl,
|
||||||
zkutil::CreateMode::Persistent));
|
zkutil::CreateMode::Persistent));
|
||||||
|
|
||||||
/// Информация о куске, в данных реплики.
|
/// Информация о куске, в данных реплики.
|
||||||
@ -138,7 +145,7 @@ public:
|
|||||||
ops.push_back(new zkutil::Op::Create(
|
ops.push_back(new zkutil::Op::Create(
|
||||||
storage.zookeeper_path + "/log/log-",
|
storage.zookeeper_path + "/log/log-",
|
||||||
log_entry.toString(),
|
log_entry.toString(),
|
||||||
zookeeper->getDefaultACL(),
|
acl,
|
||||||
zkutil::CreateMode::PersistentSequential));
|
zkutil::CreateMode::PersistentSequential));
|
||||||
|
|
||||||
/// Удаление информации о том, что номер блока используется для записи.
|
/// Удаление информации о том, что номер блока используется для записи.
|
||||||
@ -174,7 +181,7 @@ public:
|
|||||||
new zkutil::Op::Create(
|
new zkutil::Op::Create(
|
||||||
quorum_status_path,
|
quorum_status_path,
|
||||||
quorum_entry.toString(),
|
quorum_entry.toString(),
|
||||||
zookeeper->getDefaultACL(),
|
acl,
|
||||||
zkutil::CreateMode::Persistent));
|
zkutil::CreateMode::Persistent));
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -192,16 +199,20 @@ public:
|
|||||||
else if (code == ZNODEEXISTS)
|
else if (code == ZNODEEXISTS)
|
||||||
{
|
{
|
||||||
/// Если блок с таким ID уже есть в таблице, откатим его вставку.
|
/// Если блок с таким ID уже есть в таблице, откатим его вставку.
|
||||||
String expected_checksums_str;
|
String expected_checksum;
|
||||||
if (!block_id.empty() && zookeeper->tryGet(
|
if (!block_id.empty() && zookeeper->tryGet(
|
||||||
storage.zookeeper_path + "/blocks/" + block_id + "/checksums", expected_checksums_str))
|
storage.zookeeper_path + "/blocks/" + block_id + "/checksum", expected_checksum))
|
||||||
{
|
{
|
||||||
LOG_INFO(log, "Block with ID " << block_id << " already exists; ignoring it (removing part " << part->name << ")");
|
LOG_INFO(log, "Block with ID " << block_id << " already exists; ignoring it (removing part " << part->name << ")");
|
||||||
|
|
||||||
auto expected_checksums = MergeTreeData::DataPart::Checksums::parse(expected_checksums_str);
|
|
||||||
|
|
||||||
/// Если данные отличались от тех, что были вставлены ранее с тем же ID, бросим исключение.
|
/// Если данные отличались от тех, что были вставлены ранее с тем же ID, бросим исключение.
|
||||||
expected_checksums.checkEqual(part->checksums, true);
|
if (expected_checksum != checksum)
|
||||||
|
{
|
||||||
|
if (!insert_id.empty())
|
||||||
|
throw Exception("Attempt to insert block with same ID but different checksum", ErrorCodes::CHECKSUM_DOESNT_MATCH);
|
||||||
|
else
|
||||||
|
throw Exception("Logical error: got ZNODEEXISTS while inserting data, block ID is derived from checksum but checksum doesn't match", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
|
||||||
transaction.rollback();
|
transaction.rollback();
|
||||||
}
|
}
|
||||||
|
@ -3,6 +3,7 @@
|
|||||||
#include <DB/Core/Exception.h>
|
#include <DB/Core/Exception.h>
|
||||||
#include <DB/Core/ErrorCodes.h>
|
#include <DB/Core/ErrorCodes.h>
|
||||||
#include <DB/Core/Types.h>
|
#include <DB/Core/Types.h>
|
||||||
|
#include <DB/IO/WriteHelpers.h>
|
||||||
|
|
||||||
#include <mutex>
|
#include <mutex>
|
||||||
#include <condition_variable>
|
#include <condition_variable>
|
||||||
@ -19,7 +20,10 @@ class WriteBuffer;
|
|||||||
class StorageReplicatedMergeTree;
|
class StorageReplicatedMergeTree;
|
||||||
|
|
||||||
|
|
||||||
/// Добавляет кусок в множество future_parts; в деструкторе убирает.
|
/** Добавляет кусок в множество future_parts; в деструкторе убирает.
|
||||||
|
* future_parts - множество кусков, которые будут созданы после выполнения
|
||||||
|
* выполняющихся в данный момент элементов очереди.
|
||||||
|
*/
|
||||||
struct FuturePartTagger
|
struct FuturePartTagger
|
||||||
{
|
{
|
||||||
String part;
|
String part;
|
||||||
@ -32,22 +36,34 @@ struct FuturePartTagger
|
|||||||
typedef Poco::SharedPtr<FuturePartTagger> FuturePartTaggerPtr;
|
typedef Poco::SharedPtr<FuturePartTagger> FuturePartTaggerPtr;
|
||||||
|
|
||||||
|
|
||||||
/// Запись о том, что нужно сделать.
|
/// Запись о том, что нужно сделать. Только данные (их можно копировать).
|
||||||
struct ReplicatedMergeTreeLogEntry
|
struct ReplicatedMergeTreeLogEntryData
|
||||||
{
|
{
|
||||||
typedef Poco::SharedPtr<ReplicatedMergeTreeLogEntry> Ptr;
|
|
||||||
|
|
||||||
enum Type
|
enum Type
|
||||||
{
|
{
|
||||||
|
EMPTY, /// Не используется.
|
||||||
GET_PART, /// Получить кусок с другой реплики.
|
GET_PART, /// Получить кусок с другой реплики.
|
||||||
MERGE_PARTS, /// Слить куски.
|
MERGE_PARTS, /// Слить куски.
|
||||||
DROP_RANGE, /// Удалить куски в указанном месяце в указанном диапазоне номеров.
|
DROP_RANGE, /// Удалить куски в указанном месяце в указанном диапазоне номеров.
|
||||||
ATTACH_PART, /// Перенести кусок из директории detached или unreplicated.
|
ATTACH_PART, /// Перенести кусок из директории detached или unreplicated.
|
||||||
};
|
};
|
||||||
|
|
||||||
|
String typeToString() const
|
||||||
|
{
|
||||||
|
switch (type)
|
||||||
|
{
|
||||||
|
case ReplicatedMergeTreeLogEntryData::GET_PART: return "GET_PART";
|
||||||
|
case ReplicatedMergeTreeLogEntryData::MERGE_PARTS: return "MERGE_PARTS";
|
||||||
|
case ReplicatedMergeTreeLogEntryData::DROP_RANGE: return "DROP_RANGE";
|
||||||
|
case ReplicatedMergeTreeLogEntryData::ATTACH_PART: return "ATTACH_PART";
|
||||||
|
default:
|
||||||
|
throw Exception("Unknown log entry type: " + DB::toString(type), ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
String znode_name;
|
String znode_name;
|
||||||
|
|
||||||
Type type;
|
Type type = EMPTY;
|
||||||
String source_replica; /// Пустая строка значит, что эта запись была добавлена сразу в очередь, а не скопирована из лога.
|
String source_replica; /// Пустая строка значит, что эта запись была добавлена сразу в очередь, а не скопирована из лога.
|
||||||
|
|
||||||
/// Имя куска, получающегося в результате.
|
/// Имя куска, получающегося в результате.
|
||||||
@ -62,9 +78,7 @@ struct ReplicatedMergeTreeLogEntry
|
|||||||
/// Для ATTACH_PART имя куска в директории detached или unreplicated.
|
/// Для ATTACH_PART имя куска в директории detached или unreplicated.
|
||||||
String source_part_name;
|
String source_part_name;
|
||||||
/// Нужно переносить из директории unreplicated, а не detached.
|
/// Нужно переносить из директории unreplicated, а не detached.
|
||||||
bool attach_unreplicated;
|
bool attach_unreplicated = false;
|
||||||
|
|
||||||
FuturePartTaggerPtr future_part_tagger;
|
|
||||||
|
|
||||||
/// Доступ под queue_mutex.
|
/// Доступ под queue_mutex.
|
||||||
bool currently_executing = false; /// Выполняется ли действие сейчас.
|
bool currently_executing = false; /// Выполняется ли действие сейчас.
|
||||||
@ -73,16 +87,24 @@ struct ReplicatedMergeTreeLogEntry
|
|||||||
size_t num_tries = 0; /// Количество попыток выполнить действие (с момента старта сервера; включая выполняющееся).
|
size_t num_tries = 0; /// Количество попыток выполнить действие (с момента старта сервера; включая выполняющееся).
|
||||||
ExceptionPtr exception; /// Последний эксепшен, в случае безуспешной попытки выполнить действие.
|
ExceptionPtr exception; /// Последний эксепшен, в случае безуспешной попытки выполнить действие.
|
||||||
time_t last_attempt_time = 0; /// Время начала последней попытки выполнить действие.
|
time_t last_attempt_time = 0; /// Время начала последней попытки выполнить действие.
|
||||||
String last_action; /// Что делается сейчас или делалось в последний раз.
|
size_t num_postponed = 0; /// Количество раз, когда действие было отложено.
|
||||||
String postpone_reason; /// Причина, по которой действие было отложено, если оно отложено.
|
String postpone_reason; /// Причина, по которой действие было отложено, если оно отложено.
|
||||||
|
time_t last_postpone_time = 0; /// Время последнего раза, когда действие было отложено.
|
||||||
std::condition_variable execution_complete; /// Пробуждается когда currently_executing становится false.
|
|
||||||
|
|
||||||
/// Время создания или время копирования из общего лога в очередь конкретной реплики.
|
/// Время создания или время копирования из общего лога в очередь конкретной реплики.
|
||||||
time_t create_time = 0;
|
time_t create_time = 0;
|
||||||
|
|
||||||
/// Величина кворума (для GET_PART) - ненулевое значение при включенной кворумной записи.
|
/// Величина кворума (для GET_PART) - ненулевое значение при включенной кворумной записи.
|
||||||
size_t quorum = 0;
|
size_t quorum = 0;
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
struct ReplicatedMergeTreeLogEntry : ReplicatedMergeTreeLogEntryData
|
||||||
|
{
|
||||||
|
typedef Poco::SharedPtr<ReplicatedMergeTreeLogEntry> Ptr;
|
||||||
|
|
||||||
|
FuturePartTaggerPtr future_part_tagger;
|
||||||
|
std::condition_variable execution_complete; /// Пробуждается когда currently_executing становится false.
|
||||||
|
|
||||||
void addResultToVirtualParts(StorageReplicatedMergeTree & storage);
|
void addResultToVirtualParts(StorageReplicatedMergeTree & storage);
|
||||||
void tagPartAsFuture(StorageReplicatedMergeTree & storage);
|
void tagPartAsFuture(StorageReplicatedMergeTree & storage);
|
||||||
|
@ -177,6 +177,9 @@ public:
|
|||||||
/// Получить статус таблицы. Если with_zk_fields = false - не заполнять поля, требующие запросов в ZK.
|
/// Получить статус таблицы. Если with_zk_fields = false - не заполнять поля, требующие запросов в ZK.
|
||||||
void getStatus(Status & res, bool with_zk_fields = true);
|
void getStatus(Status & res, bool with_zk_fields = true);
|
||||||
|
|
||||||
|
using LogEntriesData = std::vector<ReplicatedMergeTreeLogEntryData>;
|
||||||
|
void getQueue(LogEntriesData & res, String & replica_name);
|
||||||
|
|
||||||
private:
|
private:
|
||||||
void dropUnreplicatedPartition(const Field & partition, bool detach, const Settings & settings);
|
void dropUnreplicatedPartition(const Field & partition, bool detach, const Settings & settings);
|
||||||
|
|
||||||
@ -383,7 +386,7 @@ private:
|
|||||||
/** Можно ли сейчас попробовать выполнить это действие. Если нет, нужно оставить его в очереди и попробовать выполнить другое.
|
/** Можно ли сейчас попробовать выполнить это действие. Если нет, нужно оставить его в очереди и попробовать выполнить другое.
|
||||||
* Вызывается под queue_mutex.
|
* Вызывается под queue_mutex.
|
||||||
*/
|
*/
|
||||||
bool shouldExecuteLogEntry(const LogEntry & entry);
|
bool shouldExecuteLogEntry(const LogEntry & entry, String & out_postpone_reason);
|
||||||
|
|
||||||
/** Выполнить действие из очереди. Бросает исключение, если что-то не так.
|
/** Выполнить действие из очереди. Бросает исключение, если что-то не так.
|
||||||
* Возвращает, получилось ли выполнить. Если не получилось, запись нужно положить в конец очереди.
|
* Возвращает, получилось ли выполнить. Если не получилось, запись нужно положить в конец очереди.
|
||||||
|
@ -0,0 +1,43 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Poco/SharedPtr.h>
|
||||||
|
|
||||||
|
#include <DB/Storages/IStorage.h>
|
||||||
|
#include <DB/Interpreters/Context.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
using Poco::SharedPtr;
|
||||||
|
|
||||||
|
|
||||||
|
/** Реализует системную таблицу replication_queue, которая позволяет посмотреть очереди репликации для реплицируемых таблиц.
|
||||||
|
*/
|
||||||
|
class StorageSystemReplicationQueue : public IStorage
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
static StoragePtr create(const std::string & name_);
|
||||||
|
|
||||||
|
std::string getName() const override { return "SystemReplicationQueue"; }
|
||||||
|
std::string getTableName() const override { return name; }
|
||||||
|
|
||||||
|
const NamesAndTypesList & getColumnsListImpl() const override { return columns; }
|
||||||
|
|
||||||
|
BlockInputStreams read(
|
||||||
|
const Names & column_names,
|
||||||
|
ASTPtr query,
|
||||||
|
const Context & context,
|
||||||
|
const Settings & settings,
|
||||||
|
QueryProcessingStage::Enum & processed_stage,
|
||||||
|
size_t max_block_size = DEFAULT_BLOCK_SIZE,
|
||||||
|
unsigned threads = 1) override;
|
||||||
|
|
||||||
|
private:
|
||||||
|
const std::string name;
|
||||||
|
NamesAndTypesList columns;
|
||||||
|
|
||||||
|
StorageSystemReplicationQueue(const std::string & name_);
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
11
dbms/src/AggregateFunctions/AggregateFunctionArray.cpp
Normal file
11
dbms/src/AggregateFunctions/AggregateFunctionArray.cpp
Normal file
@ -0,0 +1,11 @@
|
|||||||
|
#include <DB/AggregateFunctions/AggregateFunctionArray.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionArray(AggregateFunctionPtr & nested)
|
||||||
|
{
|
||||||
|
return new AggregateFunctionArray(nested);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
31
dbms/src/AggregateFunctions/AggregateFunctionAvg.cpp
Normal file
31
dbms/src/AggregateFunctions/AggregateFunctionAvg.cpp
Normal file
@ -0,0 +1,31 @@
|
|||||||
|
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
|
||||||
|
#include <DB/AggregateFunctions/AggregateFunctionAvg.h>
|
||||||
|
#include <DB/AggregateFunctions/Helpers.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 1)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionAvg>(*argument_types[0]);
|
||||||
|
|
||||||
|
if (!res)
|
||||||
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
void registerAggregateFunctionAvg(AggregateFunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction({"avg"}, createAggregateFunctionAvg);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
22
dbms/src/AggregateFunctions/AggregateFunctionCount.cpp
Normal file
22
dbms/src/AggregateFunctions/AggregateFunctionCount.cpp
Normal file
@ -0,0 +1,22 @@
|
|||||||
|
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
|
||||||
|
#include <DB/AggregateFunctions/AggregateFunctionCount.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionCount(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
return new AggregateFunctionCount;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
void registerAggregateFunctionCount(AggregateFunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction({"count"}, createAggregateFunctionCount);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
File diff suppressed because it is too large
Load Diff
23
dbms/src/AggregateFunctions/AggregateFunctionGroupArray.cpp
Normal file
23
dbms/src/AggregateFunctions/AggregateFunctionGroupArray.cpp
Normal file
@ -0,0 +1,23 @@
|
|||||||
|
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
|
||||||
|
#include <DB/AggregateFunctions/AggregateFunctionGroupArray.h>
|
||||||
|
#include <DB/AggregateFunctions/Helpers.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionGroupArray(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
return new AggregateFunctionGroupArray;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
void registerAggregateFunctionGroupArray(AggregateFunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction({"groupArray"}, createAggregateFunctionGroupArray);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,40 @@
|
|||||||
|
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
|
||||||
|
#include <DB/AggregateFunctions/AggregateFunctionGroupUniqArray.h>
|
||||||
|
#include <DB/AggregateFunctions/Helpers.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionGroupUniqArray(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 1)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name,
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
const DataTypeArray * arr = typeid_cast<const DataTypeArray *>(&*argument_types[0]);
|
||||||
|
|
||||||
|
AggregateFunctionPtr res;
|
||||||
|
|
||||||
|
if (!arr)
|
||||||
|
res = createWithNumericType<AggregateFunctionGroupUniqArray>(*argument_types[0]);
|
||||||
|
else
|
||||||
|
res = createWithNumericType<AggregateFunctionGroupUniqArrays>(*arr->getNestedType());
|
||||||
|
|
||||||
|
if (!res)
|
||||||
|
throw Exception("Illegal type " + argument_types[0]->getName() +
|
||||||
|
" of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
void registerAggregateFunctionGroupUniqArray(AggregateFunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction({"groupUniqArray"}, createAggregateFunctionGroupUniqArray);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
11
dbms/src/AggregateFunctions/AggregateFunctionIf.cpp
Normal file
11
dbms/src/AggregateFunctions/AggregateFunctionIf.cpp
Normal file
@ -0,0 +1,11 @@
|
|||||||
|
#include <DB/AggregateFunctions/AggregateFunctionIf.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionIf(AggregateFunctionPtr & nested)
|
||||||
|
{
|
||||||
|
return new AggregateFunctionIf(nested);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
11
dbms/src/AggregateFunctions/AggregateFunctionMerge.cpp
Normal file
11
dbms/src/AggregateFunctions/AggregateFunctionMerge.cpp
Normal file
@ -0,0 +1,11 @@
|
|||||||
|
#include <DB/AggregateFunctions/AggregateFunctionMerge.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionMerge(AggregateFunctionPtr & nested)
|
||||||
|
{
|
||||||
|
return new AggregateFunctionMerge(nested);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
11
dbms/src/AggregateFunctions/AggregateFunctionState.cpp
Normal file
11
dbms/src/AggregateFunctions/AggregateFunctionState.cpp
Normal file
@ -0,0 +1,11 @@
|
|||||||
|
#include <DB/AggregateFunctions/AggregateFunctionState.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionState(AggregateFunctionPtr & nested)
|
||||||
|
{
|
||||||
|
return new AggregateFunctionState(nested);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
31
dbms/src/AggregateFunctions/AggregateFunctionSum.cpp
Normal file
31
dbms/src/AggregateFunctions/AggregateFunctionSum.cpp
Normal file
@ -0,0 +1,31 @@
|
|||||||
|
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
|
||||||
|
#include <DB/AggregateFunctions/AggregateFunctionSum.h>
|
||||||
|
#include <DB/AggregateFunctions/Helpers.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionSum(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 1)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionSum>(*argument_types[0]);
|
||||||
|
|
||||||
|
if (!res)
|
||||||
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
void registerAggregateFunctionSum(AggregateFunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction({"sum"}, createAggregateFunctionSum);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
39
dbms/src/AggregateFunctions/AggregateFunctionUniqUpTo.cpp
Normal file
39
dbms/src/AggregateFunctions/AggregateFunctionUniqUpTo.cpp
Normal file
@ -0,0 +1,39 @@
|
|||||||
|
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
|
||||||
|
#include <DB/AggregateFunctions/Helpers.h>
|
||||||
|
#include <DB/AggregateFunctions/AggregateFunctionUniqUpTo.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionUniqUpTo(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 1)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
const IDataType & argument_type = *argument_types[0];
|
||||||
|
|
||||||
|
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniqUpTo>(*argument_types[0]);
|
||||||
|
|
||||||
|
if (res)
|
||||||
|
return res;
|
||||||
|
else if (typeid_cast<const DataTypeDate *>(&argument_type))
|
||||||
|
return new AggregateFunctionUniqUpTo<DataTypeDate::FieldType>;
|
||||||
|
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
|
||||||
|
return new AggregateFunctionUniqUpTo<DataTypeDateTime::FieldType>;
|
||||||
|
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
|
||||||
|
return new AggregateFunctionUniqUpTo<String>;
|
||||||
|
else
|
||||||
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
void registerAggregateFunctionUniqUpTo(AggregateFunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction({"uniqUpTo"}, createAggregateFunctionUniqUpTo);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
52
dbms/src/AggregateFunctions/AggregateFunctionsMinMaxAny.cpp
Normal file
52
dbms/src/AggregateFunctions/AggregateFunctionsMinMaxAny.cpp
Normal file
@ -0,0 +1,52 @@
|
|||||||
|
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
|
||||||
|
#include <DB/AggregateFunctions/HelpersMinMaxAny.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionAny(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
return createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionAnyData>(name, argument_types);
|
||||||
|
}
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionAnyLast(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
return createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionAnyLastData>(name, argument_types);
|
||||||
|
}
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionMin(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
return createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionMinData>(name, argument_types);
|
||||||
|
}
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionMax(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
return createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionMaxData>(name, argument_types);
|
||||||
|
}
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionArgMin(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
return createAggregateFunctionArgMinMax<AggregateFunctionMinData>(name, argument_types);
|
||||||
|
}
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionArgMax(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
return createAggregateFunctionArgMinMax<AggregateFunctionMaxData>(name, argument_types);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
void registerAggregateFunctionsMinMaxAny(AggregateFunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction({"any"}, createAggregateFunctionAny);
|
||||||
|
factory.registerFunction({"anyLast"}, createAggregateFunctionAnyLast);
|
||||||
|
factory.registerFunction({"min"}, createAggregateFunctionMin);
|
||||||
|
factory.registerFunction({"max"}, createAggregateFunctionMax);
|
||||||
|
factory.registerFunction({"argMin"}, createAggregateFunctionArgMin);
|
||||||
|
factory.registerFunction({"argMax"}, createAggregateFunctionArgMax);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
66
dbms/src/AggregateFunctions/AggregateFunctionsQuantile.cpp
Normal file
66
dbms/src/AggregateFunctions/AggregateFunctionsQuantile.cpp
Normal file
@ -0,0 +1,66 @@
|
|||||||
|
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
|
||||||
|
#include <DB/AggregateFunctions/Helpers.h>
|
||||||
|
#include <DB/AggregateFunctions/AggregateFunctionQuantile.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionQuantile(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 1)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
const IDataType & argument_type = *argument_types[0];
|
||||||
|
|
||||||
|
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionQuantile<UInt8>;
|
||||||
|
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return new AggregateFunctionQuantile<UInt16>;
|
||||||
|
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return new AggregateFunctionQuantile<UInt32>;
|
||||||
|
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return new AggregateFunctionQuantile<UInt64>;
|
||||||
|
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return new AggregateFunctionQuantile<Int8>;
|
||||||
|
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return new AggregateFunctionQuantile<Int16>;
|
||||||
|
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return new AggregateFunctionQuantile<Int32>;
|
||||||
|
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return new AggregateFunctionQuantile<Int64>;
|
||||||
|
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return new AggregateFunctionQuantile<Float32>;
|
||||||
|
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return new AggregateFunctionQuantile<Float64>;
|
||||||
|
else if (typeid_cast<const DataTypeDate *>(&argument_type)) return new AggregateFunctionQuantile<DataTypeDate::FieldType, false>;
|
||||||
|
else if (typeid_cast<const DataTypeDateTime*>(&argument_type)) return new AggregateFunctionQuantile<DataTypeDateTime::FieldType, false>;
|
||||||
|
else
|
||||||
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionQuantiles(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 1)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
const IDataType & argument_type = *argument_types[0];
|
||||||
|
|
||||||
|
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionQuantiles<UInt8>;
|
||||||
|
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return new AggregateFunctionQuantiles<UInt16>;
|
||||||
|
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return new AggregateFunctionQuantiles<UInt32>;
|
||||||
|
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return new AggregateFunctionQuantiles<UInt64>;
|
||||||
|
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return new AggregateFunctionQuantiles<Int8>;
|
||||||
|
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return new AggregateFunctionQuantiles<Int16>;
|
||||||
|
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return new AggregateFunctionQuantiles<Int32>;
|
||||||
|
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return new AggregateFunctionQuantiles<Int64>;
|
||||||
|
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return new AggregateFunctionQuantiles<Float32>;
|
||||||
|
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return new AggregateFunctionQuantiles<Float64>;
|
||||||
|
else if (typeid_cast<const DataTypeDate *>(&argument_type)) return new AggregateFunctionQuantiles<DataTypeDate::FieldType, false>;
|
||||||
|
else if (typeid_cast<const DataTypeDateTime*>(&argument_type)) return new AggregateFunctionQuantiles<DataTypeDateTime::FieldType, false>;
|
||||||
|
else
|
||||||
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
void registerAggregateFunctionsQuantile(AggregateFunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction({"quantile", "median"}, createAggregateFunctionQuantile);
|
||||||
|
factory.registerFunction({"quantiles"}, createAggregateFunctionQuantiles);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,92 @@
|
|||||||
|
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
|
||||||
|
#include <DB/AggregateFunctions/Helpers.h>
|
||||||
|
#include <DB/AggregateFunctions/AggregateFunctionQuantileDeterministic.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionQuantileDeterministic(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 2)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
const auto determinator_type = argument_types[1].get();
|
||||||
|
if (!typeid_cast<const DataTypeInt32 *>(determinator_type) &&
|
||||||
|
!typeid_cast<const DataTypeUInt32 *>(determinator_type) &&
|
||||||
|
!typeid_cast<const DataTypeInt64 *>(determinator_type) &&
|
||||||
|
!typeid_cast<const DataTypeUInt64 *>(determinator_type))
|
||||||
|
{
|
||||||
|
throw Exception{
|
||||||
|
"Illegal type " + determinator_type->getName() + " of second argument for aggregate function " + name +
|
||||||
|
", Int32, UInt32, Int64 or UInt64 required",
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
const IDataType & argument_type = *argument_types[0];
|
||||||
|
|
||||||
|
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<UInt8>;
|
||||||
|
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<UInt16>;
|
||||||
|
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<UInt32>;
|
||||||
|
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<UInt64>;
|
||||||
|
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<Int8>;
|
||||||
|
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<Int16>;
|
||||||
|
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<Int32>;
|
||||||
|
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<Int64>;
|
||||||
|
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<Float32>;
|
||||||
|
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<Float64>;
|
||||||
|
else if (typeid_cast<const DataTypeDate *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<DataTypeDate::FieldType, false>;
|
||||||
|
else if (typeid_cast<const DataTypeDateTime*>(&argument_type)) return new AggregateFunctionQuantileDeterministic<DataTypeDateTime::FieldType, false>;
|
||||||
|
else
|
||||||
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionQuantilesDeterministic(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 2)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
const auto determinator_type = argument_types[1].get();
|
||||||
|
if (!typeid_cast<const DataTypeInt32 *>(determinator_type) &&
|
||||||
|
!typeid_cast<const DataTypeUInt32 *>(determinator_type) &&
|
||||||
|
!typeid_cast<const DataTypeInt64 *>(determinator_type) &&
|
||||||
|
!typeid_cast<const DataTypeUInt64 *>(determinator_type))
|
||||||
|
{
|
||||||
|
throw Exception{
|
||||||
|
"Illegal type " + determinator_type->getName() + " of second argument for aggregate function " + name +
|
||||||
|
", Int32, UInt32, Int64 or UInt64 required",
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
const IDataType & argument_type = *argument_types[0];
|
||||||
|
|
||||||
|
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<UInt8>;
|
||||||
|
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<UInt16>;
|
||||||
|
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<UInt32>;
|
||||||
|
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<UInt64>;
|
||||||
|
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<Int8>;
|
||||||
|
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<Int16>;
|
||||||
|
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<Int32>;
|
||||||
|
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<Int64>;
|
||||||
|
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<Float32>;
|
||||||
|
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<Float64>;
|
||||||
|
else if (typeid_cast<const DataTypeDate *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<DataTypeDate::FieldType, false>;
|
||||||
|
else if (typeid_cast<const DataTypeDateTime*>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<DataTypeDateTime::FieldType, false>;
|
||||||
|
else
|
||||||
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
void registerAggregateFunctionsQuantileDeterministic(AggregateFunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction({"quantileDeterministic", "medianDeterministic"}, createAggregateFunctionQuantileDeterministic);
|
||||||
|
factory.registerFunction({"quantilesDeterministic"}, createAggregateFunctionQuantilesDeterministic);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,76 @@
|
|||||||
|
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
|
||||||
|
#include <DB/AggregateFunctions/Helpers.h>
|
||||||
|
#include <DB/AggregateFunctions/AggregateFunctionQuantileTiming.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionQuantileTiming(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 1)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionQuantileTiming>(*argument_types[0]);
|
||||||
|
|
||||||
|
if (!res)
|
||||||
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionQuantilesTiming(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 1)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionQuantilesTiming>(*argument_types[0]);
|
||||||
|
|
||||||
|
if (!res)
|
||||||
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionQuantileTimingWeighted(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 2)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
AggregateFunctionPtr res = createWithTwoNumericTypes<AggregateFunctionQuantileTimingWeighted>(*argument_types[0], *argument_types[1]);
|
||||||
|
|
||||||
|
if (!res)
|
||||||
|
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()
|
||||||
|
+ " of arguments for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionQuantilesTimingWeighted(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 2)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
AggregateFunctionPtr res = createWithTwoNumericTypes<AggregateFunctionQuantilesTimingWeighted>(*argument_types[0], *argument_types[1]);
|
||||||
|
|
||||||
|
if (!res)
|
||||||
|
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()
|
||||||
|
+ " of arguments for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
void registerAggregateFunctionsQuantileTiming(AggregateFunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction({"quantileTiming", "medianTiming"}, createAggregateFunctionQuantileTiming);
|
||||||
|
factory.registerFunction({"quantilesTiming"}, createAggregateFunctionQuantilesTiming);
|
||||||
|
factory.registerFunction({"quantileTimingWeighted", "medianTimingWeighted"}, createAggregateFunctionQuantileTimingWeighted);
|
||||||
|
factory.registerFunction({"quantilesTimingWeighted"}, createAggregateFunctionQuantilesTimingWeighted);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,34 @@
|
|||||||
|
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
|
||||||
|
#include <DB/AggregateFunctions/AggregateFunctionSequenceMatch.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionSequenceCount(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (!AggregateFunctionSequenceCount::sufficientArgs(argument_types.size()))
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
return new AggregateFunctionSequenceCount;
|
||||||
|
}
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionSequenceMatch(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (!AggregateFunctionSequenceMatch::sufficientArgs(argument_types.size()))
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
return new AggregateFunctionSequenceMatch;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
void registerAggregateFunctionsSequenceMatch(AggregateFunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction({"sequenceMatch"}, createAggregateFunctionSequenceMatch);
|
||||||
|
factory.registerFunction({"sequenceCount"}, createAggregateFunctionSequenceCount);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
116
dbms/src/AggregateFunctions/AggregateFunctionsStatistics.cpp
Normal file
116
dbms/src/AggregateFunctions/AggregateFunctionsStatistics.cpp
Normal file
@ -0,0 +1,116 @@
|
|||||||
|
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
|
||||||
|
#include <DB/AggregateFunctions/Helpers.h>
|
||||||
|
#include <DB/AggregateFunctions/AggregateFunctionsStatistics.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionVarPop(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 1)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionVarPop>(*argument_types[0]);
|
||||||
|
|
||||||
|
if (!res)
|
||||||
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionVarSamp(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 1)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionVarSamp>(*argument_types[0]);
|
||||||
|
|
||||||
|
if (!res)
|
||||||
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionStdDevPop(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 1)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionStdDevPop>(*argument_types[0]);
|
||||||
|
|
||||||
|
if (!res)
|
||||||
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionStdDevSamp(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 1)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionStdDevSamp>(*argument_types[0]);
|
||||||
|
|
||||||
|
if (!res)
|
||||||
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionCovarPop(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 2)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
AggregateFunctionPtr res = createWithTwoNumericTypes<AggregateFunctionCovarPop>(*argument_types[0], *argument_types[1]);
|
||||||
|
if (!res)
|
||||||
|
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()
|
||||||
|
+ " of arguments for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionCovarSamp(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 2)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
AggregateFunctionPtr res = createWithTwoNumericTypes<AggregateFunctionCovarSamp>(*argument_types[0], *argument_types[1]);
|
||||||
|
if (!res)
|
||||||
|
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()
|
||||||
|
+ " of arguments for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionCorr(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 2)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
AggregateFunctionPtr res = createWithTwoNumericTypes<AggregateFunctionCorr>(*argument_types[0], *argument_types[1]);
|
||||||
|
if (!res)
|
||||||
|
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()
|
||||||
|
+ " of arguments for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
void registerAggregateFunctionsStatistics(AggregateFunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction({"varSamp"}, createAggregateFunctionVarSamp);
|
||||||
|
factory.registerFunction({"varPop"}, createAggregateFunctionVarPop);
|
||||||
|
factory.registerFunction({"stddevSamp"}, createAggregateFunctionStdDevSamp);
|
||||||
|
factory.registerFunction({"stddevPop"}, createAggregateFunctionStdDevPop);
|
||||||
|
factory.registerFunction({"covarSamp"}, createAggregateFunctionCovarSamp);
|
||||||
|
factory.registerFunction({"covarPop"}, createAggregateFunctionCovarPop);
|
||||||
|
factory.registerFunction({"corr"}, createAggregateFunctionCorr);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
191
dbms/src/AggregateFunctions/AggregateFunctionsUniq.cpp
Normal file
191
dbms/src/AggregateFunctions/AggregateFunctionsUniq.cpp
Normal file
@ -0,0 +1,191 @@
|
|||||||
|
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
|
||||||
|
#include <DB/AggregateFunctions/AggregateFunctionUniq.h>
|
||||||
|
#include <DB/AggregateFunctions/Helpers.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 1)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name,
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
const IDataType & argument_type = *argument_types[0];
|
||||||
|
|
||||||
|
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq, AggregateFunctionUniqUniquesHashSetData>(*argument_types[0]);
|
||||||
|
|
||||||
|
if (res)
|
||||||
|
return res;
|
||||||
|
else if (typeid_cast<const DataTypeDate *>(&argument_type))
|
||||||
|
return new AggregateFunctionUniq<DataTypeDate::FieldType, AggregateFunctionUniqUniquesHashSetData>;
|
||||||
|
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
|
||||||
|
return new AggregateFunctionUniq<DataTypeDateTime::FieldType, AggregateFunctionUniqUniquesHashSetData>;
|
||||||
|
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
|
||||||
|
return new AggregateFunctionUniq<String, AggregateFunctionUniqUniquesHashSetData>;
|
||||||
|
else
|
||||||
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
}
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionUniqExact(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 1)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name,
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
const IDataType & argument_type = *argument_types[0];
|
||||||
|
|
||||||
|
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq, AggregateFunctionUniqUniquesHashSetData>(*argument_types[0]);
|
||||||
|
|
||||||
|
if (res)
|
||||||
|
return res;
|
||||||
|
else if (typeid_cast<const DataTypeDate *>(&argument_type))
|
||||||
|
return new AggregateFunctionUniq<DataTypeDate::FieldType, AggregateFunctionUniqUniquesHashSetData>;
|
||||||
|
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
|
||||||
|
return new AggregateFunctionUniq<DataTypeDateTime::FieldType, AggregateFunctionUniqUniquesHashSetData>;
|
||||||
|
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
|
||||||
|
return new AggregateFunctionUniq<String, AggregateFunctionUniqUniquesHashSetData>;
|
||||||
|
else
|
||||||
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
}
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionUniqHLL12(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 1)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name,
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
const IDataType & argument_type = *argument_types[0];
|
||||||
|
|
||||||
|
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq, AggregateFunctionUniqHLL12Data>(*argument_types[0]);
|
||||||
|
|
||||||
|
if (res)
|
||||||
|
return res;
|
||||||
|
else if (typeid_cast<const DataTypeDate *>(&argument_type))
|
||||||
|
return new AggregateFunctionUniq<DataTypeDate::FieldType, AggregateFunctionUniqHLL12Data<DataTypeDate::FieldType>>;
|
||||||
|
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
|
||||||
|
return new AggregateFunctionUniq<DataTypeDateTime::FieldType, AggregateFunctionUniqHLL12Data<DataTypeDateTime::FieldType>>;
|
||||||
|
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
|
||||||
|
return new AggregateFunctionUniq<String, AggregateFunctionUniqHLL12Data<String>>;
|
||||||
|
else
|
||||||
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
}
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionUniqCombined(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 1)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name,
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
const IDataType & argument_type = *argument_types[0];
|
||||||
|
|
||||||
|
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq, AggregateFunctionUniqCombinedData>(*argument_types[0]);
|
||||||
|
|
||||||
|
if (res)
|
||||||
|
return res;
|
||||||
|
else if (typeid_cast<const DataTypeDate *>(&argument_type))
|
||||||
|
return new AggregateFunctionUniq<DataTypeDate::FieldType, AggregateFunctionUniqCombinedData<DataTypeDate::FieldType>>;
|
||||||
|
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
|
||||||
|
return new AggregateFunctionUniq<DataTypeDateTime::FieldType, AggregateFunctionUniqCombinedData<DataTypeDateTime::FieldType>>;
|
||||||
|
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
|
||||||
|
return new AggregateFunctionUniq<String, AggregateFunctionUniqCombinedData<String>>;
|
||||||
|
else
|
||||||
|
throw Exception("Illegal type " + argument_types[0]->getName() +
|
||||||
|
" of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
}
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionUniqCombinedRaw(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 1)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name,
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
const IDataType & argument_type = *argument_types[0];
|
||||||
|
|
||||||
|
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq, AggregateFunctionUniqCombinedRawData>(*argument_types[0]);
|
||||||
|
|
||||||
|
if (res)
|
||||||
|
return res;
|
||||||
|
else if (typeid_cast<const DataTypeDate *>(&argument_type))
|
||||||
|
return new AggregateFunctionUniq<DataTypeDate::FieldType, AggregateFunctionUniqCombinedRawData<DataTypeDate::FieldType>>;
|
||||||
|
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
|
||||||
|
return new AggregateFunctionUniq<DataTypeDateTime::FieldType, AggregateFunctionUniqCombinedRawData<DataTypeDateTime::FieldType>>;
|
||||||
|
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
|
||||||
|
return new AggregateFunctionUniq<String, AggregateFunctionUniqCombinedRawData<String>>;
|
||||||
|
else
|
||||||
|
throw Exception("Illegal type " + argument_types[0]->getName() +
|
||||||
|
" of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
}
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionUniqCombinedLinearCounting(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 1)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name,
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
const IDataType & argument_type = *argument_types[0];
|
||||||
|
|
||||||
|
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq,
|
||||||
|
AggregateFunctionUniqCombinedLinearCountingData>(*argument_types[0]);
|
||||||
|
|
||||||
|
if (res)
|
||||||
|
return res;
|
||||||
|
else if (typeid_cast<const DataTypeDate *>(&argument_type))
|
||||||
|
return new AggregateFunctionUniq<DataTypeDate::FieldType,
|
||||||
|
AggregateFunctionUniqCombinedLinearCountingData<DataTypeDate::FieldType>>;
|
||||||
|
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
|
||||||
|
return new AggregateFunctionUniq<DataTypeDateTime::FieldType,
|
||||||
|
AggregateFunctionUniqCombinedLinearCountingData<DataTypeDateTime::FieldType>>;
|
||||||
|
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
|
||||||
|
return new AggregateFunctionUniq<String, AggregateFunctionUniqCombinedLinearCountingData<String>>;
|
||||||
|
else
|
||||||
|
throw Exception("Illegal type " + argument_types[0]->getName() +
|
||||||
|
" of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
}
|
||||||
|
|
||||||
|
AggregateFunctionPtr createAggregateFunctionUniqCombinedBiasCorrected(const std::string & name, const DataTypes & argument_types)
|
||||||
|
{
|
||||||
|
if (argument_types.size() != 1)
|
||||||
|
throw Exception("Incorrect number of arguments for aggregate function " + name,
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
const IDataType & argument_type = *argument_types[0];
|
||||||
|
|
||||||
|
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq,
|
||||||
|
AggregateFunctionUniqCombinedBiasCorrectedData>(*argument_types[0]);
|
||||||
|
|
||||||
|
if (res)
|
||||||
|
return res;
|
||||||
|
else if (typeid_cast<const DataTypeDate *>(&argument_type))
|
||||||
|
return new AggregateFunctionUniq<DataTypeDate::FieldType,
|
||||||
|
AggregateFunctionUniqCombinedBiasCorrectedData<DataTypeDate::FieldType>>;
|
||||||
|
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
|
||||||
|
return new AggregateFunctionUniq<DataTypeDateTime::FieldType,
|
||||||
|
AggregateFunctionUniqCombinedBiasCorrectedData<DataTypeDateTime::FieldType>>;
|
||||||
|
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
|
||||||
|
return new AggregateFunctionUniq<String, AggregateFunctionUniqCombinedBiasCorrectedData<String>>;
|
||||||
|
else
|
||||||
|
throw Exception("Illegal type " + argument_types[0]->getName() +
|
||||||
|
" of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
void registerAggregateFunctionsUniq(AggregateFunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction({"uniq"}, createAggregateFunctionUniq);
|
||||||
|
factory.registerFunction({"uniqHLL12"}, createAggregateFunctionUniqHLL12);
|
||||||
|
factory.registerFunction({"uniqExact"}, createAggregateFunctionUniqExact);
|
||||||
|
factory.registerFunction({"uniqCombinedRaw"}, createAggregateFunctionUniqCombinedRaw);
|
||||||
|
factory.registerFunction({"uniqCombinedLinearCounting"}, createAggregateFunctionUniqCombinedLinearCounting);
|
||||||
|
factory.registerFunction({"uniqCombinedBiasCorrected"}, createAggregateFunctionUniqCombinedBiasCorrected);
|
||||||
|
factory.registerFunction({"uniqCombined"}, createAggregateFunctionUniqCombined);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
10
dbms/src/Common/setThreadName.cpp
Normal file
10
dbms/src/Common/setThreadName.cpp
Normal file
@ -0,0 +1,10 @@
|
|||||||
|
#include <sys/prctl.h>
|
||||||
|
#include <DB/Core/Exception.h>
|
||||||
|
#include <DB/Common/setThreadName.h>
|
||||||
|
|
||||||
|
|
||||||
|
void setThreadName(const char * name)
|
||||||
|
{
|
||||||
|
if (0 != prctl(PR_SET_NAME, name, 0, 0, 0))
|
||||||
|
DB::throwFromErrno("Cannot set thread name with prctl(PR_SET_NAME...)");
|
||||||
|
}
|
@ -6,7 +6,7 @@
|
|||||||
|
|
||||||
#include <DB/IO/WriteBufferFromOStream.h>
|
#include <DB/IO/WriteBufferFromOStream.h>
|
||||||
|
|
||||||
#include <DB/Storages/StorageSystemNumbers.h>
|
#include <DB/Storages/System/StorageSystemNumbers.h>
|
||||||
|
|
||||||
#include <DB/DataStreams/LimitBlockInputStream.h>
|
#include <DB/DataStreams/LimitBlockInputStream.h>
|
||||||
#include <DB/DataStreams/ExpressionBlockInputStream.h>
|
#include <DB/DataStreams/ExpressionBlockInputStream.h>
|
||||||
|
@ -6,7 +6,7 @@
|
|||||||
|
|
||||||
#include <DB/IO/WriteBufferFromOStream.h>
|
#include <DB/IO/WriteBufferFromOStream.h>
|
||||||
|
|
||||||
#include <DB/Storages/StorageSystemNumbers.h>
|
#include <DB/Storages/System/StorageSystemNumbers.h>
|
||||||
|
|
||||||
#include <DB/DataStreams/LimitBlockInputStream.h>
|
#include <DB/DataStreams/LimitBlockInputStream.h>
|
||||||
#include <DB/DataStreams/ExpressionBlockInputStream.h>
|
#include <DB/DataStreams/ExpressionBlockInputStream.h>
|
||||||
|
@ -7,7 +7,7 @@
|
|||||||
|
|
||||||
#include <DB/IO/WriteBufferFromOStream.h>
|
#include <DB/IO/WriteBufferFromOStream.h>
|
||||||
|
|
||||||
#include <DB/Storages/StorageSystemNumbers.h>
|
#include <DB/Storages/System/StorageSystemNumbers.h>
|
||||||
|
|
||||||
#include <DB/DataStreams/LimitBlockInputStream.h>
|
#include <DB/DataStreams/LimitBlockInputStream.h>
|
||||||
#include <DB/DataStreams/ExpressionBlockInputStream.h>
|
#include <DB/DataStreams/ExpressionBlockInputStream.h>
|
||||||
|
@ -6,7 +6,7 @@
|
|||||||
|
|
||||||
#include <DB/IO/WriteBufferFromFileDescriptor.h>
|
#include <DB/IO/WriteBufferFromFileDescriptor.h>
|
||||||
|
|
||||||
#include <DB/Storages/StorageSystemNumbers.h>
|
#include <DB/Storages/System/StorageSystemNumbers.h>
|
||||||
|
|
||||||
#include <DB/DataStreams/LimitBlockInputStream.h>
|
#include <DB/DataStreams/LimitBlockInputStream.h>
|
||||||
#include <DB/DataStreams/UnionBlockInputStream.h>
|
#include <DB/DataStreams/UnionBlockInputStream.h>
|
||||||
|
@ -6,7 +6,7 @@
|
|||||||
|
|
||||||
#include <DB/IO/WriteBufferFromFileDescriptor.h>
|
#include <DB/IO/WriteBufferFromFileDescriptor.h>
|
||||||
|
|
||||||
#include <DB/Storages/StorageSystemNumbers.h>
|
#include <DB/Storages/System/StorageSystemNumbers.h>
|
||||||
|
|
||||||
#include <DB/DataStreams/LimitBlockInputStream.h>
|
#include <DB/DataStreams/LimitBlockInputStream.h>
|
||||||
#include <DB/DataStreams/UnionBlockInputStream.h>
|
#include <DB/DataStreams/UnionBlockInputStream.h>
|
||||||
@ -32,7 +32,7 @@ int main(int argc, char ** argv)
|
|||||||
context.setPath("./");
|
context.setPath("./");
|
||||||
|
|
||||||
DB::loadMetadata(context);
|
DB::loadMetadata(context);
|
||||||
|
|
||||||
DB::Names column_names;
|
DB::Names column_names;
|
||||||
column_names.push_back("WatchID");
|
column_names.push_back("WatchID");
|
||||||
|
|
||||||
@ -43,7 +43,7 @@ int main(int argc, char ** argv)
|
|||||||
|
|
||||||
for (size_t i = 0, size = streams.size(); i < size; ++i)
|
for (size_t i = 0, size = streams.size(); i < size; ++i)
|
||||||
streams[i] = new DB::AsynchronousBlockInputStream(streams[i]);
|
streams[i] = new DB::AsynchronousBlockInputStream(streams[i]);
|
||||||
|
|
||||||
DB::BlockInputStreamPtr stream = new DB::UnionBlockInputStream(streams, nullptr, settings.max_threads);
|
DB::BlockInputStreamPtr stream = new DB::UnionBlockInputStream(streams, nullptr, settings.max_threads);
|
||||||
stream = new DB::LimitBlockInputStream(stream, 10, 0);
|
stream = new DB::LimitBlockInputStream(stream, 10, 0);
|
||||||
|
|
||||||
|
@ -13,8 +13,7 @@ void registerFunctionsStringSearch(FunctionFactory & factory)
|
|||||||
factory.registerFunction<FunctionPosition>();
|
factory.registerFunction<FunctionPosition>();
|
||||||
factory.registerFunction<FunctionPositionUTF8>();
|
factory.registerFunction<FunctionPositionUTF8>();
|
||||||
factory.registerFunction<FunctionPositionCaseInsensitive>();
|
factory.registerFunction<FunctionPositionCaseInsensitive>();
|
||||||
/// @todo implement
|
factory.registerFunction<FunctionPositionCaseInsensitiveUTF8>();
|
||||||
// factory.registerFunction<FunctionPositionCaseInsensitiveUTF8>();
|
|
||||||
factory.registerFunction<FunctionMatch>();
|
factory.registerFunction<FunctionMatch>();
|
||||||
factory.registerFunction<FunctionLike>();
|
factory.registerFunction<FunctionLike>();
|
||||||
factory.registerFunction<FunctionNotLike>();
|
factory.registerFunction<FunctionNotLike>();
|
||||||
|
@ -826,6 +826,11 @@ std::pair<String, UInt16> Context::getInterserverIOAddress() const
|
|||||||
return { shared->interserver_io_host, shared->interserver_io_port };
|
return { shared->interserver_io_host, shared->interserver_io_port };
|
||||||
}
|
}
|
||||||
|
|
||||||
|
UInt16 Context::getTCPPort() const
|
||||||
|
{
|
||||||
|
return Poco::Util::Application::instance().config().getInt("tcp_port");
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
void Context::initClusters()
|
void Context::initClusters()
|
||||||
{
|
{
|
||||||
|
@ -18,7 +18,7 @@
|
|||||||
|
|
||||||
#include <DB/Storages/StorageFactory.h>
|
#include <DB/Storages/StorageFactory.h>
|
||||||
#include <DB/Storages/StorageLog.h>
|
#include <DB/Storages/StorageLog.h>
|
||||||
#include <DB/Storages/StorageSystemNumbers.h>
|
#include <DB/Storages/System/StorageSystemNumbers.h>
|
||||||
|
|
||||||
#include <DB/Parsers/formatAST.h>
|
#include <DB/Parsers/formatAST.h>
|
||||||
#include <DB/Parsers/ASTIdentifier.h>
|
#include <DB/Parsers/ASTIdentifier.h>
|
||||||
|
@ -15,6 +15,7 @@
|
|||||||
#include <DB/Interpreters/InterpreterCreateQuery.h>
|
#include <DB/Interpreters/InterpreterCreateQuery.h>
|
||||||
#include <DB/Interpreters/InterpreterRenameQuery.h>
|
#include <DB/Interpreters/InterpreterRenameQuery.h>
|
||||||
#include <DB/Interpreters/QueryLog.h>
|
#include <DB/Interpreters/QueryLog.h>
|
||||||
|
#include <DB/Common/setThreadName.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -116,6 +117,8 @@ QueryLog::~QueryLog()
|
|||||||
|
|
||||||
void QueryLog::threadFunction()
|
void QueryLog::threadFunction()
|
||||||
{
|
{
|
||||||
|
setThreadName("QueryLogFlush");
|
||||||
|
|
||||||
Stopwatch time_after_last_write;
|
Stopwatch time_after_last_write;
|
||||||
bool first = true;
|
bool first = true;
|
||||||
|
|
||||||
|
@ -190,7 +190,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
|||||||
|
|
||||||
setClientInfo(elem, context);
|
setClientInfo(elem, context);
|
||||||
|
|
||||||
bool log_queries = settings.log_queries;
|
bool log_queries = settings.log_queries && !internal;
|
||||||
|
|
||||||
/// Логгируем в таблицу начало выполнения запроса, если нужно.
|
/// Логгируем в таблицу начало выполнения запроса, если нужно.
|
||||||
if (log_queries)
|
if (log_queries)
|
||||||
|
@ -13,6 +13,7 @@
|
|||||||
#include <DB/IO/ReadBufferFromFile.h>
|
#include <DB/IO/ReadBufferFromFile.h>
|
||||||
#include <DB/IO/WriteBufferFromString.h>
|
#include <DB/IO/WriteBufferFromString.h>
|
||||||
#include <DB/IO/copyData.h>
|
#include <DB/IO/copyData.h>
|
||||||
|
#include <DB/Common/escapeForFileName.h>
|
||||||
|
|
||||||
#include <statdaemons/Stopwatch.h>
|
#include <statdaemons/Stopwatch.h>
|
||||||
|
|
||||||
@ -75,9 +76,10 @@ void loadMetadata(Context & context)
|
|||||||
if (it.name().at(0) == '.')
|
if (it.name().at(0) == '.')
|
||||||
continue;
|
continue;
|
||||||
|
|
||||||
LOG_INFO(log, "Loading database " << it.name());
|
String database = unescapeForFileName(it.name());
|
||||||
|
|
||||||
executeCreateQuery("ATTACH DATABASE " + it.name(), context, it.name(), it->path());
|
LOG_INFO(log, "Loading database " << database);
|
||||||
|
executeCreateQuery("ATTACH DATABASE " + backQuoteIfNeed(database), context, database, it->path());
|
||||||
|
|
||||||
/// Цикл по таблицам
|
/// Цикл по таблицам
|
||||||
typedef std::vector<std::string> Tables;
|
typedef std::vector<std::string> Tables;
|
||||||
@ -135,7 +137,7 @@ void loadMetadata(Context & context)
|
|||||||
|
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
executeCreateQuery(s, context, it.name(), tables[j]);
|
executeCreateQuery(s, context, database, tables[j]);
|
||||||
}
|
}
|
||||||
catch (const Exception & e)
|
catch (const Exception & e)
|
||||||
{
|
{
|
||||||
|
@ -9,8 +9,8 @@
|
|||||||
#include <DB/IO/WriteBufferFromOStream.h>
|
#include <DB/IO/WriteBufferFromOStream.h>
|
||||||
|
|
||||||
#include <DB/Storages/StorageLog.h>
|
#include <DB/Storages/StorageLog.h>
|
||||||
#include <DB/Storages/StorageSystemNumbers.h>
|
#include <DB/Storages/System/StorageSystemNumbers.h>
|
||||||
#include <DB/Storages/StorageSystemOne.h>
|
#include <DB/Storages/System/StorageSystemOne.h>
|
||||||
#include <DB/Storages/StorageFactory.h>
|
#include <DB/Storages/StorageFactory.h>
|
||||||
|
|
||||||
#include <DB/Interpreters/loadMetadata.h>
|
#include <DB/Interpreters/loadMetadata.h>
|
||||||
|
@ -19,23 +19,25 @@
|
|||||||
|
|
||||||
#include <DB/Common/Macros.h>
|
#include <DB/Common/Macros.h>
|
||||||
#include <DB/Common/getFQDNOrHostName.h>
|
#include <DB/Common/getFQDNOrHostName.h>
|
||||||
|
#include <DB/Common/setThreadName.h>
|
||||||
#include <DB/Interpreters/loadMetadata.h>
|
#include <DB/Interpreters/loadMetadata.h>
|
||||||
#include <DB/Interpreters/ProcessList.h>
|
#include <DB/Interpreters/ProcessList.h>
|
||||||
#include <DB/Storages/StorageSystemNumbers.h>
|
#include <DB/Storages/System/StorageSystemNumbers.h>
|
||||||
#include <DB/Storages/StorageSystemTables.h>
|
#include <DB/Storages/System/StorageSystemTables.h>
|
||||||
#include <DB/Storages/StorageSystemParts.h>
|
#include <DB/Storages/System/StorageSystemParts.h>
|
||||||
#include <DB/Storages/StorageSystemDatabases.h>
|
#include <DB/Storages/System/StorageSystemDatabases.h>
|
||||||
#include <DB/Storages/StorageSystemProcesses.h>
|
#include <DB/Storages/System/StorageSystemProcesses.h>
|
||||||
#include <DB/Storages/StorageSystemEvents.h>
|
#include <DB/Storages/System/StorageSystemEvents.h>
|
||||||
#include <DB/Storages/StorageSystemOne.h>
|
#include <DB/Storages/System/StorageSystemOne.h>
|
||||||
#include <DB/Storages/StorageSystemMerges.h>
|
#include <DB/Storages/System/StorageSystemMerges.h>
|
||||||
#include <DB/Storages/StorageSystemSettings.h>
|
#include <DB/Storages/System/StorageSystemSettings.h>
|
||||||
#include <DB/Storages/StorageSystemZooKeeper.h>
|
#include <DB/Storages/System/StorageSystemZooKeeper.h>
|
||||||
#include <DB/Storages/StorageSystemReplicas.h>
|
#include <DB/Storages/System/StorageSystemReplicas.h>
|
||||||
#include <DB/Storages/StorageSystemDictionaries.h>
|
#include <DB/Storages/System/StorageSystemReplicationQueue.h>
|
||||||
#include <DB/Storages/StorageSystemColumns.h>
|
#include <DB/Storages/System/StorageSystemDictionaries.h>
|
||||||
#include <DB/Storages/StorageSystemFunctions.h>
|
#include <DB/Storages/System/StorageSystemColumns.h>
|
||||||
#include <DB/Storages/StorageSystemClusters.h>
|
#include <DB/Storages/System/StorageSystemFunctions.h>
|
||||||
|
#include <DB/Storages/System/StorageSystemClusters.h>
|
||||||
|
|
||||||
#include <DB/IO/copyData.h>
|
#include <DB/IO/copyData.h>
|
||||||
#include <DB/IO/LimitReadBuffer.h>
|
#include <DB/IO/LimitReadBuffer.h>
|
||||||
@ -81,6 +83,8 @@ public:
|
|||||||
private:
|
private:
|
||||||
void run()
|
void run()
|
||||||
{
|
{
|
||||||
|
setThreadName("ProfileEventsTx");
|
||||||
|
|
||||||
const auto get_next_minute = [] {
|
const auto get_next_minute = [] {
|
||||||
return std::chrono::time_point_cast<std::chrono::minutes, std::chrono::system_clock>(
|
return std::chrono::time_point_cast<std::chrono::minutes, std::chrono::system_clock>(
|
||||||
std::chrono::system_clock::now() + std::chrono::minutes(1)
|
std::chrono::system_clock::now() + std::chrono::minutes(1)
|
||||||
@ -259,6 +263,8 @@ UsersConfigReloader::~UsersConfigReloader()
|
|||||||
|
|
||||||
void UsersConfigReloader::run()
|
void UsersConfigReloader::run()
|
||||||
{
|
{
|
||||||
|
setThreadName("UserConfReload");
|
||||||
|
|
||||||
while (!quit)
|
while (!quit)
|
||||||
{
|
{
|
||||||
std::this_thread::sleep_for(std::chrono::seconds(2));
|
std::this_thread::sleep_for(std::chrono::seconds(2));
|
||||||
@ -548,6 +554,7 @@ int Server::main(const std::vector<std::string> & args)
|
|||||||
global_context->addTable("system", "events", StorageSystemEvents::create("events"));
|
global_context->addTable("system", "events", StorageSystemEvents::create("events"));
|
||||||
global_context->addTable("system", "merges", StorageSystemMerges::create("merges"));
|
global_context->addTable("system", "merges", StorageSystemMerges::create("merges"));
|
||||||
global_context->addTable("system", "replicas", StorageSystemReplicas::create("replicas"));
|
global_context->addTable("system", "replicas", StorageSystemReplicas::create("replicas"));
|
||||||
|
global_context->addTable("system", "replication_queue", StorageSystemReplicationQueue::create("replication_queue"));
|
||||||
global_context->addTable("system", "dictionaries", StorageSystemDictionaries::create("dictionaries"));
|
global_context->addTable("system", "dictionaries", StorageSystemDictionaries::create("dictionaries"));
|
||||||
global_context->addTable("system", "columns", StorageSystemColumns::create("columns"));
|
global_context->addTable("system", "columns", StorageSystemColumns::create("columns"));
|
||||||
global_context->addTable("system", "functions", StorageSystemFunctions::create("functions"));
|
global_context->addTable("system", "functions", StorageSystemFunctions::create("functions"));
|
||||||
|
196
dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp
Normal file
196
dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp
Normal file
@ -0,0 +1,196 @@
|
|||||||
|
#include <DB/Core/Exception.h>
|
||||||
|
#include <DB/Core/ErrorCodes.h>
|
||||||
|
#include <DB/Common/setThreadName.h>
|
||||||
|
#include <DB/IO/WriteHelpers.h>
|
||||||
|
#include <Yandex/logger_useful.h>
|
||||||
|
#include <DB/Storages/MergeTree/BackgroundProcessingPool.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
|
||||||
|
constexpr double BackgroundProcessingPool::sleep_seconds;
|
||||||
|
constexpr double BackgroundProcessingPool::sleep_seconds_random_part;
|
||||||
|
|
||||||
|
|
||||||
|
void BackgroundProcessingPool::TaskInfo::wake()
|
||||||
|
{
|
||||||
|
if (removed)
|
||||||
|
return;
|
||||||
|
|
||||||
|
std::unique_lock<std::mutex> lock(pool.mutex);
|
||||||
|
pool.tasks.splice(pool.tasks.begin(), pool.tasks, iterator);
|
||||||
|
|
||||||
|
/// Если эта задача в прошлый раз ничего не сделала, и ей было назначено спать, то отменим время сна.
|
||||||
|
time_t current_time = time(0);
|
||||||
|
if (next_time_to_execute > current_time)
|
||||||
|
next_time_to_execute = current_time;
|
||||||
|
|
||||||
|
/// Если все потоки сейчас выполняют работу, этот вызов никого не разбудит.
|
||||||
|
pool.wake_event.notify_one();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
BackgroundProcessingPool::BackgroundProcessingPool(int size_) : size(size_)
|
||||||
|
{
|
||||||
|
LOG_INFO(&Logger::get("BackgroundProcessingPool"), "Create BackgroundProcessingPool with " << size << " threads");
|
||||||
|
|
||||||
|
threads.resize(size);
|
||||||
|
for (auto & thread : threads)
|
||||||
|
thread = std::thread([this] { threadFunction(); });
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
int BackgroundProcessingPool::getCounter(const String & name)
|
||||||
|
{
|
||||||
|
std::unique_lock<std::mutex> lock(mutex);
|
||||||
|
return counters[name];
|
||||||
|
}
|
||||||
|
|
||||||
|
BackgroundProcessingPool::TaskHandle BackgroundProcessingPool::addTask(const Task & task)
|
||||||
|
{
|
||||||
|
TaskHandle res(new TaskInfo(*this, task));
|
||||||
|
|
||||||
|
{
|
||||||
|
std::unique_lock<std::mutex> lock(mutex);
|
||||||
|
res->iterator = tasks.insert(tasks.begin(), res);
|
||||||
|
}
|
||||||
|
|
||||||
|
wake_event.notify_all();
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
void BackgroundProcessingPool::removeTask(const TaskHandle & task)
|
||||||
|
{
|
||||||
|
task->removed = true;
|
||||||
|
|
||||||
|
/// Дождёмся завершения всех выполнений этой задачи.
|
||||||
|
{
|
||||||
|
Poco::ScopedWriteRWLock wlock(task->rwlock);
|
||||||
|
}
|
||||||
|
|
||||||
|
{
|
||||||
|
std::unique_lock<std::mutex> lock(mutex);
|
||||||
|
tasks.erase(task->iterator);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
BackgroundProcessingPool::~BackgroundProcessingPool()
|
||||||
|
{
|
||||||
|
try
|
||||||
|
{
|
||||||
|
shutdown = true;
|
||||||
|
wake_event.notify_all();
|
||||||
|
for (std::thread & thread : threads)
|
||||||
|
thread.join();
|
||||||
|
}
|
||||||
|
catch (...)
|
||||||
|
{
|
||||||
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void BackgroundProcessingPool::threadFunction()
|
||||||
|
{
|
||||||
|
setThreadName("BackgrProcPool");
|
||||||
|
|
||||||
|
std::mt19937 rng(reinterpret_cast<intptr_t>(&rng));
|
||||||
|
std::this_thread::sleep_for(std::chrono::duration<double>(std::uniform_real_distribution<double>(0, sleep_seconds_random_part)(rng)));
|
||||||
|
|
||||||
|
while (!shutdown)
|
||||||
|
{
|
||||||
|
Counters counters_diff;
|
||||||
|
bool has_exception = false;
|
||||||
|
|
||||||
|
try
|
||||||
|
{
|
||||||
|
TaskHandle task;
|
||||||
|
time_t min_time = std::numeric_limits<time_t>::max();
|
||||||
|
|
||||||
|
{
|
||||||
|
std::unique_lock<std::mutex> lock(mutex);
|
||||||
|
|
||||||
|
if (!tasks.empty())
|
||||||
|
{
|
||||||
|
/// O(n), n - число задач. По сути, количество таблиц. Обычно их мало.
|
||||||
|
for (const auto & handle : tasks)
|
||||||
|
{
|
||||||
|
time_t next_time_to_execute = handle->next_time_to_execute;
|
||||||
|
|
||||||
|
if (next_time_to_execute < min_time)
|
||||||
|
{
|
||||||
|
min_time = next_time_to_execute;
|
||||||
|
task = handle;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (task) /// Переложим в конец очереди (уменьшим приоритет среди задач с одинаковым next_time_to_execute).
|
||||||
|
tasks.splice(tasks.end(), tasks, task->iterator);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (shutdown)
|
||||||
|
break;
|
||||||
|
|
||||||
|
if (!task)
|
||||||
|
{
|
||||||
|
std::unique_lock<std::mutex> lock(mutex);
|
||||||
|
wake_event.wait_for(lock,
|
||||||
|
std::chrono::duration<double>(sleep_seconds
|
||||||
|
+ std::uniform_real_distribution<double>(0, sleep_seconds_random_part)(rng)));
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (task->removed)
|
||||||
|
continue;
|
||||||
|
|
||||||
|
/// Лучшей задачи не нашлось, а эта задача в прошлый раз ничего не сделала, и поэтому ей назначено некоторое время спать.
|
||||||
|
time_t current_time = time(0);
|
||||||
|
if (min_time > current_time)
|
||||||
|
{
|
||||||
|
std::unique_lock<std::mutex> lock(mutex);
|
||||||
|
wake_event.wait_for(lock, std::chrono::duration<double>(
|
||||||
|
min_time - current_time + std::uniform_real_distribution<double>(0, sleep_seconds_random_part)(rng)));
|
||||||
|
}
|
||||||
|
|
||||||
|
Poco::ScopedReadRWLock rlock(task->rwlock);
|
||||||
|
|
||||||
|
if (task->removed)
|
||||||
|
continue;
|
||||||
|
|
||||||
|
Context context(*this, counters_diff);
|
||||||
|
bool done_work = task->function(context);
|
||||||
|
|
||||||
|
/// Если задача сделала полезную работу, то она сможет выполняться в следующий раз хоть сразу.
|
||||||
|
/// Если нет - добавляем задержку перед повторным исполнением.
|
||||||
|
task->next_time_to_execute = time(0) + (done_work ? 0 : sleep_seconds);
|
||||||
|
}
|
||||||
|
catch (...)
|
||||||
|
{
|
||||||
|
has_exception = true;
|
||||||
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Вычтем все счётчики обратно.
|
||||||
|
if (!counters_diff.empty())
|
||||||
|
{
|
||||||
|
std::unique_lock<std::mutex> lock(mutex);
|
||||||
|
for (const auto & it : counters_diff)
|
||||||
|
counters[it.first] -= it.second;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (shutdown)
|
||||||
|
break;
|
||||||
|
|
||||||
|
if (has_exception)
|
||||||
|
{
|
||||||
|
std::unique_lock<std::mutex> lock(mutex);
|
||||||
|
wake_event.wait_for(lock, std::chrono::duration<double>(sleep_seconds));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -1,5 +1,6 @@
|
|||||||
#include <DB/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h>
|
#include <DB/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h>
|
||||||
#include <DB/Storages/StorageReplicatedMergeTree.h>
|
#include <DB/Storages/StorageReplicatedMergeTree.h>
|
||||||
|
#include <DB/Common/setThreadName.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -13,6 +14,8 @@ ReplicatedMergeTreeCleanupThread::ReplicatedMergeTreeCleanupThread(StorageReplic
|
|||||||
|
|
||||||
void ReplicatedMergeTreeCleanupThread::run()
|
void ReplicatedMergeTreeCleanupThread::run()
|
||||||
{
|
{
|
||||||
|
setThreadName("ReplMTCleanup");
|
||||||
|
|
||||||
const auto CLEANUP_SLEEP_MS = 30 * 1000;
|
const auto CLEANUP_SLEEP_MS = 30 * 1000;
|
||||||
|
|
||||||
while (!storage.shutdown_called)
|
while (!storage.shutdown_called)
|
||||||
@ -176,9 +179,12 @@ void ReplicatedMergeTreeCleanupThread::clearOldBlocks()
|
|||||||
std::sort(timed_blocks.begin(), timed_blocks.end(), std::greater<std::pair<Int64, String>>());
|
std::sort(timed_blocks.begin(), timed_blocks.end(), std::greater<std::pair<Int64, String>>());
|
||||||
for (size_t i = storage.data.settings.replicated_deduplication_window; i < timed_blocks.size(); ++i)
|
for (size_t i = storage.data.settings.replicated_deduplication_window; i < timed_blocks.size(); ++i)
|
||||||
{
|
{
|
||||||
|
/// Устаревшие ноды. Этот код можно будет убрать через пол года.
|
||||||
|
zookeeper->tryRemove(storage.zookeeper_path + "/blocks/" + timed_blocks[i].second + "/columns");
|
||||||
|
zookeeper->tryRemove(storage.zookeeper_path + "/blocks/" + timed_blocks[i].second + "/checksums");
|
||||||
|
|
||||||
ops.push_back(new zkutil::Op::Remove(storage.zookeeper_path + "/blocks/" + timed_blocks[i].second + "/number", -1));
|
ops.push_back(new zkutil::Op::Remove(storage.zookeeper_path + "/blocks/" + timed_blocks[i].second + "/number", -1));
|
||||||
ops.push_back(new zkutil::Op::Remove(storage.zookeeper_path + "/blocks/" + timed_blocks[i].second + "/columns", -1));
|
ops.push_back(new zkutil::Op::Remove(storage.zookeeper_path + "/blocks/" + timed_blocks[i].second + "/checksum", -1));
|
||||||
ops.push_back(new zkutil::Op::Remove(storage.zookeeper_path + "/blocks/" + timed_blocks[i].second + "/checksums", -1));
|
|
||||||
ops.push_back(new zkutil::Op::Remove(storage.zookeeper_path + "/blocks/" + timed_blocks[i].second, -1));
|
ops.push_back(new zkutil::Op::Remove(storage.zookeeper_path + "/blocks/" + timed_blocks[i].second, -1));
|
||||||
|
|
||||||
if (ops.size() > 400 || i + 1 == timed_blocks.size())
|
if (ops.size() > 400 || i + 1 == timed_blocks.size())
|
||||||
|
@ -78,6 +78,9 @@ void ReplicatedMergeTreeLogEntry::writeText(WriteBuffer & out) const
|
|||||||
out << "detached\n";
|
out << "detached\n";
|
||||||
out << source_part_name << "\ninto\n" << new_part_name;
|
out << source_part_name << "\ninto\n" << new_part_name;
|
||||||
break;
|
break;
|
||||||
|
|
||||||
|
default:
|
||||||
|
throw Exception("Unknown log entry type: " + DB::toString(type), ErrorCodes::LOGICAL_ERROR);
|
||||||
}
|
}
|
||||||
|
|
||||||
out << '\n';
|
out << '\n';
|
||||||
|
@ -2,6 +2,7 @@
|
|||||||
#include <DB/Storages/StorageReplicatedMergeTree.h>
|
#include <DB/Storages/StorageReplicatedMergeTree.h>
|
||||||
#include <DB/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h>
|
#include <DB/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h>
|
||||||
#include <DB/Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h>
|
#include <DB/Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h>
|
||||||
|
#include <DB/Common/setThreadName.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -32,6 +33,8 @@ void ReplicatedMergeTreeRestartingThread::run()
|
|||||||
constexpr auto retry_delay_ms = 10 * 1000;
|
constexpr auto retry_delay_ms = 10 * 1000;
|
||||||
constexpr auto check_delay_ms = 60 * 1000;
|
constexpr auto check_delay_ms = 60 * 1000;
|
||||||
|
|
||||||
|
setThreadName("ReplMTRestart");
|
||||||
|
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
bool first_time = true;
|
bool first_time = true;
|
||||||
@ -226,7 +229,8 @@ void ReplicatedMergeTreeRestartingThread::activateReplica()
|
|||||||
WriteBufferFromString address_buf(address);
|
WriteBufferFromString address_buf(address);
|
||||||
address_buf
|
address_buf
|
||||||
<< "host: " << host_port.first << '\n'
|
<< "host: " << host_port.first << '\n'
|
||||||
<< "port: " << host_port.second << '\n';
|
<< "port: " << host_port.second << '\n'
|
||||||
|
<< "tcp_port: " << storage.context.getTCPPort() << '\n';
|
||||||
}
|
}
|
||||||
|
|
||||||
String is_active_path = storage.replica_path + "/is_active";
|
String is_active_path = storage.replica_path + "/is_active";
|
||||||
|
@ -4,6 +4,7 @@
|
|||||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||||
#include <DB/Storages/StorageBuffer.h>
|
#include <DB/Storages/StorageBuffer.h>
|
||||||
#include <DB/Parsers/ASTInsertQuery.h>
|
#include <DB/Parsers/ASTInsertQuery.h>
|
||||||
|
#include <DB/Common/setThreadName.h>
|
||||||
#include <Poco/Ext/ThreadNumber.h>
|
#include <Poco/Ext/ThreadNumber.h>
|
||||||
|
|
||||||
#include <statdaemons/ext/range.hpp>
|
#include <statdaemons/ext/range.hpp>
|
||||||
@ -449,6 +450,8 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl
|
|||||||
|
|
||||||
void StorageBuffer::flushThread()
|
void StorageBuffer::flushThread()
|
||||||
{
|
{
|
||||||
|
setThreadName("BufferFlush");
|
||||||
|
|
||||||
do
|
do
|
||||||
{
|
{
|
||||||
try
|
try
|
||||||
|
@ -17,8 +17,8 @@
|
|||||||
#include <DB/Storages/StorageMerge.h>
|
#include <DB/Storages/StorageMerge.h>
|
||||||
#include <DB/Storages/StorageMergeTree.h>
|
#include <DB/Storages/StorageMergeTree.h>
|
||||||
#include <DB/Storages/StorageDistributed.h>
|
#include <DB/Storages/StorageDistributed.h>
|
||||||
#include <DB/Storages/StorageSystemNumbers.h>
|
#include <DB/Storages/System/StorageSystemNumbers.h>
|
||||||
#include <DB/Storages/StorageSystemOne.h>
|
#include <DB/Storages/System/StorageSystemOne.h>
|
||||||
#include <DB/Storages/StorageFactory.h>
|
#include <DB/Storages/StorageFactory.h>
|
||||||
#include <DB/Storages/StorageView.h>
|
#include <DB/Storages/StorageView.h>
|
||||||
#include <DB/Storages/StorageMaterializedView.h>
|
#include <DB/Storages/StorageMaterializedView.h>
|
||||||
|
@ -255,6 +255,7 @@ bool StorageMergeTree::mergeTask(BackgroundProcessingPool::Context & background_
|
|||||||
{
|
{
|
||||||
if (shutdown_called)
|
if (shutdown_called)
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
size_t aio_threshold = context.getSettings().min_bytes_to_use_direct_io;
|
size_t aio_threshold = context.getSettings().min_bytes_to_use_direct_io;
|
||||||
|
@ -10,12 +10,14 @@
|
|||||||
#include <DB/Parsers/formatAST.h>
|
#include <DB/Parsers/formatAST.h>
|
||||||
#include <DB/IO/WriteBufferFromOStream.h>
|
#include <DB/IO/WriteBufferFromOStream.h>
|
||||||
#include <DB/IO/ReadBufferFromString.h>
|
#include <DB/IO/ReadBufferFromString.h>
|
||||||
|
#include <DB/IO/Operators.h>
|
||||||
#include <DB/Interpreters/InterpreterAlterQuery.h>
|
#include <DB/Interpreters/InterpreterAlterQuery.h>
|
||||||
#include <DB/Common/VirtualColumnUtils.h>
|
#include <DB/Common/VirtualColumnUtils.h>
|
||||||
#include <DB/Parsers/ASTInsertQuery.h>
|
#include <DB/Parsers/ASTInsertQuery.h>
|
||||||
#include <DB/DataStreams/AddingConstColumnBlockInputStream.h>
|
#include <DB/DataStreams/AddingConstColumnBlockInputStream.h>
|
||||||
#include <DB/Common/Macros.h>
|
#include <DB/Common/Macros.h>
|
||||||
#include <DB/Common/formatReadable.h>
|
#include <DB/Common/formatReadable.h>
|
||||||
|
#include <DB/Common/setThreadName.h>
|
||||||
#include <Poco/DirectoryIterator.h>
|
#include <Poco/DirectoryIterator.h>
|
||||||
#include <time.h>
|
#include <time.h>
|
||||||
|
|
||||||
@ -141,6 +143,9 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
|
|||||||
|
|
||||||
if (!attach)
|
if (!attach)
|
||||||
{
|
{
|
||||||
|
if (!data.getDataParts().empty())
|
||||||
|
throw Exception("Data directory for table already containing data parts - probably it was unclean DROP table or manual intervention. You must either clear directory by hand or use ATTACH TABLE instead of CREATE TABLE if you need to use that parts.", ErrorCodes::INCORRECT_DATA);
|
||||||
|
|
||||||
createTableIfNotExists();
|
createTableIfNotExists();
|
||||||
|
|
||||||
checkTableStructure(false, false);
|
checkTableStructure(false, false);
|
||||||
@ -240,38 +245,43 @@ void StorageReplicatedMergeTree::createTableIfNotExists()
|
|||||||
zookeeper->createAncestors(zookeeper_path);
|
zookeeper->createAncestors(zookeeper_path);
|
||||||
|
|
||||||
/// Запишем метаданные таблицы, чтобы реплики могли сверять с ними параметры таблицы.
|
/// Запишем метаданные таблицы, чтобы реплики могли сверять с ними параметры таблицы.
|
||||||
std::stringstream metadata;
|
std::string metadata;
|
||||||
metadata << "metadata format version: 1" << std::endl;
|
{
|
||||||
metadata << "date column: " << data.date_column_name << std::endl;
|
WriteBufferFromString out(metadata);
|
||||||
metadata << "sampling expression: " << formattedAST(data.sampling_expression) << std::endl;
|
out << "metadata format version: 1" << "\n"
|
||||||
metadata << "index granularity: " << data.index_granularity << std::endl;
|
<< "date column: " << data.date_column_name << "\n"
|
||||||
metadata << "mode: " << static_cast<int>(data.mode) << std::endl;
|
<< "sampling expression: " << formattedAST(data.sampling_expression) << "\n"
|
||||||
metadata << "sign column: " << data.sign_column << std::endl;
|
<< "index granularity: " << data.index_granularity << "\n"
|
||||||
metadata << "primary key: " << formattedAST(data.primary_expr_ast) << std::endl;
|
<< "mode: " << static_cast<int>(data.mode) << "\n"
|
||||||
|
<< "sign column: " << data.sign_column << "\n"
|
||||||
|
<< "primary key: " << formattedAST(data.primary_expr_ast) << "\n";
|
||||||
|
}
|
||||||
|
|
||||||
|
auto acl = zookeeper->getDefaultACL();
|
||||||
|
|
||||||
zkutil::Ops ops;
|
zkutil::Ops ops;
|
||||||
ops.push_back(new zkutil::Op::Create(zookeeper_path, "",
|
ops.push_back(new zkutil::Op::Create(zookeeper_path, "",
|
||||||
zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
|
acl, zkutil::CreateMode::Persistent));
|
||||||
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/metadata", metadata.str(),
|
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/metadata", metadata,
|
||||||
zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
|
acl, zkutil::CreateMode::Persistent));
|
||||||
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/columns", ColumnsDescription<false>{
|
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/columns", ColumnsDescription<false>{
|
||||||
data.getColumnsListNonMaterialized(), data.materialized_columns,
|
data.getColumnsListNonMaterialized(), data.materialized_columns,
|
||||||
data.alias_columns, data.column_defaults}.toString(),
|
data.alias_columns, data.column_defaults}.toString(),
|
||||||
zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
|
acl, zkutil::CreateMode::Persistent));
|
||||||
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/log", "",
|
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/log", "",
|
||||||
zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
|
acl, zkutil::CreateMode::Persistent));
|
||||||
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/blocks", "",
|
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/blocks", "",
|
||||||
zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
|
acl, zkutil::CreateMode::Persistent));
|
||||||
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/block_numbers", "",
|
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/block_numbers", "",
|
||||||
zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
|
acl, zkutil::CreateMode::Persistent));
|
||||||
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/nonincrement_block_numbers", "",
|
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/nonincrement_block_numbers", "",
|
||||||
zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
|
acl, zkutil::CreateMode::Persistent));
|
||||||
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/leader_election", "",
|
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/leader_election", "",
|
||||||
zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
|
acl, zkutil::CreateMode::Persistent));
|
||||||
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/temp", "",
|
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/temp", "",
|
||||||
zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
|
acl, zkutil::CreateMode::Persistent));
|
||||||
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/replicas", "",
|
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/replicas", "",
|
||||||
zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
|
acl, zkutil::CreateMode::Persistent));
|
||||||
|
|
||||||
auto code = zookeeper->tryMulti(ops);
|
auto code = zookeeper->tryMulti(ops);
|
||||||
if (code != ZOK && code != ZNODEEXISTS)
|
if (code != ZOK && code != ZNODEEXISTS)
|
||||||
@ -370,13 +380,14 @@ void StorageReplicatedMergeTree::createReplica()
|
|||||||
LOG_DEBUG(log, "Creating replica " << replica_path);
|
LOG_DEBUG(log, "Creating replica " << replica_path);
|
||||||
|
|
||||||
/// Создадим пустую реплику. Ноду columns создадим в конце - будем использовать ее в качестве признака, что создание реплики завершено.
|
/// Создадим пустую реплику. Ноду columns создадим в конце - будем использовать ее в качестве признака, что создание реплики завершено.
|
||||||
|
auto acl = zookeeper->getDefaultACL();
|
||||||
zkutil::Ops ops;
|
zkutil::Ops ops;
|
||||||
ops.push_back(new zkutil::Op::Create(replica_path, "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
|
ops.push_back(new zkutil::Op::Create(replica_path, "", acl, zkutil::CreateMode::Persistent));
|
||||||
ops.push_back(new zkutil::Op::Create(replica_path + "/host", "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
|
ops.push_back(new zkutil::Op::Create(replica_path + "/host", "", acl, zkutil::CreateMode::Persistent));
|
||||||
ops.push_back(new zkutil::Op::Create(replica_path + "/log_pointer", "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
|
ops.push_back(new zkutil::Op::Create(replica_path + "/log_pointer", "", acl, zkutil::CreateMode::Persistent));
|
||||||
ops.push_back(new zkutil::Op::Create(replica_path + "/queue", "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
|
ops.push_back(new zkutil::Op::Create(replica_path + "/queue", "", acl, zkutil::CreateMode::Persistent));
|
||||||
ops.push_back(new zkutil::Op::Create(replica_path + "/parts", "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
|
ops.push_back(new zkutil::Op::Create(replica_path + "/parts", "", acl, zkutil::CreateMode::Persistent));
|
||||||
ops.push_back(new zkutil::Op::Create(replica_path + "/flags", "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
|
ops.push_back(new zkutil::Op::Create(replica_path + "/flags", "", acl, zkutil::CreateMode::Persistent));
|
||||||
|
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
@ -554,36 +565,48 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
|
|||||||
for (const String & name : parts_to_fetch)
|
for (const String & name : parts_to_fetch)
|
||||||
expected_parts.erase(name);
|
expected_parts.erase(name);
|
||||||
|
|
||||||
String sanity_report =
|
/** Для проверки адекватности, для кусков, которые есть в ФС, но нет в ZK, будем учитывать только не самые новые куски.
|
||||||
"There are " + toString(unexpected_parts.size()) + " unexpected parts, "
|
* Потому что неожиданные новые куски обычно возникают лишь оттого, что они не успели записаться в ZK при грубом перезапуске сервера.
|
||||||
+ toString(parts_to_add.size()) + " unexpectedly merged parts, "
|
* Также это возникает от дедуплицированных кусков, которые не успели удалиться.
|
||||||
+ toString(expected_parts.size()) + " missing obsolete parts, "
|
*/
|
||||||
+ toString(parts_to_fetch.size()) + " missing parts";
|
size_t unexpected_parts_nonnew = 0;
|
||||||
|
for (const auto & part : unexpected_parts)
|
||||||
|
if (part->level > 0 || part->right < RESERVED_BLOCK_NUMBERS)
|
||||||
|
++unexpected_parts_nonnew;
|
||||||
|
|
||||||
|
String sanity_report = "There are "
|
||||||
|
+ toString(unexpected_parts.size()) + " unexpected parts ("
|
||||||
|
+ toString(unexpected_parts_nonnew) + " of them is not just-written), "
|
||||||
|
+ toString(parts_to_add.size()) + " unexpectedly merged parts, "
|
||||||
|
+ toString(expected_parts.size()) + " missing obsolete parts, "
|
||||||
|
+ toString(parts_to_fetch.size()) + " missing parts";
|
||||||
|
|
||||||
/** Можно автоматически синхронизировать данные,
|
/** Можно автоматически синхронизировать данные,
|
||||||
* если количество ошибок каждого из четырёх типов не больше соответствующих порогов,
|
* если количество ошибок каждого из четырёх типов не больше соответствующих порогов,
|
||||||
* или если отношение общего количества ошибок к общему количеству кусков (минимальному - в локальной файловой системе или в ZK)
|
* или если отношение общего количества ошибок к общему количеству кусков (минимальному - в локальной файловой системе или в ZK)
|
||||||
* не больше некоторого отношения (например 5%).
|
* не больше некоторого отношения (например 5%).
|
||||||
|
*
|
||||||
|
* Большое количество несовпадений в данных на файловой системе и ожидаемых данных
|
||||||
|
* может свидетельствовать об ошибке конфигурации (сервер случайно подключили как реплику не от того шарда).
|
||||||
|
* В этом случае, защитный механизм не даёт стартовать серверу.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
size_t min_parts_local_or_expected = std::min(expected_parts_vec.size(), parts.size());
|
size_t min_parts_local_or_expected = std::min(expected_parts_vec.size(), parts.size());
|
||||||
|
size_t total_difference = parts_to_add.size() + unexpected_parts_nonnew + expected_parts.size() + parts_to_fetch.size();
|
||||||
|
|
||||||
bool insane =
|
bool insane =
|
||||||
(parts_to_add.size() > data.settings.replicated_max_unexpectedly_merged_parts
|
(parts_to_add.size() > data.settings.replicated_max_unexpectedly_merged_parts
|
||||||
|| unexpected_parts.size() > data.settings.replicated_max_unexpected_parts
|
|| unexpected_parts_nonnew > data.settings.replicated_max_unexpected_parts
|
||||||
|| expected_parts.size() > data.settings.replicated_max_missing_obsolete_parts
|
|| expected_parts.size() > data.settings.replicated_max_missing_obsolete_parts
|
||||||
|| parts_to_fetch.size() > data.settings.replicated_max_missing_active_parts)
|
|| parts_to_fetch.size() > data.settings.replicated_max_missing_active_parts)
|
||||||
&& ((parts_to_add.size() + unexpected_parts.size() + expected_parts.size() + parts_to_fetch.size())
|
&& (total_difference > min_parts_local_or_expected * data.settings.replicated_max_ratio_of_wrong_parts);
|
||||||
> min_parts_local_or_expected * data.settings.replicated_max_ratio_of_wrong_parts);
|
|
||||||
|
|
||||||
if (insane)
|
if (insane && !skip_sanity_checks)
|
||||||
{
|
throw Exception("The local set of parts of table " + getTableName() + " doesn't look like the set of parts in ZooKeeper. "
|
||||||
if (skip_sanity_checks)
|
+ sanity_report, ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS);
|
||||||
LOG_WARNING(log, sanity_report);
|
|
||||||
else
|
if (total_difference > 0)
|
||||||
throw Exception("The local set of parts of table " + getTableName() + " doesn't look like the set of parts in ZooKeeper. "
|
LOG_WARNING(log, sanity_report);
|
||||||
+ sanity_report, ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS);
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Добавим в ZK информацию о кусках, покрывающих недостающие куски.
|
/// Добавим в ZK информацию о кусках, покрывающих недостающие куски.
|
||||||
for (const MergeTreeData::DataPartPtr & part : parts_to_add)
|
for (const MergeTreeData::DataPartPtr & part : parts_to_add)
|
||||||
@ -689,23 +712,25 @@ void StorageReplicatedMergeTree::checkPartAndAddToZooKeeper(const MergeTreeData:
|
|||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
auto acl = zookeeper->getDefaultACL();
|
||||||
|
|
||||||
ops.push_back(new zkutil::Op::Check(
|
ops.push_back(new zkutil::Op::Check(
|
||||||
zookeeper_path + "/columns",
|
zookeeper_path + "/columns",
|
||||||
expected_columns_version));
|
expected_columns_version));
|
||||||
ops.push_back(new zkutil::Op::Create(
|
ops.push_back(new zkutil::Op::Create(
|
||||||
replica_path + "/parts/" + part_name,
|
replica_path + "/parts/" + part_name,
|
||||||
"",
|
"",
|
||||||
zookeeper->getDefaultACL(),
|
acl,
|
||||||
zkutil::CreateMode::Persistent));
|
zkutil::CreateMode::Persistent));
|
||||||
ops.push_back(new zkutil::Op::Create(
|
ops.push_back(new zkutil::Op::Create(
|
||||||
replica_path + "/parts/" + part_name + "/columns",
|
replica_path + "/parts/" + part_name + "/columns",
|
||||||
part->columns.toString(),
|
part->columns.toString(),
|
||||||
zookeeper->getDefaultACL(),
|
acl,
|
||||||
zkutil::CreateMode::Persistent));
|
zkutil::CreateMode::Persistent));
|
||||||
ops.push_back(new zkutil::Op::Create(
|
ops.push_back(new zkutil::Op::Create(
|
||||||
replica_path + "/parts/" + part_name + "/checksums",
|
replica_path + "/parts/" + part_name + "/checksums",
|
||||||
part->checksums.toString(),
|
part->checksums.toString(),
|
||||||
zookeeper->getDefaultACL(),
|
acl,
|
||||||
zkutil::CreateMode::Persistent));
|
zkutil::CreateMode::Persistent));
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -796,7 +821,7 @@ void StorageReplicatedMergeTree::pullLogsToQueue(zkutil::EventPtr next_update_ev
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
bool StorageReplicatedMergeTree::shouldExecuteLogEntry(const LogEntry & entry)
|
bool StorageReplicatedMergeTree::shouldExecuteLogEntry(const LogEntry & entry, String & out_postpone_reason)
|
||||||
{
|
{
|
||||||
/// queue_mutex уже захвачен. Функция вызывается только из queueTask.
|
/// queue_mutex уже захвачен. Функция вызывается только из queueTask.
|
||||||
|
|
||||||
@ -805,8 +830,10 @@ bool StorageReplicatedMergeTree::shouldExecuteLogEntry(const LogEntry & entry)
|
|||||||
/// Проверим, не создаётся ли сейчас этот же кусок другим действием.
|
/// Проверим, не создаётся ли сейчас этот же кусок другим действием.
|
||||||
if (future_parts.count(entry.new_part_name))
|
if (future_parts.count(entry.new_part_name))
|
||||||
{
|
{
|
||||||
LOG_DEBUG(log, "Not executing log entry for part " << entry.new_part_name
|
String reason = "Not executing log entry for part " + entry.new_part_name
|
||||||
<< " because another log entry for the same part is being processed. This shouldn't happen often.");
|
+ " because another log entry for the same part is being processed. This shouldn't happen often.";
|
||||||
|
LOG_DEBUG(log, reason);
|
||||||
|
out_postpone_reason = reason;
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
/** Когда соответствующее действие завершится, то shouldExecuteLogEntry, в следующий раз, пройдёт успешно,
|
/** Когда соответствующее действие завершится, то shouldExecuteLogEntry, в следующий раз, пройдёт успешно,
|
||||||
@ -828,8 +855,10 @@ bool StorageReplicatedMergeTree::shouldExecuteLogEntry(const LogEntry & entry)
|
|||||||
|
|
||||||
if (future_part.contains(result_part))
|
if (future_part.contains(result_part))
|
||||||
{
|
{
|
||||||
LOG_DEBUG(log, "Not executing log entry for part " << entry.new_part_name
|
String reason = "Not executing log entry for part " + entry.new_part_name
|
||||||
<< " because another log entry for covering part " << future_part_name << " is being processed.");
|
+ " because another log entry for covering part " + future_part_name + " is being processed.";
|
||||||
|
LOG_DEBUG(log, reason);
|
||||||
|
out_postpone_reason = reason;
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -846,15 +875,19 @@ bool StorageReplicatedMergeTree::shouldExecuteLogEntry(const LogEntry & entry)
|
|||||||
{
|
{
|
||||||
if (future_parts.count(name))
|
if (future_parts.count(name))
|
||||||
{
|
{
|
||||||
LOG_TRACE(log, "Not merging into part " << entry.new_part_name
|
String reason = "Not merging into part " + entry.new_part_name
|
||||||
<< " because part " << name << " is not ready yet (log entry for that part is being processed).");
|
+ " because part " + name + " is not ready yet (log entry for that part is being processed).";
|
||||||
|
LOG_TRACE(log, reason);
|
||||||
|
out_postpone_reason = reason;
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (merger.isCancelled())
|
if (merger.isCancelled())
|
||||||
{
|
{
|
||||||
LOG_DEBUG(log, "Not executing log entry for part " << entry.new_part_name << " because merges are cancelled now.");
|
String reason = "Not executing log entry for part " + entry.new_part_name + " because merges are cancelled now.";
|
||||||
|
LOG_DEBUG(log, reason);
|
||||||
|
out_postpone_reason = reason;
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -1091,16 +1124,18 @@ bool StorageReplicatedMergeTree::executeLogEntry(const LogEntry & entry, Backgro
|
|||||||
throw Exception("Logical error: log entry with quorum for part covering more than one block number",
|
throw Exception("Logical error: log entry with quorum for part covering more than one block number",
|
||||||
ErrorCodes::LOGICAL_ERROR);
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
auto acl = zookeeper->getDefaultACL();
|
||||||
|
|
||||||
ops.push_back(new zkutil::Op::Create(
|
ops.push_back(new zkutil::Op::Create(
|
||||||
zookeeper_path + "/nonincrement_block_numbers/" + partition_str + "/block-" + padIndex(part_info.left),
|
zookeeper_path + "/nonincrement_block_numbers/" + partition_str + "/block-" + padIndex(part_info.left),
|
||||||
"",
|
"",
|
||||||
zookeeper->getDefaultACL(),
|
acl,
|
||||||
zkutil::CreateMode::Persistent));
|
zkutil::CreateMode::Persistent));
|
||||||
|
|
||||||
ops.push_back(new zkutil::Op::Create(
|
ops.push_back(new zkutil::Op::Create(
|
||||||
zookeeper_path + "/quorum/failed_parts/" + entry.new_part_name,
|
zookeeper_path + "/quorum/failed_parts/" + entry.new_part_name,
|
||||||
"",
|
"",
|
||||||
zookeeper->getDefaultACL(),
|
acl,
|
||||||
zkutil::CreateMode::Persistent));
|
zkutil::CreateMode::Persistent));
|
||||||
|
|
||||||
auto code = zookeeper->tryMulti(ops);
|
auto code = zookeeper->tryMulti(ops);
|
||||||
@ -1330,6 +1365,8 @@ bool StorageReplicatedMergeTree::executeAttachPart(const StorageReplicatedMergeT
|
|||||||
|
|
||||||
void StorageReplicatedMergeTree::queueUpdatingThread()
|
void StorageReplicatedMergeTree::queueUpdatingThread()
|
||||||
{
|
{
|
||||||
|
setThreadName("ReplMTQueueUpd");
|
||||||
|
|
||||||
while (!shutdown_called)
|
while (!shutdown_called)
|
||||||
{
|
{
|
||||||
try
|
try
|
||||||
@ -1370,7 +1407,10 @@ bool StorageReplicatedMergeTree::queueTask(BackgroundProcessingPool::Context & p
|
|||||||
{
|
{
|
||||||
for (LogEntries::iterator it = queue.begin(); it != queue.end(); ++it)
|
for (LogEntries::iterator it = queue.begin(); it != queue.end(); ++it)
|
||||||
{
|
{
|
||||||
if (!(*it)->currently_executing && shouldExecuteLogEntry(**it))
|
if ((*it)->currently_executing)
|
||||||
|
continue;
|
||||||
|
|
||||||
|
if (shouldExecuteLogEntry(**it, (*it)->postpone_reason))
|
||||||
{
|
{
|
||||||
entry = *it;
|
entry = *it;
|
||||||
entry->tagPartAsFuture(*this);
|
entry->tagPartAsFuture(*this);
|
||||||
@ -1380,6 +1420,11 @@ bool StorageReplicatedMergeTree::queueTask(BackgroundProcessingPool::Context & p
|
|||||||
entry->last_attempt_time = time(0);
|
entry->last_attempt_time = time(0);
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
++entry->num_postponed;
|
||||||
|
entry->last_postpone_time = time(0);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -1469,6 +1514,8 @@ bool StorageReplicatedMergeTree::queueTask(BackgroundProcessingPool::Context & p
|
|||||||
|
|
||||||
void StorageReplicatedMergeTree::mergeSelectingThread()
|
void StorageReplicatedMergeTree::mergeSelectingThread()
|
||||||
{
|
{
|
||||||
|
setThreadName("ReplMTMergeSel");
|
||||||
|
|
||||||
bool need_pull = true;
|
bool need_pull = true;
|
||||||
|
|
||||||
/** Может много времени тратиться на определение, можно ли мерджить два рядом стоящих куска.
|
/** Может много времени тратиться на определение, можно ли мерджить два рядом стоящих куска.
|
||||||
@ -1688,6 +1735,8 @@ void StorageReplicatedMergeTree::mergeSelectingThread()
|
|||||||
|
|
||||||
void StorageReplicatedMergeTree::alterThread()
|
void StorageReplicatedMergeTree::alterThread()
|
||||||
{
|
{
|
||||||
|
setThreadName("ReplMTAlter");
|
||||||
|
|
||||||
bool force_recheck_parts = true;
|
bool force_recheck_parts = true;
|
||||||
|
|
||||||
while (!shutdown_called)
|
while (!shutdown_called)
|
||||||
@ -2175,6 +2224,8 @@ void StorageReplicatedMergeTree::checkPart(const String & part_name)
|
|||||||
|
|
||||||
void StorageReplicatedMergeTree::partCheckThread()
|
void StorageReplicatedMergeTree::partCheckThread()
|
||||||
{
|
{
|
||||||
|
setThreadName("ReplMTPartCheck");
|
||||||
|
|
||||||
while (!shutdown_called)
|
while (!shutdown_called)
|
||||||
{
|
{
|
||||||
try
|
try
|
||||||
@ -2352,12 +2403,8 @@ void StorageReplicatedMergeTree::fetchPart(const String & part_name, const Strin
|
|||||||
|
|
||||||
String host_port_str = zookeeper->get(replica_path + "/host");
|
String host_port_str = zookeeper->get(replica_path + "/host");
|
||||||
ReadBufferFromString buf(host_port_str);
|
ReadBufferFromString buf(host_port_str);
|
||||||
assertString("host: ", buf);
|
buf >> "host: " >> host >> "\n"
|
||||||
readString(host, buf);
|
>> "port: " >> port >> "\n";
|
||||||
assertString("\nport: ", buf);
|
|
||||||
readText(port, buf);
|
|
||||||
assertString("\n", buf);
|
|
||||||
assertEOF(buf);
|
|
||||||
|
|
||||||
MergeTreeData::MutableDataPartPtr part = fetcher.fetchPart(part_name, replica_path, host, port, to_detached);
|
MergeTreeData::MutableDataPartPtr part = fetcher.fetchPart(part_name, replica_path, host, port, to_detached);
|
||||||
|
|
||||||
@ -2777,7 +2824,7 @@ void StorageReplicatedMergeTree::dropPartition(const Field & field, bool detach,
|
|||||||
|
|
||||||
/// Такого никогда не должно происходить.
|
/// Такого никогда не должно происходить.
|
||||||
if (right == 0)
|
if (right == 0)
|
||||||
return;
|
throw Exception("Logical error: just allocated block number is zero", ErrorCodes::LOGICAL_ERROR);
|
||||||
--right;
|
--right;
|
||||||
|
|
||||||
String fake_part_name = getFakePartNameForDrop(month_name, 0, right);
|
String fake_part_name = getFakePartNameForDrop(month_name, 0, right);
|
||||||
@ -3221,6 +3268,18 @@ void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void StorageReplicatedMergeTree::getQueue(LogEntriesData & res, String & replica_name_)
|
||||||
|
{
|
||||||
|
res.clear();
|
||||||
|
replica_name_ = replica_name;
|
||||||
|
|
||||||
|
std::lock_guard<std::mutex> lock(queue_mutex);
|
||||||
|
res.reserve(queue.size());
|
||||||
|
for (const auto & entry : queue)
|
||||||
|
res.emplace_back(*entry);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
void StorageReplicatedMergeTree::fetchPartition(const Field & partition, const String & from_, const Settings & settings)
|
void StorageReplicatedMergeTree::fetchPartition(const Field & partition, const String & from_, const Settings & settings)
|
||||||
{
|
{
|
||||||
auto zookeeper = getZooKeeper();
|
auto zookeeper = getZooKeeper();
|
||||||
|
@ -1,4 +1,4 @@
|
|||||||
#include <DB/Storages/StorageSystemClusters.h>
|
#include <DB/Storages/System/StorageSystemClusters.h>
|
||||||
#include <DB/Interpreters/Cluster.h>
|
#include <DB/Interpreters/Cluster.h>
|
||||||
#include <DB/Columns/ColumnString.h>
|
#include <DB/Columns/ColumnString.h>
|
||||||
#include <DB/Columns/ColumnsNumber.h>
|
#include <DB/Columns/ColumnsNumber.h>
|
@ -1,4 +1,4 @@
|
|||||||
#include <DB/Storages/StorageSystemColumns.h>
|
#include <DB/Storages/System/StorageSystemColumns.h>
|
||||||
#include <DB/Storages/MergeTree/MergeTreeData.h>
|
#include <DB/Storages/MergeTree/MergeTreeData.h>
|
||||||
#include <DB/Storages/StorageMergeTree.h>
|
#include <DB/Storages/StorageMergeTree.h>
|
||||||
#include <DB/Storages/StorageReplicatedMergeTree.h>
|
#include <DB/Storages/StorageReplicatedMergeTree.h>
|
@ -1,7 +1,7 @@
|
|||||||
#include <DB/Columns/ColumnString.h>
|
#include <DB/Columns/ColumnString.h>
|
||||||
#include <DB/DataTypes/DataTypeString.h>
|
#include <DB/DataTypes/DataTypeString.h>
|
||||||
#include <DB/DataStreams/OneBlockInputStream.h>
|
#include <DB/DataStreams/OneBlockInputStream.h>
|
||||||
#include <DB/Storages/StorageSystemDatabases.h>
|
#include <DB/Storages/System/StorageSystemDatabases.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
@ -1,4 +1,4 @@
|
|||||||
#include <DB/Storages/StorageSystemDictionaries.h>
|
#include <DB/Storages/System/StorageSystemDictionaries.h>
|
||||||
#include <DB/DataTypes/DataTypeString.h>
|
#include <DB/DataTypes/DataTypeString.h>
|
||||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||||
#include <DB/DataTypes/DataTypeArray.h>
|
#include <DB/DataTypes/DataTypeArray.h>
|
@ -3,7 +3,7 @@
|
|||||||
#include <DB/DataTypes/DataTypeString.h>
|
#include <DB/DataTypes/DataTypeString.h>
|
||||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||||
#include <DB/DataStreams/OneBlockInputStream.h>
|
#include <DB/DataStreams/OneBlockInputStream.h>
|
||||||
#include <DB/Storages/StorageSystemEvents.h>
|
#include <DB/Storages/System/StorageSystemEvents.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
@ -1,4 +1,4 @@
|
|||||||
#include <DB/Storages/StorageSystemFunctions.h>
|
#include <DB/Storages/System/StorageSystemFunctions.h>
|
||||||
#include <DB/Functions/FunctionFactory.h>
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
|
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
|
||||||
#include <DB/Columns/ColumnString.h>
|
#include <DB/Columns/ColumnString.h>
|
||||||
@ -46,14 +46,17 @@ BlockInputStreams StorageSystemFunctions::read(
|
|||||||
column_is_aggregate.column->insert(UInt64(0));
|
column_is_aggregate.column->insert(UInt64(0));
|
||||||
}
|
}
|
||||||
|
|
||||||
const auto & aggregate_functions = context.getAggregateFunctionFactory().getFunctionNames();
|
const auto & aggregate_function_factory = context.getAggregateFunctionFactory();
|
||||||
for (const auto & it : aggregate_functions)
|
for (const auto & details : aggregate_function_factory)
|
||||||
{
|
{
|
||||||
column_name.column->insert(it);
|
if (!details.is_alias)
|
||||||
column_is_aggregate.column->insert(UInt64(1));
|
{
|
||||||
|
column_name.column->insert(details.name);
|
||||||
|
column_is_aggregate.column->insert(UInt64(1));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
return BlockInputStreams{ 1, new OneBlockInputStream{{ column_name, column_is_aggregate }} };
|
return BlockInputStreams{ 1, new OneBlockInputStream{{ column_name, column_is_aggregate }} };
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
@ -1,4 +1,4 @@
|
|||||||
#include <DB/Storages/StorageSystemMerges.h>
|
#include <DB/Storages/System/StorageSystemMerges.h>
|
||||||
#include <DB/DataTypes/DataTypeString.h>
|
#include <DB/DataTypes/DataTypeString.h>
|
||||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||||
#include <DB/Columns/ColumnString.h>
|
#include <DB/Columns/ColumnString.h>
|
@ -5,7 +5,7 @@
|
|||||||
#include <DB/Columns/ColumnsNumber.h>
|
#include <DB/Columns/ColumnsNumber.h>
|
||||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||||
#include <DB/Storages/StorageSystemNumbers.h>
|
#include <DB/Storages/System/StorageSystemNumbers.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
@ -4,7 +4,7 @@
|
|||||||
#include <DB/Columns/ColumnsNumber.h>
|
#include <DB/Columns/ColumnsNumber.h>
|
||||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||||
#include <DB/DataStreams/OneBlockInputStream.h>
|
#include <DB/DataStreams/OneBlockInputStream.h>
|
||||||
#include <DB/Storages/StorageSystemOne.h>
|
#include <DB/Storages/System/StorageSystemOne.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
@ -4,7 +4,7 @@
|
|||||||
#include <DB/DataTypes/DataTypeDateTime.h>
|
#include <DB/DataTypes/DataTypeDateTime.h>
|
||||||
#include <DB/DataTypes/DataTypeDate.h>
|
#include <DB/DataTypes/DataTypeDate.h>
|
||||||
#include <DB/DataStreams/OneBlockInputStream.h>
|
#include <DB/DataStreams/OneBlockInputStream.h>
|
||||||
#include <DB/Storages/StorageSystemParts.h>
|
#include <DB/Storages/System/StorageSystemParts.h>
|
||||||
#include <DB/Storages/StorageMergeTree.h>
|
#include <DB/Storages/StorageMergeTree.h>
|
||||||
#include <DB/Storages/StorageReplicatedMergeTree.h>
|
#include <DB/Storages/StorageReplicatedMergeTree.h>
|
||||||
#include <DB/Common/VirtualColumnUtils.h>
|
#include <DB/Common/VirtualColumnUtils.h>
|
@ -3,7 +3,7 @@
|
|||||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||||
#include <DB/DataStreams/OneBlockInputStream.h>
|
#include <DB/DataStreams/OneBlockInputStream.h>
|
||||||
#include <DB/Interpreters/ProcessList.h>
|
#include <DB/Interpreters/ProcessList.h>
|
||||||
#include <DB/Storages/StorageSystemProcesses.h>
|
#include <DB/Storages/System/StorageSystemProcesses.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
@ -3,7 +3,7 @@
|
|||||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||||
#include <DB/DataTypes/DataTypeDateTime.h>
|
#include <DB/DataTypes/DataTypeDateTime.h>
|
||||||
#include <DB/DataStreams/OneBlockInputStream.h>
|
#include <DB/DataStreams/OneBlockInputStream.h>
|
||||||
#include <DB/Storages/StorageSystemReplicas.h>
|
#include <DB/Storages/System/StorageSystemReplicas.h>
|
||||||
#include <DB/Storages/StorageReplicatedMergeTree.h>
|
#include <DB/Storages/StorageReplicatedMergeTree.h>
|
||||||
#include <DB/Common/VirtualColumnUtils.h>
|
#include <DB/Common/VirtualColumnUtils.h>
|
||||||
|
|
193
dbms/src/Storages/System/StorageSystemReplicationQueue.cpp
Normal file
193
dbms/src/Storages/System/StorageSystemReplicationQueue.cpp
Normal file
@ -0,0 +1,193 @@
|
|||||||
|
#include <DB/Columns/ColumnString.h>
|
||||||
|
#include <DB/DataTypes/DataTypeString.h>
|
||||||
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||||
|
#include <DB/DataTypes/DataTypeDateTime.h>
|
||||||
|
#include <DB/DataStreams/OneBlockInputStream.h>
|
||||||
|
#include <DB/Storages/System/StorageSystemReplicationQueue.h>
|
||||||
|
#include <DB/Storages/StorageReplicatedMergeTree.h>
|
||||||
|
#include <DB/Common/VirtualColumnUtils.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
|
||||||
|
StorageSystemReplicationQueue::StorageSystemReplicationQueue(const std::string & name_)
|
||||||
|
: name(name_)
|
||||||
|
, columns{
|
||||||
|
/// Свойства таблицы.
|
||||||
|
{ "database", new DataTypeString },
|
||||||
|
{ "table", new DataTypeString },
|
||||||
|
{ "replica_name", new DataTypeString },
|
||||||
|
/// Неизменяемые свойства элемента.
|
||||||
|
{ "position", new DataTypeUInt32 },
|
||||||
|
{ "node_name", new DataTypeString },
|
||||||
|
{ "type", new DataTypeString },
|
||||||
|
{ "create_time", new DataTypeDateTime},
|
||||||
|
{ "required_quorum", new DataTypeUInt32 },
|
||||||
|
{ "source_replica", new DataTypeString },
|
||||||
|
{ "new_part_name", new DataTypeString },
|
||||||
|
{ "parts_to_merge", new DataTypeArray(new DataTypeString) },
|
||||||
|
{ "is_detach", new DataTypeUInt8 },
|
||||||
|
{ "is_attach_unreplicated", new DataTypeUInt8 },
|
||||||
|
{ "attach_source_part_name",new DataTypeString },
|
||||||
|
/// Статус обработки элемента.
|
||||||
|
{ "is_currently_executing", new DataTypeUInt8 },
|
||||||
|
{ "num_tries", new DataTypeUInt32 },
|
||||||
|
{ "last_exception", new DataTypeString },
|
||||||
|
{ "last_attempt_time", new DataTypeDateTime},
|
||||||
|
{ "num_postponed", new DataTypeUInt32 },
|
||||||
|
{ "postpone_reason", new DataTypeString },
|
||||||
|
{ "last_postpone_time", new DataTypeDateTime},
|
||||||
|
}
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
StoragePtr StorageSystemReplicationQueue::create(const std::string & name_)
|
||||||
|
{
|
||||||
|
return (new StorageSystemReplicationQueue(name_))->thisPtr();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
BlockInputStreams StorageSystemReplicationQueue::read(
|
||||||
|
const Names & column_names,
|
||||||
|
ASTPtr query,
|
||||||
|
const Context & context,
|
||||||
|
const Settings & settings,
|
||||||
|
QueryProcessingStage::Enum & processed_stage,
|
||||||
|
const size_t max_block_size,
|
||||||
|
const unsigned threads)
|
||||||
|
{
|
||||||
|
check(column_names);
|
||||||
|
processed_stage = QueryProcessingStage::FetchColumns;
|
||||||
|
|
||||||
|
/// Собираем набор реплицируемых таблиц.
|
||||||
|
Databases replicated_tables;
|
||||||
|
{
|
||||||
|
Poco::ScopedLock<Poco::Mutex> lock(context.getMutex());
|
||||||
|
|
||||||
|
for (const auto & db : context.getDatabases())
|
||||||
|
for (const auto & table : db.second)
|
||||||
|
if (typeid_cast<const StorageReplicatedMergeTree *>(table.second.get()))
|
||||||
|
replicated_tables[db.first][table.first] = table.second;
|
||||||
|
}
|
||||||
|
|
||||||
|
ColumnWithTypeAndName col_database_to_filter { new ColumnString, new DataTypeString, "database" };
|
||||||
|
ColumnWithTypeAndName col_table_to_filter { new ColumnString, new DataTypeString, "table" };
|
||||||
|
|
||||||
|
for (auto & db : replicated_tables)
|
||||||
|
{
|
||||||
|
for (auto & table : db.second)
|
||||||
|
{
|
||||||
|
col_database_to_filter.column->insert(db.first);
|
||||||
|
col_table_to_filter.column->insert(table.first);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Определяем, какие нужны таблицы, по условиям в запросе.
|
||||||
|
{
|
||||||
|
Block filtered_block { col_database_to_filter, col_table_to_filter };
|
||||||
|
|
||||||
|
VirtualColumnUtils::filterBlockWithQuery(query, filtered_block, context);
|
||||||
|
|
||||||
|
if (!filtered_block.rows())
|
||||||
|
return BlockInputStreams();
|
||||||
|
|
||||||
|
col_database_to_filter = filtered_block.getByName("database");
|
||||||
|
col_table_to_filter = filtered_block.getByName("table");
|
||||||
|
}
|
||||||
|
|
||||||
|
ColumnWithTypeAndName col_database { new ColumnString, new DataTypeString, "database" };
|
||||||
|
ColumnWithTypeAndName col_table { new ColumnString, new DataTypeString, "table" };
|
||||||
|
ColumnWithTypeAndName col_replica_name { new ColumnString, new DataTypeString, "replica_name" };
|
||||||
|
ColumnWithTypeAndName col_position { new ColumnUInt32, new DataTypeUInt32, "position" };
|
||||||
|
ColumnWithTypeAndName col_node_name { new ColumnString, new DataTypeString, "node_name" };
|
||||||
|
ColumnWithTypeAndName col_type { new ColumnString, new DataTypeString, "type" };
|
||||||
|
ColumnWithTypeAndName col_create_time { new ColumnUInt32, new DataTypeDateTime, "create_time" };
|
||||||
|
ColumnWithTypeAndName col_required_quorum { new ColumnUInt32, new DataTypeUInt32, "required_quorum" };
|
||||||
|
ColumnWithTypeAndName col_source_replica { new ColumnString, new DataTypeString, "source_replica" };
|
||||||
|
ColumnWithTypeAndName col_new_part_name { new ColumnString, new DataTypeString, "new_part_name" };
|
||||||
|
ColumnWithTypeAndName col_parts_to_merge { new ColumnArray(new ColumnString), new DataTypeArray(new DataTypeString), "parts_to_merge" };
|
||||||
|
ColumnWithTypeAndName col_is_detach { new ColumnUInt8, new DataTypeUInt8, "is_detach" };
|
||||||
|
ColumnWithTypeAndName col_is_attach_unreplicated { new ColumnUInt8, new DataTypeUInt8, "is_attach_unreplicated" };
|
||||||
|
ColumnWithTypeAndName col_attach_source_part_name { new ColumnString, new DataTypeString, "attach_source_part_name" };
|
||||||
|
ColumnWithTypeAndName col_is_currently_executing { new ColumnUInt8, new DataTypeUInt8, "is_currently_executing" };
|
||||||
|
ColumnWithTypeAndName col_num_tries { new ColumnUInt32, new DataTypeUInt32, "num_tries" };
|
||||||
|
ColumnWithTypeAndName col_last_exception { new ColumnString, new DataTypeString, "last_exception" };
|
||||||
|
ColumnWithTypeAndName col_last_attempt_time { new ColumnUInt32, new DataTypeDateTime, "last_attempt_time" };
|
||||||
|
ColumnWithTypeAndName col_num_postponed { new ColumnUInt32, new DataTypeUInt32, "num_postponed" };
|
||||||
|
ColumnWithTypeAndName col_postpone_reason { new ColumnString, new DataTypeString, "postpone_reason" };
|
||||||
|
ColumnWithTypeAndName col_last_postpone_time { new ColumnUInt32, new DataTypeDateTime, "last_postpone_time" };
|
||||||
|
|
||||||
|
StorageReplicatedMergeTree::LogEntriesData queue;
|
||||||
|
String replica_name;
|
||||||
|
|
||||||
|
for (size_t i = 0, tables_size = col_database_to_filter.column->size(); i < tables_size; ++i)
|
||||||
|
{
|
||||||
|
String database = (*col_database_to_filter.column)[i].safeGet<const String &>();
|
||||||
|
String table = (*col_table_to_filter.column)[i].safeGet<const String &>();
|
||||||
|
|
||||||
|
typeid_cast<StorageReplicatedMergeTree &>(*replicated_tables[database][table]).getQueue(queue, replica_name);
|
||||||
|
|
||||||
|
for (size_t j = 0, queue_size = queue.size(); j < queue_size; ++j)
|
||||||
|
{
|
||||||
|
const auto & entry = queue[j];
|
||||||
|
|
||||||
|
Array parts_to_merge;
|
||||||
|
parts_to_merge.reserve(entry.parts_to_merge.size());
|
||||||
|
for (const auto & name : entry.parts_to_merge)
|
||||||
|
parts_to_merge.push_back(name);
|
||||||
|
|
||||||
|
col_database .column->insert(database);
|
||||||
|
col_table .column->insert(table);
|
||||||
|
col_replica_name .column->insert(replica_name);
|
||||||
|
col_position .column->insert(UInt64(j));
|
||||||
|
col_node_name .column->insert(entry.znode_name);
|
||||||
|
col_type .column->insert(entry.typeToString());
|
||||||
|
col_create_time .column->insert(UInt64(entry.create_time));
|
||||||
|
col_required_quorum .column->insert(UInt64(entry.quorum));
|
||||||
|
col_source_replica .column->insert(entry.source_replica);
|
||||||
|
col_new_part_name .column->insert(entry.new_part_name);
|
||||||
|
col_parts_to_merge .column->insert(parts_to_merge);
|
||||||
|
col_is_detach .column->insert(UInt64(entry.detach));
|
||||||
|
col_is_attach_unreplicated .column->insert(UInt64(entry.attach_unreplicated));
|
||||||
|
col_attach_source_part_name .column->insert(entry.source_part_name);
|
||||||
|
col_is_currently_executing .column->insert(UInt64(entry.currently_executing));
|
||||||
|
col_num_tries .column->insert(UInt64(entry.num_tries));
|
||||||
|
col_last_exception .column->insert(entry.exception ? entry.exception->displayText() : "");
|
||||||
|
col_last_attempt_time .column->insert(UInt64(entry.last_attempt_time));
|
||||||
|
col_num_postponed .column->insert(UInt64(entry.num_postponed));
|
||||||
|
col_postpone_reason .column->insert(entry.postpone_reason);
|
||||||
|
col_last_postpone_time .column->insert(UInt64(entry.last_postpone_time));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
Block block{
|
||||||
|
col_database,
|
||||||
|
col_table,
|
||||||
|
col_replica_name,
|
||||||
|
col_position,
|
||||||
|
col_node_name,
|
||||||
|
col_type,
|
||||||
|
col_create_time,
|
||||||
|
col_required_quorum,
|
||||||
|
col_source_replica,
|
||||||
|
col_new_part_name,
|
||||||
|
col_parts_to_merge,
|
||||||
|
col_is_detach,
|
||||||
|
col_is_attach_unreplicated,
|
||||||
|
col_attach_source_part_name,
|
||||||
|
col_is_currently_executing,
|
||||||
|
col_num_tries,
|
||||||
|
col_last_exception,
|
||||||
|
col_last_attempt_time,
|
||||||
|
col_num_postponed,
|
||||||
|
col_postpone_reason,
|
||||||
|
col_last_postpone_time,
|
||||||
|
};
|
||||||
|
|
||||||
|
return BlockInputStreams(1, new OneBlockInputStream(block));
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
}
|
@ -2,7 +2,7 @@
|
|||||||
#include <DB/DataTypes/DataTypeString.h>
|
#include <DB/DataTypes/DataTypeString.h>
|
||||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||||
#include <DB/DataStreams/OneBlockInputStream.h>
|
#include <DB/DataStreams/OneBlockInputStream.h>
|
||||||
#include <DB/Storages/StorageSystemSettings.h>
|
#include <DB/Storages/System/StorageSystemSettings.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
@ -1,7 +1,7 @@
|
|||||||
#include <DB/Columns/ColumnString.h>
|
#include <DB/Columns/ColumnString.h>
|
||||||
#include <DB/DataTypes/DataTypeString.h>
|
#include <DB/DataTypes/DataTypeString.h>
|
||||||
#include <DB/DataStreams/OneBlockInputStream.h>
|
#include <DB/DataStreams/OneBlockInputStream.h>
|
||||||
#include <DB/Storages/StorageSystemTables.h>
|
#include <DB/Storages/System/StorageSystemTables.h>
|
||||||
#include <DB/Common/VirtualColumnUtils.h>
|
#include <DB/Common/VirtualColumnUtils.h>
|
||||||
|
|
||||||
|
|
@ -3,7 +3,7 @@
|
|||||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||||
#include <DB/DataTypes/DataTypeDateTime.h>
|
#include <DB/DataTypes/DataTypeDateTime.h>
|
||||||
#include <DB/DataStreams/OneBlockInputStream.h>
|
#include <DB/DataStreams/OneBlockInputStream.h>
|
||||||
#include <DB/Storages/StorageSystemZooKeeper.h>
|
#include <DB/Storages/System/StorageSystemZooKeeper.h>
|
||||||
#include <DB/Parsers/ASTSelectQuery.h>
|
#include <DB/Parsers/ASTSelectQuery.h>
|
||||||
#include <DB/Parsers/ASTIdentifier.h>
|
#include <DB/Parsers/ASTIdentifier.h>
|
||||||
#include <DB/Parsers/ASTLiteral.h>
|
#include <DB/Parsers/ASTLiteral.h>
|
@ -3,7 +3,7 @@
|
|||||||
#include <Poco/SharedPtr.h>
|
#include <Poco/SharedPtr.h>
|
||||||
|
|
||||||
#include <DB/IO/WriteBufferFromOStream.h>
|
#include <DB/IO/WriteBufferFromOStream.h>
|
||||||
#include <DB/Storages/StorageSystemNumbers.h>
|
#include <DB/Storages/System/StorageSystemNumbers.h>
|
||||||
#include <DB/DataStreams/LimitBlockInputStream.h>
|
#include <DB/DataStreams/LimitBlockInputStream.h>
|
||||||
#include <DB/DataStreams/TabSeparatedRowOutputStream.h>
|
#include <DB/DataStreams/TabSeparatedRowOutputStream.h>
|
||||||
#include <DB/DataStreams/BlockOutputStreamFromRowOutputStream.h>
|
#include <DB/DataStreams/BlockOutputStreamFromRowOutputStream.h>
|
||||||
@ -31,11 +31,11 @@ int main(int argc, char ** argv)
|
|||||||
DB::WriteBufferFromOStream out_buf(std::cout);
|
DB::WriteBufferFromOStream out_buf(std::cout);
|
||||||
|
|
||||||
DB::QueryProcessingStage::Enum stage;
|
DB::QueryProcessingStage::Enum stage;
|
||||||
|
|
||||||
DB::LimitBlockInputStream input(table->read(column_names, 0, DB::Context{}, DB::Settings(), stage, 10)[0], 10, 96);
|
DB::LimitBlockInputStream input(table->read(column_names, 0, DB::Context{}, DB::Settings(), stage, 10)[0], 10, 96);
|
||||||
DB::RowOutputStreamPtr output_ = new DB::TabSeparatedRowOutputStream(out_buf, sample);
|
DB::RowOutputStreamPtr output_ = new DB::TabSeparatedRowOutputStream(out_buf, sample);
|
||||||
DB::BlockOutputStreamFromRowOutputStream output(output_);
|
DB::BlockOutputStreamFromRowOutputStream output(output_);
|
||||||
|
|
||||||
DB::copyData(input, output);
|
DB::copyData(input, output);
|
||||||
}
|
}
|
||||||
catch (const DB::Exception & e)
|
catch (const DB::Exception & e)
|
||||||
|
@ -264,6 +264,10 @@ public:
|
|||||||
GetChildrenFuture asyncGetChildren(const std::string & path);
|
GetChildrenFuture asyncGetChildren(const std::string & path);
|
||||||
|
|
||||||
|
|
||||||
|
using RemoveFuture = Future<void, int>;
|
||||||
|
RemoveFuture asyncRemove(const std::string & path);
|
||||||
|
|
||||||
|
|
||||||
static std::string error2string(int32_t code);
|
static std::string error2string(int32_t code);
|
||||||
|
|
||||||
/// максимальный размер данных в узле в байтах
|
/// максимальный размер данных в узле в байтах
|
||||||
|
@ -760,4 +760,32 @@ ZooKeeper::GetChildrenFuture ZooKeeper::asyncGetChildren(const std::string & pat
|
|||||||
return future;
|
return future;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
ZooKeeper::RemoveFuture ZooKeeper::asyncRemove(const std::string & path)
|
||||||
|
{
|
||||||
|
RemoveFuture future {
|
||||||
|
[path] (int rc)
|
||||||
|
{
|
||||||
|
if (rc != ZOK)
|
||||||
|
throw KeeperException(rc, path);
|
||||||
|
}};
|
||||||
|
|
||||||
|
int32_t code = zoo_adelete(
|
||||||
|
impl, path.c_str(), -1,
|
||||||
|
[] (int rc, const void * data)
|
||||||
|
{
|
||||||
|
RemoveFuture::TaskPtr owned_task =
|
||||||
|
std::move(const_cast<RemoveFuture::TaskPtr &>(*static_cast<const RemoveFuture::TaskPtr *>(data)));
|
||||||
|
(*owned_task)(rc);
|
||||||
|
},
|
||||||
|
future.task.get());
|
||||||
|
|
||||||
|
ProfileEvents::increment(ProfileEvents::ZooKeeperRemove);
|
||||||
|
ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions);
|
||||||
|
|
||||||
|
if (code != ZOK)
|
||||||
|
throw KeeperException(code, path);
|
||||||
|
|
||||||
|
return future;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
65
utils/zookeeper-remove-by-list/main.cpp
Normal file
65
utils/zookeeper-remove-by-list/main.cpp
Normal file
@ -0,0 +1,65 @@
|
|||||||
|
#include <zkutil/ZooKeeper.h>
|
||||||
|
#include <DB/IO/ReadHelpers.h>
|
||||||
|
#include <DB/IO/ReadBufferFromFileDescriptor.h>
|
||||||
|
#include <boost/program_options.hpp>
|
||||||
|
|
||||||
|
|
||||||
|
int main(int argc, char ** argv)
|
||||||
|
try
|
||||||
|
{
|
||||||
|
boost::program_options::options_description desc("Allowed options");
|
||||||
|
desc.add_options()
|
||||||
|
("help,h", "produce help message")
|
||||||
|
("address,a", boost::program_options::value<std::string>()->required(),
|
||||||
|
"addresses of ZooKeeper instances, comma separated. Example: example01e.yandex.ru:2181")
|
||||||
|
;
|
||||||
|
|
||||||
|
boost::program_options::variables_map options;
|
||||||
|
boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options);
|
||||||
|
|
||||||
|
if (options.count("help"))
|
||||||
|
{
|
||||||
|
std::cout << "Remove nodes in ZooKeeper by list passed in stdin." << std::endl;
|
||||||
|
std::cout << "Usage: " << argv[0] << " [options] < list_of_nodes_on_each_line" << std::endl;
|
||||||
|
std::cout << desc << std::endl;
|
||||||
|
return 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
zkutil::ZooKeeper zookeeper(options.at("address").as<std::string>());
|
||||||
|
|
||||||
|
DB::ReadBufferFromFileDescriptor in(STDIN_FILENO);
|
||||||
|
std::list<zkutil::ZooKeeper::RemoveFuture> futures;
|
||||||
|
|
||||||
|
std::cerr << "Requested: ";
|
||||||
|
while (!in.eof())
|
||||||
|
{
|
||||||
|
std::string path;
|
||||||
|
DB::readEscapedString(path, in);
|
||||||
|
DB::assertString("\n", in);
|
||||||
|
futures.push_back(zookeeper.asyncRemove(path));
|
||||||
|
std::cerr << ".";
|
||||||
|
}
|
||||||
|
std::cerr << "\n";
|
||||||
|
|
||||||
|
std::cerr << "Done: ";
|
||||||
|
for (auto & future : futures)
|
||||||
|
{
|
||||||
|
try
|
||||||
|
{
|
||||||
|
future.get();
|
||||||
|
}
|
||||||
|
catch (...)
|
||||||
|
{
|
||||||
|
std::cerr << DB::getCurrentExceptionMessage(true) << '\n';
|
||||||
|
}
|
||||||
|
std::cerr << ".";
|
||||||
|
}
|
||||||
|
std::cerr << "\n";
|
||||||
|
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
catch (const Poco::Exception & e)
|
||||||
|
{
|
||||||
|
std::cerr << DB::getCurrentExceptionMessage(true) << '\n';
|
||||||
|
throw;
|
||||||
|
}
|
Loading…
Reference in New Issue
Block a user