mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge
This commit is contained in:
commit
d6e101ae4b
@ -76,6 +76,7 @@ public:
|
||||
v3 = 0x7465646279746573ULL ^ k1;
|
||||
|
||||
cnt = 0;
|
||||
current_word = 0;
|
||||
}
|
||||
|
||||
void update(const char * data, u64 size)
|
||||
|
@ -13,7 +13,7 @@ using Poco::SharedPtr;
|
||||
/** Тип - состояние агрегатной функции.
|
||||
* Параметры типа - это агрегатная функция, типы её аргументов и её параметры (для параметрических агрегатных функций).
|
||||
*/
|
||||
class DataTypeAggregateFunction : public IDataType
|
||||
class DataTypeAggregateFunction final : public IDataType
|
||||
{
|
||||
private:
|
||||
AggregateFunctionPtr function;
|
||||
|
@ -9,7 +9,7 @@ namespace DB
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
class DataTypeArray : public IDataType
|
||||
class DataTypeArray final : public IDataType
|
||||
{
|
||||
private:
|
||||
/// Тип элементов массивов.
|
||||
|
@ -10,7 +10,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class DataTypeDate : public IDataTypeNumberFixed<UInt16, ColumnUInt16>
|
||||
class DataTypeDate final : public IDataTypeNumberFixed<UInt16, ColumnUInt16>
|
||||
{
|
||||
public:
|
||||
DataTypeDate() {}
|
||||
|
@ -10,7 +10,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class DataTypeDateTime : public IDataTypeNumberFixed<UInt32, ColumnUInt32>
|
||||
class DataTypeDateTime final : public IDataTypeNumberFixed<UInt32, ColumnUInt32>
|
||||
{
|
||||
public:
|
||||
DataTypeDateTime() {}
|
||||
|
@ -9,7 +9,7 @@ namespace DB
|
||||
/**
|
||||
* Лямбда-выражение.
|
||||
*/
|
||||
class DataTypeExpression : public IDataTypeDummy
|
||||
class DataTypeExpression final : public IDataTypeDummy
|
||||
{
|
||||
private:
|
||||
DataTypes argument_types;
|
||||
|
@ -13,7 +13,7 @@ namespace DB
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
class DataTypeFixedString : public IDataType
|
||||
class DataTypeFixedString final : public IDataType
|
||||
{
|
||||
private:
|
||||
size_t n;
|
||||
|
@ -9,7 +9,7 @@ namespace DB
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
class DataTypeNested : public IDataType
|
||||
class DataTypeNested final : public IDataType
|
||||
{
|
||||
private:
|
||||
/// Имена и типы вложенных массивов.
|
||||
|
@ -9,7 +9,7 @@ namespace DB
|
||||
/** Тип данных, соответствующий множеству значений в секции IN.
|
||||
* Используется только как промежуточный вариант при вычислении выражений.
|
||||
*/
|
||||
class DataTypeSet : public IDataTypeDummy
|
||||
class DataTypeSet final : public IDataTypeDummy
|
||||
{
|
||||
public:
|
||||
std::string getName() const { return "Set"; }
|
||||
|
@ -13,7 +13,7 @@ namespace DB
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
class DataTypeString : public IDataType
|
||||
class DataTypeString final : public IDataType
|
||||
{
|
||||
public:
|
||||
using FieldType = String;
|
||||
|
@ -15,7 +15,7 @@ namespace DB
|
||||
* Также может быть использовать в качестве столбца - результата выполнения запроса.
|
||||
* Не может быть сохранён в таблицы.
|
||||
*/
|
||||
class DataTypeTuple : public IDataType
|
||||
class DataTypeTuple final : public IDataType
|
||||
{
|
||||
private:
|
||||
DataTypes elems;
|
||||
@ -138,6 +138,7 @@ public:
|
||||
{
|
||||
ColumnWithNameAndType col;
|
||||
col.column = elems[i]->createColumn();
|
||||
col.type = elems[i]->clone();
|
||||
tuple_block.insert(col);
|
||||
}
|
||||
return new ColumnTuple(tuple_block);
|
||||
|
@ -13,11 +13,11 @@ template <typename T>
|
||||
struct DataTypeFromFieldType;
|
||||
|
||||
#define DEFINE_DATA_TYPE_NUMBER_FIXED(TYPE) \
|
||||
class DataType ## TYPE : public IDataTypeNumberFixed<TYPE, Column ## TYPE> \
|
||||
class DataType ## TYPE final : public IDataTypeNumberFixed<TYPE, Column ## TYPE> \
|
||||
{ \
|
||||
public: \
|
||||
std::string getName() const { return #TYPE; } \
|
||||
DataTypePtr clone() const { return new DataType ## TYPE; } \
|
||||
DataTypePtr clone() const { return new DataType ## TYPE; } \
|
||||
}; \
|
||||
\
|
||||
template <> struct DataTypeFromFieldType<TYPE> \
|
||||
|
@ -42,6 +42,8 @@ public:
|
||||
: CacheDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.size}
|
||||
{}
|
||||
|
||||
std::exception_ptr getCreationException() const override { return {}; }
|
||||
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
std::string getTypeName() const override { return "Cache"; }
|
||||
|
@ -25,8 +25,17 @@ public:
|
||||
source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime)
|
||||
{
|
||||
createAttributes();
|
||||
loadData();
|
||||
calculateBytesAllocated();
|
||||
|
||||
try
|
||||
{
|
||||
loadData();
|
||||
calculateBytesAllocated();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
creation_exception = std::current_exception();
|
||||
}
|
||||
|
||||
creation_time = std::chrono::system_clock::now();
|
||||
}
|
||||
|
||||
@ -34,6 +43,8 @@ public:
|
||||
: FlatDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime}
|
||||
{}
|
||||
|
||||
std::exception_ptr getCreationException() const override { return creation_exception; }
|
||||
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
std::string getTypeName() const override { return "Flat"; }
|
||||
@ -398,10 +409,11 @@ private:
|
||||
std::size_t bytes_allocated = 0;
|
||||
std::size_t element_count = 0;
|
||||
std::size_t bucket_count = 0;
|
||||
mutable std::atomic<std::size_t> query_count;
|
||||
|
||||
std::chrono::time_point<std::chrono::system_clock> creation_time;
|
||||
|
||||
mutable std::atomic<std::size_t> query_count;
|
||||
std::exception_ptr creation_exception;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -22,8 +22,17 @@ public:
|
||||
source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime)
|
||||
{
|
||||
createAttributes();
|
||||
loadData();
|
||||
calculateBytesAllocated();
|
||||
|
||||
try
|
||||
{
|
||||
loadData();
|
||||
calculateBytesAllocated();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
creation_exception = std::current_exception();
|
||||
}
|
||||
|
||||
creation_time = std::chrono::system_clock::now();
|
||||
}
|
||||
|
||||
@ -31,6 +40,8 @@ public:
|
||||
: HashedDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime}
|
||||
{}
|
||||
|
||||
std::exception_ptr getCreationException() const override { return creation_exception; }
|
||||
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
std::string getTypeName() const override { return "Hashed"; }
|
||||
@ -389,6 +400,8 @@ private:
|
||||
mutable std::atomic<std::size_t> query_count{};
|
||||
|
||||
std::chrono::time_point<std::chrono::system_clock> creation_time;
|
||||
|
||||
std::exception_ptr creation_exception;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -24,6 +24,8 @@ class IDictionary
|
||||
public:
|
||||
using id_t = std::uint64_t;
|
||||
|
||||
virtual std::exception_ptr getCreationException() const = 0;
|
||||
|
||||
virtual std::string getName() const = 0;
|
||||
|
||||
virtual std::string getTypeName() const = 0;
|
||||
|
@ -26,6 +26,10 @@ namespace DB
|
||||
* - даты;
|
||||
* - даты-с-временем;
|
||||
* внутри каждой группы, но не из разных групп.
|
||||
*
|
||||
* Исключение: можно сравнивать дату и дату-с-временем с константной строкой. Пример: EventDate = '2015-01-01'.
|
||||
*
|
||||
* TODO Массивы, кортежи.
|
||||
*/
|
||||
|
||||
/** Игнорируем warning о сравнении signed и unsigned.
|
||||
@ -391,9 +395,9 @@ public:
|
||||
|
||||
private:
|
||||
template <typename T0, typename T1>
|
||||
bool executeNumRightType(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnVector<T0> * col_left)
|
||||
bool executeNumRightType(Block & block, size_t result, const ColumnVector<T0> * col_left, const IColumn * col_right_untyped)
|
||||
{
|
||||
if (ColumnVector<T1> * col_right = typeid_cast<ColumnVector<T1> *>(&*block.getByPosition(arguments[1]).column))
|
||||
if (const ColumnVector<T1> * col_right = typeid_cast<const ColumnVector<T1> *>(col_right_untyped))
|
||||
{
|
||||
ColumnUInt8 * col_res = new ColumnUInt8;
|
||||
block.getByPosition(result).column = col_res;
|
||||
@ -404,7 +408,7 @@ private:
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (ColumnConst<T1> * col_right = typeid_cast<ColumnConst<T1> *>(&*block.getByPosition(arguments[1]).column))
|
||||
else if (const ColumnConst<T1> * col_right = typeid_cast<const ColumnConst<T1> *>(col_right_untyped))
|
||||
{
|
||||
ColumnUInt8 * col_res = new ColumnUInt8;
|
||||
block.getByPosition(result).column = col_res;
|
||||
@ -420,9 +424,9 @@ private:
|
||||
}
|
||||
|
||||
template <typename T0, typename T1>
|
||||
bool executeNumConstRightType(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnConst<T0> * col_left)
|
||||
bool executeNumConstRightType(Block & block, size_t result, const ColumnConst<T0> * col_left, const IColumn * col_right_untyped)
|
||||
{
|
||||
if (ColumnVector<T1> * col_right = typeid_cast<ColumnVector<T1> *>(&*block.getByPosition(arguments[1]).column))
|
||||
if (const ColumnVector<T1> * col_right = typeid_cast<const ColumnVector<T1> *>(col_right_untyped))
|
||||
{
|
||||
ColumnUInt8 * col_res = new ColumnUInt8;
|
||||
block.getByPosition(result).column = col_res;
|
||||
@ -433,7 +437,7 @@ private:
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (ColumnConst<T1> * col_right = typeid_cast<ColumnConst<T1> *>(&*block.getByPosition(arguments[1]).column))
|
||||
else if (const ColumnConst<T1> * col_right = typeid_cast<const ColumnConst<T1> *>(col_right_untyped))
|
||||
{
|
||||
UInt8 res = 0;
|
||||
NumComparisonImpl<T0, T1, Op<T0, T1>>::constant_constant(col_left->getData(), col_right->getData(), res);
|
||||
@ -448,41 +452,41 @@ private:
|
||||
}
|
||||
|
||||
template <typename T0>
|
||||
bool executeNumLeftType(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
bool executeNumLeftType(Block & block, size_t result, const IColumn * col_left_untyped, const IColumn * col_right_untyped)
|
||||
{
|
||||
if (ColumnVector<T0> * col_left = typeid_cast<ColumnVector<T0> *>(&*block.getByPosition(arguments[0]).column))
|
||||
if (const ColumnVector<T0> * col_left = typeid_cast<const ColumnVector<T0> *>(col_left_untyped))
|
||||
{
|
||||
if ( executeNumRightType<T0, UInt8>(block, arguments, result, col_left)
|
||||
|| executeNumRightType<T0, UInt16>(block, arguments, result, col_left)
|
||||
|| executeNumRightType<T0, UInt32>(block, arguments, result, col_left)
|
||||
|| executeNumRightType<T0, UInt64>(block, arguments, result, col_left)
|
||||
|| executeNumRightType<T0, Int8>(block, arguments, result, col_left)
|
||||
|| executeNumRightType<T0, Int16>(block, arguments, result, col_left)
|
||||
|| executeNumRightType<T0, Int32>(block, arguments, result, col_left)
|
||||
|| executeNumRightType<T0, Int64>(block, arguments, result, col_left)
|
||||
|| executeNumRightType<T0, Float32>(block, arguments, result, col_left)
|
||||
|| executeNumRightType<T0, Float64>(block, arguments, result, col_left))
|
||||
if ( executeNumRightType<T0, UInt8>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, UInt16>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, UInt32>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, UInt64>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Int8>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Int16>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Int32>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Int64>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Float32>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Float64>(block, result, col_left, col_right_untyped))
|
||||
return true;
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||
throw Exception("Illegal column " + col_right_untyped->getName()
|
||||
+ " of second argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else if (ColumnConst<T0> * col_left = typeid_cast<ColumnConst<T0> *>(&*block.getByPosition(arguments[0]).column))
|
||||
else if (const ColumnConst<T0> * col_left = typeid_cast<const ColumnConst<T0> *>(col_left_untyped))
|
||||
{
|
||||
if ( executeNumConstRightType<T0, UInt8>(block, arguments, result, col_left)
|
||||
|| executeNumConstRightType<T0, UInt16>(block, arguments, result, col_left)
|
||||
|| executeNumConstRightType<T0, UInt32>(block, arguments, result, col_left)
|
||||
|| executeNumConstRightType<T0, UInt64>(block, arguments, result, col_left)
|
||||
|| executeNumConstRightType<T0, Int8>(block, arguments, result, col_left)
|
||||
|| executeNumConstRightType<T0, Int16>(block, arguments, result, col_left)
|
||||
|| executeNumConstRightType<T0, Int32>(block, arguments, result, col_left)
|
||||
|| executeNumConstRightType<T0, Int64>(block, arguments, result, col_left)
|
||||
|| executeNumConstRightType<T0, Float32>(block, arguments, result, col_left)
|
||||
|| executeNumConstRightType<T0, Float64>(block, arguments, result, col_left))
|
||||
if ( executeNumConstRightType<T0, UInt8>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, UInt16>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, UInt32>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, UInt64>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Int8>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Int16>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Int32>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Int64>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Float32>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Float64>(block, result, col_left, col_right_untyped))
|
||||
return true;
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||
throw Exception("Illegal column " + col_right_untyped->getName()
|
||||
+ " of second argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -490,17 +494,14 @@ private:
|
||||
return false;
|
||||
}
|
||||
|
||||
void executeString(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
void executeString(Block & block, size_t result, const IColumn * c0, const IColumn * c1)
|
||||
{
|
||||
IColumn * c0 = &*block.getByPosition(arguments[0]).column;
|
||||
IColumn * c1 = &*block.getByPosition(arguments[1]).column;
|
||||
|
||||
ColumnString * c0_string = typeid_cast<ColumnString *>(c0);
|
||||
ColumnString * c1_string = typeid_cast<ColumnString *>(c1);
|
||||
ColumnFixedString * c0_fixed_string = typeid_cast<ColumnFixedString *>(c0);
|
||||
ColumnFixedString * c1_fixed_string = typeid_cast<ColumnFixedString *>(c1);
|
||||
ColumnConstString * c0_const = typeid_cast<ColumnConstString *>(c0);
|
||||
ColumnConstString * c1_const = typeid_cast<ColumnConstString *>(c1);
|
||||
const ColumnString * c0_string = typeid_cast<const ColumnString *>(c0);
|
||||
const ColumnString * c1_string = typeid_cast<const ColumnString *>(c1);
|
||||
const ColumnFixedString * c0_fixed_string = typeid_cast<const ColumnFixedString *>(c0);
|
||||
const ColumnFixedString * c1_fixed_string = typeid_cast<const ColumnFixedString *>(c1);
|
||||
const ColumnConstString * c0_const = typeid_cast<const ColumnConstString *>(c0);
|
||||
const ColumnConstString * c1_const = typeid_cast<const ColumnConstString *>(c1);
|
||||
|
||||
using StringImpl = StringComparisonImpl<Op<int, int>>;
|
||||
|
||||
@ -559,13 +560,66 @@ private:
|
||||
c_res->getData());
|
||||
else
|
||||
throw Exception("Illegal columns "
|
||||
+ block.getByPosition(arguments[0]).column->getName() + " and "
|
||||
+ block.getByPosition(arguments[1]).column->getName()
|
||||
+ c0->getName() + " and " + c1->getName()
|
||||
+ " of arguments of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
|
||||
void executeDateOrDateTimeWithConstString(Block & block, size_t result,
|
||||
const IColumn * col_left_untyped, const IColumn * col_right_untyped,
|
||||
bool left_is_num, bool right_is_num)
|
||||
{
|
||||
/// Особый случай - сравнение дат и дат-с-временем со строковой константой.
|
||||
const IColumn * column_date_or_datetime = left_is_num ? col_left_untyped : col_right_untyped;
|
||||
const IColumn * column_string_untyped = !left_is_num ? col_left_untyped : col_right_untyped;
|
||||
|
||||
bool is_date = false;
|
||||
bool is_date_time = false;
|
||||
|
||||
is_date = typeid_cast<const ColumnVector<DataTypeDate::FieldType> *>(column_date_or_datetime)
|
||||
|| typeid_cast<const ColumnConst<DataTypeDate::FieldType> *>(column_date_or_datetime);
|
||||
|
||||
if (!is_date)
|
||||
is_date_time = typeid_cast<const ColumnVector<DataTypeDateTime::FieldType> *>(column_date_or_datetime)
|
||||
|| typeid_cast<const ColumnConst<DataTypeDateTime::FieldType> *>(column_date_or_datetime);
|
||||
|
||||
const ColumnConstString * column_string = typeid_cast<const ColumnConstString *>(column_string_untyped);
|
||||
|
||||
if (!column_string
|
||||
|| (!is_date && !is_date_time))
|
||||
throw Exception("Illegal columns " + col_left_untyped->getName() + " and " + col_right_untyped->getName()
|
||||
+ " of arguments of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
if (is_date)
|
||||
{
|
||||
DayNum_t date;
|
||||
ReadBufferFromString in(column_string->getData());
|
||||
readDateText(date, in);
|
||||
if (!in.eof())
|
||||
throw Exception("String is too long for Date: " + column_string->getData());
|
||||
|
||||
ColumnConst<DataTypeDate::FieldType> parsed_const_date(block.rowsInFirstColumn(), date);
|
||||
executeNumLeftType<DataTypeDate::FieldType>(block, result,
|
||||
left_is_num ? col_left_untyped : &parsed_const_date,
|
||||
left_is_num ? &parsed_const_date : col_right_untyped);
|
||||
}
|
||||
else if (is_date_time)
|
||||
{
|
||||
time_t date_time;
|
||||
ReadBufferFromString in(column_string->getData());
|
||||
readDateTimeText(date_time, in);
|
||||
if (!in.eof())
|
||||
throw Exception("String is too long for DateTime: " + column_string->getData());
|
||||
|
||||
ColumnConst<DataTypeDateTime::FieldType> parsed_const_date_time(block.rowsInFirstColumn(), date_time);
|
||||
executeNumLeftType<DataTypeDateTime::FieldType>(block, result,
|
||||
left_is_num ? col_left_untyped : &parsed_const_date_time,
|
||||
left_is_num ? &parsed_const_date_time : col_right_untyped);
|
||||
}
|
||||
}
|
||||
|
||||
public:
|
||||
/// Получить имя функции.
|
||||
String getName() const
|
||||
@ -581,12 +635,36 @@ public:
|
||||
+ toString(arguments.size()) + ", should be 2.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!( ( arguments[0]->isNumeric() && arguments[0]->behavesAsNumber()
|
||||
&& arguments[1]->isNumeric() && arguments[1]->behavesAsNumber())
|
||||
|| ( (arguments[0]->getName() == "String" || arguments[0]->getName().substr(0, 11) == "FixedString")
|
||||
&& (arguments[1]->getName() == "String" || arguments[1]->getName().substr(0, 11) == "FixedString"))
|
||||
|| (arguments[0]->getName() == "Date" && arguments[1]->getName() == "Date")
|
||||
|| (arguments[0]->getName() == "DateTime" && arguments[1]->getName() == "DateTime")))
|
||||
bool left_is_date = false;
|
||||
bool left_is_date_time = false;
|
||||
bool left_is_string = false;
|
||||
bool left_is_fixed_string = false;
|
||||
|
||||
false
|
||||
|| (left_is_date = typeid_cast<const DataTypeDate *>(arguments[0].get()))
|
||||
|| (left_is_date_time = typeid_cast<const DataTypeDateTime *>(arguments[0].get()))
|
||||
|| (left_is_string = typeid_cast<const DataTypeString *>(arguments[0].get()))
|
||||
|| (left_is_fixed_string = typeid_cast<const DataTypeFixedString *>(arguments[0].get()));
|
||||
|
||||
bool right_is_date = false;
|
||||
bool right_is_date_time = false;
|
||||
bool right_is_string = false;
|
||||
bool right_is_fixed_string = false;
|
||||
|
||||
false
|
||||
|| (right_is_date = typeid_cast<const DataTypeDate *>(arguments[1].get()))
|
||||
|| (right_is_date_time = typeid_cast<const DataTypeDateTime *>(arguments[1].get()))
|
||||
|| (right_is_string = typeid_cast<const DataTypeString *>(arguments[1].get()))
|
||||
|| (right_is_fixed_string = typeid_cast<const DataTypeFixedString *>(arguments[1].get()));
|
||||
|
||||
if (!( (arguments[0]->behavesAsNumber() && arguments[1]->behavesAsNumber())
|
||||
|| ((left_is_string || left_is_fixed_string) && (right_is_string || right_is_fixed_string))
|
||||
|| (left_is_date && right_is_date)
|
||||
|| (left_is_date && right_is_string) /// Можно сравнивать дату и дату-с-временем с константной строкой.
|
||||
|| (left_is_string && right_is_date)
|
||||
|| (left_is_date_time && right_is_date_time)
|
||||
|| (left_is_date_time && right_is_string)
|
||||
|| (left_is_string && right_is_date_time)))
|
||||
throw Exception("Illegal types of arguments (" + arguments[0]->getName() + ", " + arguments[1]->getName() + ")"
|
||||
" of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -596,24 +674,36 @@ public:
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (block.getByPosition(arguments[0]).column->isNumeric())
|
||||
const IColumn * col_left_untyped = block.getByPosition(arguments[0]).column.get();
|
||||
const IColumn * col_right_untyped = block.getByPosition(arguments[1]).column.get();
|
||||
|
||||
bool left_is_num = col_left_untyped->isNumeric();
|
||||
bool right_is_num = col_right_untyped->isNumeric();
|
||||
|
||||
if (left_is_num && right_is_num)
|
||||
{
|
||||
if (!( executeNumLeftType<UInt8>(block, arguments, result)
|
||||
|| executeNumLeftType<UInt16>(block, arguments, result)
|
||||
|| executeNumLeftType<UInt32>(block, arguments, result)
|
||||
|| executeNumLeftType<UInt64>(block, arguments, result)
|
||||
|| executeNumLeftType<Int8>(block, arguments, result)
|
||||
|| executeNumLeftType<Int16>(block, arguments, result)
|
||||
|| executeNumLeftType<Int32>(block, arguments, result)
|
||||
|| executeNumLeftType<Int64>(block, arguments, result)
|
||||
|| executeNumLeftType<Float32>(block, arguments, result)
|
||||
|| executeNumLeftType<Float64>(block, arguments, result)))
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
if (!( executeNumLeftType<UInt8>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<UInt16>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<UInt32>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<UInt64>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Int8>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Int16>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Int32>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Int64>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Float32>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Float64>(block, result, col_left_untyped, col_right_untyped)))
|
||||
throw Exception("Illegal column " + col_left_untyped->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else if (!left_is_num && !right_is_num)
|
||||
{
|
||||
executeString(block, result, col_left_untyped, col_right_untyped);
|
||||
}
|
||||
else
|
||||
executeString(block, arguments, result);
|
||||
{
|
||||
executeDateOrDateTimeWithConstString(block, result, col_left_untyped, col_right_untyped, left_is_num, right_is_num);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -202,46 +202,11 @@ struct LengthUTF8Impl
|
||||
};
|
||||
|
||||
|
||||
/** Переводит строку в нижний (верхний) регистр, в текущей локали, в однобайтовой кодировке.
|
||||
*/
|
||||
template <int F(int)>
|
||||
template <char not_case_lower_bound, char not_case_upper_bound>
|
||||
struct LowerUpperImpl
|
||||
{
|
||||
static void vector(const ColumnString::Chars_t & data, const ColumnString::Offsets_t & offsets,
|
||||
ColumnString::Chars_t & res_data, ColumnString::Offsets_t & res_offsets)
|
||||
{
|
||||
res_data.resize(data.size());
|
||||
res_offsets.assign(offsets);
|
||||
array(&*data.begin(), &*data.end(), &*res_data.begin());
|
||||
}
|
||||
|
||||
static void vector_fixed(const ColumnString::Chars_t & data, size_t n,
|
||||
ColumnString::Chars_t & res_data)
|
||||
{
|
||||
res_data.resize(data.size());
|
||||
array(&*data.begin(), &*data.end(), &*res_data.begin());
|
||||
}
|
||||
|
||||
static void constant(const std::string & data, std::string & res_data)
|
||||
{
|
||||
res_data.resize(data.size());
|
||||
array(reinterpret_cast<const UInt8 *>(&*data.begin()), reinterpret_cast<const UInt8 *>(&*data.end()),
|
||||
reinterpret_cast<UInt8 *>(&*res_data.begin()));
|
||||
}
|
||||
|
||||
private:
|
||||
static void array(const UInt8 * src, const UInt8 * src_end, UInt8 * dst)
|
||||
{
|
||||
for (; src < src_end; ++src, ++dst)
|
||||
*dst = F(*src);
|
||||
}
|
||||
};
|
||||
|
||||
template <char not_case_lower_bound, char not_case_upper_bound>
|
||||
struct LowerUpperImplVectorized
|
||||
{
|
||||
static void vector(const ColumnString::Chars_t & data, const ColumnString::Offsets_t & offsets,
|
||||
ColumnString::Chars_t & res_data, ColumnString::Offsets_t & res_offsets)
|
||||
{
|
||||
res_data.resize(data.size());
|
||||
res_offsets.assign(offsets);
|
||||
@ -348,9 +313,14 @@ inline void UTF8CyrillicToCase(const UInt8 * & src, const UInt8 * const src_end,
|
||||
}
|
||||
};
|
||||
|
||||
/** Если строка содержит текст в кодировке UTF-8 - перевести его в нижний (верхний) регистр.
|
||||
* Замечание: предполагается, что после перевода символа в другой регистр,
|
||||
* длина его мультибайтовой последовательности в UTF-8 не меняется.
|
||||
* Иначе - поведение не определено.
|
||||
*/
|
||||
template <char not_case_lower_bound, char not_case_upper_bound,
|
||||
int to_case(int), void cyrillic_to_case(const UInt8 * &, const UInt8 *, UInt8 * &)>
|
||||
struct LowerUpperUTF8ImplVectorized
|
||||
struct LowerUpperUTF8Impl
|
||||
{
|
||||
static void vector(const ColumnString::Chars_t & data, const ColumnString::Offsets_t & offsets,
|
||||
ColumnString::Chars_t & res_data, ColumnString::Offsets_t & res_offsets)
|
||||
@ -487,59 +457,6 @@ private:
|
||||
};
|
||||
|
||||
|
||||
/** Если строка содержит текст в кодировке UTF-8 - перевести его в нижний (верхний) регистр.
|
||||
* Замечание: предполагается, что после перевода символа в другой регистр,
|
||||
* длина его мультибайтовой последовательности в UTF-8 не меняется.
|
||||
* Иначе - поведение не определено.
|
||||
*/
|
||||
template <int F(int)>
|
||||
struct LowerUpperUTF8Impl
|
||||
{
|
||||
static void vector(const ColumnString::Chars_t & data, const ColumnString::Offsets_t & offsets,
|
||||
ColumnString::Chars_t & res_data, ColumnString::Offsets_t & res_offsets)
|
||||
{
|
||||
res_data.resize(data.size());
|
||||
res_offsets.assign(offsets);
|
||||
array(&*data.begin(), &*data.end(), &*res_data.begin());
|
||||
}
|
||||
|
||||
static void vector_fixed(const ColumnString::Chars_t & data, size_t n,
|
||||
ColumnString::Chars_t & res_data)
|
||||
{
|
||||
res_data.resize(data.size());
|
||||
array(&*data.begin(), &*data.end(), &*res_data.begin());
|
||||
}
|
||||
|
||||
static void constant(const std::string & data, std::string & res_data)
|
||||
{
|
||||
res_data.resize(data.size());
|
||||
array(reinterpret_cast<const UInt8 *>(&*data.begin()), reinterpret_cast<const UInt8 *>(&*data.end()),
|
||||
reinterpret_cast<UInt8 *>(&*res_data.begin()));
|
||||
}
|
||||
|
||||
private:
|
||||
static void array(const UInt8 * src, const UInt8 * src_end, UInt8 * dst)
|
||||
{
|
||||
static Poco::UTF8Encoding utf8;
|
||||
|
||||
while (src < src_end)
|
||||
{
|
||||
int chars = utf8.convert(F(utf8.convert(src)), dst, src_end - src);
|
||||
if (chars)
|
||||
{
|
||||
src += chars;
|
||||
dst += chars;
|
||||
}
|
||||
else
|
||||
{
|
||||
++src;
|
||||
++dst;
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/** Разворачивает строку в байтах.
|
||||
*/
|
||||
struct ReverseImpl
|
||||
@ -1676,32 +1593,22 @@ struct NameReverseUTF8 { static constexpr auto name = "reverseUTF8"; };
|
||||
struct NameSubstring { static constexpr auto name = "substring"; };
|
||||
struct NameSubstringUTF8 { static constexpr auto name = "substringUTF8"; };
|
||||
|
||||
struct NameSSELower { static constexpr auto name = "sse_lower"; };
|
||||
struct NameSSEUpper { static constexpr auto name = "sse_upper"; };
|
||||
struct NameSSELowerUTF8 { static constexpr auto name = "sse_lowerUTF8"; };
|
||||
struct NameSSEUpperUTF8 { static constexpr auto name = "sse_upperUTF8"; };
|
||||
|
||||
typedef FunctionStringOrArrayToT<EmptyImpl<false>, NameEmpty, UInt8> FunctionEmpty;
|
||||
typedef FunctionStringOrArrayToT<EmptyImpl<true>, NameNotEmpty, UInt8> FunctionNotEmpty;
|
||||
typedef FunctionStringOrArrayToT<LengthImpl, NameLength, UInt64> FunctionLength;
|
||||
typedef FunctionStringOrArrayToT<LengthUTF8Impl, NameLengthUTF8, UInt64> FunctionLengthUTF8;
|
||||
typedef FunctionStringToString<LowerUpperImpl<tolower>, NameLower> FunctionLower;
|
||||
typedef FunctionStringToString<LowerUpperImpl<toupper>, NameUpper> FunctionUpper;
|
||||
typedef FunctionStringToString<LowerUpperUTF8Impl<Poco::Unicode::toLower>, NameLowerUTF8> FunctionLowerUTF8;
|
||||
typedef FunctionStringToString<LowerUpperUTF8Impl<Poco::Unicode::toUpper>, NameUpperUTF8> FunctionUpperUTF8;
|
||||
typedef FunctionStringToString<LowerUpperImpl<'A', 'Z'>, NameLower> FunctionLower;
|
||||
typedef FunctionStringToString<LowerUpperImpl<'a', 'z'>, NameUpper> FunctionUpper;
|
||||
typedef FunctionStringToString<
|
||||
LowerUpperUTF8Impl<'A', 'Z', Poco::Unicode::toLower, UTF8CyrillicToCase<true>>,
|
||||
NameLowerUTF8> FunctionLowerUTF8;
|
||||
typedef FunctionStringToString<
|
||||
LowerUpperUTF8Impl<'a', 'z', Poco::Unicode::toUpper, UTF8CyrillicToCase<false>>,
|
||||
NameUpperUTF8> FunctionUpperUTF8;
|
||||
typedef FunctionStringToString<ReverseImpl, NameReverse> FunctionReverse;
|
||||
typedef FunctionStringToString<ReverseUTF8Impl, NameReverseUTF8> FunctionReverseUTF8;
|
||||
typedef FunctionStringNumNumToString<SubstringImpl, NameSubstring> FunctionSubstring;
|
||||
typedef FunctionStringNumNumToString<SubstringUTF8Impl, NameSubstringUTF8> FunctionSubstringUTF8;
|
||||
|
||||
using FunctionSSELower = FunctionStringToString<LowerUpperImplVectorized<'A', 'Z'>, NameSSELower>;
|
||||
using FunctionSSEUpper = FunctionStringToString<LowerUpperImplVectorized<'a', 'z'>, NameSSEUpper>;
|
||||
using FunctionSSELowerUTF8 = FunctionStringToString<
|
||||
LowerUpperUTF8ImplVectorized<'A', 'Z', Poco::Unicode::toLower, UTF8CyrillicToCase<true>>,
|
||||
NameSSELowerUTF8>;
|
||||
using FunctionSSEUpperUTF8 = FunctionStringToString<
|
||||
LowerUpperUTF8ImplVectorized<'a', 'z', Poco::Unicode::toUpper, UTF8CyrillicToCase<false>>,
|
||||
NameSSEUpperUTF8>;
|
||||
|
||||
|
||||
}
|
||||
|
@ -1,23 +1,23 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Dictionaries/IDictionary.h>
|
||||
#include <DB/Core/Exception.h>
|
||||
#include <DB/Core/ErrorCodes.h>
|
||||
#include <Yandex/MultiVersion.h>
|
||||
#include <Yandex/logger_useful.h>
|
||||
#include <Poco/Event.h>
|
||||
#include <unistd.h>
|
||||
#include <time.h>
|
||||
#include <mutex>
|
||||
#include <thread>
|
||||
#include <unordered_map>
|
||||
#include <chrono>
|
||||
#include <random>
|
||||
#include <unistd.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
class IDictionary;
|
||||
|
||||
/** Manages user-defined dictionaries.
|
||||
* Monitors configuration file and automatically reloads dictionaries in a separate thread.
|
||||
@ -50,8 +50,16 @@ private:
|
||||
std::exception_ptr exception;
|
||||
};
|
||||
|
||||
struct failed_dictionary_info final
|
||||
{
|
||||
std::unique_ptr<IDictionary> dict;
|
||||
std::chrono::system_clock::time_point next_attempt_time;
|
||||
std::uint64_t error_count;
|
||||
};
|
||||
|
||||
std::unordered_map<std::string, dictionary_info> dictionaries;
|
||||
std::unordered_map<std::string, std::chrono::system_clock::time_point> update_times;
|
||||
std::unordered_map<std::string, failed_dictionary_info> failed_dictionaries;
|
||||
std::mt19937_64 rnd_engine{getSeed()};
|
||||
|
||||
Context & context;
|
||||
@ -81,7 +89,7 @@ private:
|
||||
{
|
||||
timespec ts;
|
||||
clock_gettime(CLOCK_MONOTONIC, &ts);
|
||||
return ts.tv_nsec ^ getpid();
|
||||
return static_cast<std::uint64_t>(ts.tv_nsec ^ getpid());
|
||||
}
|
||||
|
||||
public:
|
||||
|
@ -222,7 +222,8 @@ private:
|
||||
bool keys_fit_128_bits;
|
||||
Sizes key_sizes;
|
||||
|
||||
Block sample_block;
|
||||
Block sample_block_with_columns_to_add;
|
||||
Block sample_block_with_keys;
|
||||
|
||||
Logger * log;
|
||||
|
||||
@ -250,6 +251,9 @@ private:
|
||||
|
||||
/// Проверить не превышены ли допустимые размеры множества
|
||||
bool checkSizeLimits() const;
|
||||
|
||||
/// Кинуть исключение, если в блоках не совпадают типы ключей.
|
||||
void checkTypesOfKeys(const Block & block_left, const Block & block_right) const;
|
||||
};
|
||||
|
||||
typedef Poco::SharedPtr<Join> JoinPtr;
|
||||
|
@ -283,7 +283,7 @@ public:
|
||||
* node - это список значений: 1, 2, 3 или список tuple-ов: (1, 2), (3, 4), (5, 6).
|
||||
* create_ordered_set - создавать ли вектор упорядоченных элементов. Нужен для работы индекса
|
||||
*/
|
||||
void createFromAST(DataTypes & types, ASTPtr node, bool create_ordered_set);
|
||||
void createFromAST(DataTypes & types, ASTPtr node, const Context & context, bool create_ordered_set);
|
||||
|
||||
// Возвращает false, если превышено какое-нибудь ограничение, и больше не нужно вставлять.
|
||||
bool insertFromBlock(const Block & block, bool create_ordered_set = false);
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <DB/Core/Exception.h>
|
||||
#include <DB/Core/ErrorCodes.h>
|
||||
#include <DB/IO/WriteHelpers.h>
|
||||
#include <DB/Common/formatReadable.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -116,8 +117,8 @@ public:
|
||||
{
|
||||
size_t free_bytes = getUnreservedFreeSpace(path);
|
||||
if (free_bytes < size)
|
||||
throw Exception("Not enough free disk space to reserve: " + toString(free_bytes) + " available, "
|
||||
+ toString(size) + " requested", ErrorCodes::NOT_ENOUGH_SPACE);
|
||||
throw Exception("Not enough free disk space to reserve: " + formatReadableSizeWithBinarySuffix(free_bytes) + " available, "
|
||||
+ formatReadableSizeWithBinarySuffix(size) + " requested", ErrorCodes::NOT_ENOUGH_SPACE);
|
||||
return new Reservation(size);
|
||||
}
|
||||
|
||||
|
@ -43,9 +43,7 @@ public:
|
||||
{
|
||||
pre_column_names = prewhere_actions->getRequiredColumns();
|
||||
|
||||
/// @todo somehow decide which injected columns belong to PREWHERE, optimizing reads
|
||||
pre_column_names.insert(std::end(pre_column_names),
|
||||
std::begin(injected_columns), std::end(injected_columns));
|
||||
injectRequiredColumns(pre_column_names);
|
||||
|
||||
if (pre_column_names.empty())
|
||||
pre_column_names.push_back(column_names[0]);
|
||||
|
@ -49,7 +49,7 @@ public:
|
||||
DiskSpaceMonitor::Reservation * disk_reservation = nullptr);
|
||||
|
||||
/// Примерное количество места на диске, нужное для мерджа. С запасом.
|
||||
size_t estimateDiskSpaceForMerge(const MergeTreeData::DataPartsVector & parts);
|
||||
static size_t estimateDiskSpaceForMerge(const MergeTreeData::DataPartsVector & parts);
|
||||
|
||||
/** Отменяет все мерджи. Все выполняющиеся сейчас вызовы mergeParts скоро бросят исключение.
|
||||
* Все новые вызовы будут бросать исключения, пока не будет вызван uncancelAll().
|
||||
|
@ -48,7 +48,11 @@ public:
|
||||
/** num_shards - уровень внутреннего параллелизма (количество независимых буферов)
|
||||
* Буфер сбрасывается, если превышены все минимальные пороги или хотя бы один из максимальных.
|
||||
*/
|
||||
static StoragePtr create(const std::string & name_, NamesAndTypesListPtr columns_, Context & context_,
|
||||
static StoragePtr create(const std::string & name_, NamesAndTypesListPtr columns_,
|
||||
const NamesAndTypesList & materialized_columns_,
|
||||
const NamesAndTypesList & alias_columns_,
|
||||
const ColumnDefaults & column_defaults_,
|
||||
Context & context_,
|
||||
size_t num_shards_, const Thresholds & min_thresholds_, const Thresholds & max_thresholds_,
|
||||
const String & destination_database_, const String & destination_table_);
|
||||
|
||||
@ -113,7 +117,11 @@ private:
|
||||
/// Выполняет сброс данных по таймауту.
|
||||
std::thread flush_thread;
|
||||
|
||||
StorageBuffer(const std::string & name_, NamesAndTypesListPtr columns_, Context & context_,
|
||||
StorageBuffer(const std::string & name_, NamesAndTypesListPtr columns_,
|
||||
const NamesAndTypesList & materialized_columns_,
|
||||
const NamesAndTypesList & alias_columns_,
|
||||
const ColumnDefaults & column_defaults_,
|
||||
Context & context_,
|
||||
size_t num_shards_, const Thresholds & min_thresholds_, const Thresholds & max_thresholds_,
|
||||
const String & destination_database_, const String & destination_table_);
|
||||
|
||||
|
71
dbms/src/Common/tests/sip_hash_perf.cpp
Normal file
71
dbms/src/Common/tests/sip_hash_perf.cpp
Normal file
@ -0,0 +1,71 @@
|
||||
#include <vector>
|
||||
#include <string>
|
||||
#include <iomanip>
|
||||
|
||||
#include <DB/Common/SipHash.h>
|
||||
#include <DB/IO/ReadBufferFromFileDescriptor.h>
|
||||
#include <DB/IO/ReadHelpers.h>
|
||||
#include <statdaemons/Stopwatch.h>
|
||||
|
||||
|
||||
/** Тестировать так:
|
||||
*
|
||||
* clickhouse-client --query="SELECT SearchPhrase AS k FROM test.hits WHERE k != ''" > phrases.tsv
|
||||
* clickhouse-client --query="SELECT URL AS k FROM test.hits" > urls.tsv
|
||||
* clickhouse-client --query="SELECT SearchPhrase AS k FROM test.hits" > phrases_with_empty.tsv
|
||||
* clickhouse-client --query="SELECT Title AS k FROM test.hits" > titles.tsv
|
||||
* clickhouse-client --query="SELECT PageCharset AS k FROM test.hits" > charset.tsv
|
||||
*
|
||||
* for i in {1..1000}; do ./sip_hash_perf < titles.tsv 2>&1 | grep Processed | grep -oP '\d+\.\d+ rows/sec'; done | awk '{ if ($1 > x) { x = $1; print x } }'
|
||||
*/
|
||||
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
{
|
||||
std::vector<std::string> data;
|
||||
DB::ReadBufferFromFileDescriptor in(STDIN_FILENO);
|
||||
|
||||
std::cerr << std::fixed << std::setprecision(3);
|
||||
|
||||
{
|
||||
Stopwatch watch;
|
||||
|
||||
while (!in.eof())
|
||||
{
|
||||
data.emplace_back();
|
||||
DB::readEscapedString(data.back(), in);
|
||||
DB::assertString("\n", in);
|
||||
}
|
||||
|
||||
double seconds = watch.elapsedSeconds();
|
||||
std::cerr << "Read "
|
||||
<< data.size() << " rows, "
|
||||
<< (in.count() / 1048576.0) << " MiB "
|
||||
<< " in " << seconds << " sec., "
|
||||
<< (data.size() / seconds) << " rows/sec., "
|
||||
<< (in.count() / 1048576.0 / seconds) << " MiB/sec.\n";
|
||||
}
|
||||
|
||||
{
|
||||
size_t res = 0;
|
||||
Stopwatch watch;
|
||||
|
||||
for (const auto & s : data)
|
||||
{
|
||||
SipHash hash;
|
||||
hash.update(s.data(), s.size());
|
||||
res += hash.get64();
|
||||
}
|
||||
|
||||
double seconds = watch.elapsedSeconds();
|
||||
std::cerr << "Processed "
|
||||
<< data.size() << " rows, "
|
||||
<< (in.count() / 1048576.0) << " MiB "
|
||||
<< " in " << seconds << " sec., "
|
||||
<< (data.size() / seconds) << " rows/sec., "
|
||||
<< (in.count() / 1048576.0 / seconds) << " MiB/sec. "
|
||||
<< "(res = " << res << ")\n";
|
||||
}
|
||||
|
||||
return 0;
|
||||
}
|
@ -20,10 +20,6 @@ void registerFunctionsString(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionSubstring>();
|
||||
factory.registerFunction<FunctionSubstringUTF8>();
|
||||
factory.registerFunction<FunctionAppendTrailingCharIfAbsent>();
|
||||
factory.registerFunction<FunctionSSELower>();
|
||||
factory.registerFunction<FunctionSSEUpper>();
|
||||
factory.registerFunction<FunctionSSELowerUTF8>();
|
||||
factory.registerFunction<FunctionSSEUpperUTF8>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -124,12 +124,13 @@ void ExpressionAction::prepare(Block & sample_block)
|
||||
/// Если все аргументы и требуемые столбцы - константы, выполним функцию.
|
||||
if (all_const)
|
||||
{
|
||||
size_t result_position = sample_block.columns();
|
||||
|
||||
ColumnWithNameAndType new_column;
|
||||
new_column.name = result_name;
|
||||
new_column.type = result_type;
|
||||
sample_block.insert(new_column);
|
||||
|
||||
size_t result_position = sample_block.getPositionByName(result_name);
|
||||
function->execute(sample_block, arguments, prerequisites, result_position);
|
||||
|
||||
/// Если получилась не константа, на всякий случай будем считать результат неизвестным.
|
||||
|
@ -922,16 +922,24 @@ void ExpressionAnalyzer::makeExplicitSet(ASTFunction * node, const Block & sampl
|
||||
|
||||
if (ASTFunction * set_func = typeid_cast<ASTFunction *>(&*arg))
|
||||
{
|
||||
if (set_func->name != "tuple")
|
||||
throw Exception("Incorrect type of 2nd argument for function " + node->name + ". Must be subquery or set of values.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
/// Отличм случай (x, y) in ((1, 2), (3, 4)) от случая (x, y) in (1, 2).
|
||||
ASTFunction * any_element = typeid_cast<ASTFunction *>(&*set_func->arguments->children.at(0));
|
||||
if (set_element_types.size() >= 2 && (!any_element || any_element->name != "tuple"))
|
||||
single_value = true;
|
||||
if (set_func->name == "tuple")
|
||||
{
|
||||
/// Отличм случай (x, y) in ((1, 2), (3, 4)) от случая (x, y) in (1, 2).
|
||||
ASTFunction * any_element = typeid_cast<ASTFunction *>(&*set_func->arguments->children.at(0));
|
||||
if (set_element_types.size() >= 2 && (!any_element || any_element->name != "tuple"))
|
||||
single_value = true;
|
||||
else
|
||||
elements_ast = set_func->arguments;
|
||||
}
|
||||
else
|
||||
elements_ast = set_func->arguments;
|
||||
{
|
||||
if (set_element_types.size() >= 2)
|
||||
throw Exception("Incorrect type of 2nd argument for function " + node->name
|
||||
+ ". Must be subquery or set of " + toString(set_element_types.size()) + "-element tuples.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
single_value = true;
|
||||
}
|
||||
}
|
||||
else if (typeid_cast<ASTLiteral *>(&*arg))
|
||||
{
|
||||
@ -954,7 +962,7 @@ void ExpressionAnalyzer::makeExplicitSet(ASTFunction * node, const Block & sampl
|
||||
ASTPtr ast_set_ptr = ast_set;
|
||||
ast_set->set = new Set(settings.limits);
|
||||
ast_set->is_explicit = true;
|
||||
ast_set->set->createFromAST(set_element_types, elements_ast, create_ordered_set);
|
||||
ast_set->set->createFromAST(set_element_types, elements_ast, context, create_ordered_set);
|
||||
arg = ast_set_ptr;
|
||||
}
|
||||
|
||||
|
@ -6,6 +6,15 @@
|
||||
#include <Poco/Util/Application.h>
|
||||
#include <Poco/Glob.h>
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
/// 5 seconds
|
||||
const auto backoff_initial_sec = 5;
|
||||
/// 10 minutes
|
||||
const auto backoff_max_sec = 10 * 60;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -41,6 +50,63 @@ void ExternalDictionaries::reloadImpl(const bool throw_on_error)
|
||||
for (const auto & config_path : config_paths)
|
||||
reloadFromFile(config_path, throw_on_error);
|
||||
|
||||
/// list of recreated dictionaries to perform delayed removal from unordered_map
|
||||
std::list<std::string> recreated_failed_dictionaries;
|
||||
|
||||
/// retry loading failed dictionaries
|
||||
for (auto & failed_dictionary : failed_dictionaries)
|
||||
{
|
||||
if (std::chrono::system_clock::now() < failed_dictionary.second.next_attempt_time)
|
||||
continue;
|
||||
|
||||
const auto & name = failed_dictionary.first;
|
||||
|
||||
try
|
||||
{
|
||||
auto dict_ptr = failed_dictionary.second.dict->clone();
|
||||
if (dict_ptr->getCreationException())
|
||||
{
|
||||
/// recalculate next attempt time
|
||||
std::uniform_int_distribution<std::uint64_t> distribution(
|
||||
0, std::exp2(failed_dictionary.second.error_count));
|
||||
|
||||
failed_dictionary.second.next_attempt_time = std::chrono::system_clock::now() +
|
||||
std::chrono::seconds{
|
||||
std::min<std::uint64_t>(backoff_max_sec, backoff_initial_sec + distribution(rnd_engine))
|
||||
};
|
||||
|
||||
++failed_dictionary.second.error_count;
|
||||
}
|
||||
else
|
||||
{
|
||||
const std::lock_guard<std::mutex> lock{dictionaries_mutex};
|
||||
|
||||
const auto dict_it = dictionaries.find(name);
|
||||
if (dict_it->second.dict)
|
||||
dict_it->second.dict->set(dict_ptr.release());
|
||||
else
|
||||
dict_it->second.dict = std::make_shared<MultiVersion<IDictionary>>(dict_ptr.release());
|
||||
|
||||
/// erase stored exception on success
|
||||
dict_it->second.exception = std::exception_ptr{};
|
||||
|
||||
const auto & lifetime = dict_ptr->getLifetime();
|
||||
std::uniform_int_distribution<std::uint64_t> distribution{lifetime.min_sec, lifetime.max_sec};
|
||||
update_times[name] = std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)};
|
||||
|
||||
recreated_failed_dictionaries.push_back(name);
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
LOG_ERROR(log, "Failed reloading " << name << " dictionary due to unexpected error");
|
||||
}
|
||||
}
|
||||
|
||||
/// do not undertake further attempts to recreate these dictionaries
|
||||
for (const auto & name : recreated_failed_dictionaries)
|
||||
failed_dictionaries.erase(name);
|
||||
|
||||
/// periodic update
|
||||
for (auto & dictionary : dictionaries)
|
||||
{
|
||||
@ -122,10 +188,10 @@ void ExternalDictionaries::reloadFromFile(const std::string & config_path, const
|
||||
}
|
||||
else
|
||||
{
|
||||
auto it = last_modification_times.find(config_path);
|
||||
if (it == std::end(last_modification_times))
|
||||
it = last_modification_times.emplace(config_path, Poco::Timestamp{0}).first;
|
||||
auto & config_last_modified = it->second;
|
||||
auto modification_time_it = last_modification_times.find(config_path);
|
||||
if (modification_time_it == std::end(last_modification_times))
|
||||
modification_time_it = last_modification_times.emplace(config_path, Poco::Timestamp{0}).first;
|
||||
auto & config_last_modified = modification_time_it->second;
|
||||
|
||||
const auto last_modified = config_file.getLastModified();
|
||||
if (last_modified > config_last_modified)
|
||||
@ -163,12 +229,31 @@ void ExternalDictionaries::reloadFromFile(const std::string & config_path, const
|
||||
continue;
|
||||
}
|
||||
|
||||
auto it = dictionaries.find(name);
|
||||
if (it != std::end(dictionaries))
|
||||
if (it->second.origin != config_path)
|
||||
throw std::runtime_error{"Overriding dictionary from file " + it->second.origin};
|
||||
const auto dict_it = dictionaries.find(name);
|
||||
if (dict_it != std::end(dictionaries))
|
||||
if (dict_it->second.origin != config_path)
|
||||
throw std::runtime_error{"Overriding dictionary from file " + dict_it->second.origin};
|
||||
|
||||
auto dict_ptr = DictionaryFactory::instance().create(name, *config, key, context);
|
||||
if (const auto exception_ptr = dict_ptr->getCreationException())
|
||||
{
|
||||
const auto failed_dict_it = failed_dictionaries.find(name);
|
||||
if (failed_dict_it != std::end(failed_dictionaries))
|
||||
{
|
||||
failed_dict_it->second = failed_dictionary_info{
|
||||
std::move(dict_ptr),
|
||||
std::chrono::system_clock::now() + std::chrono::seconds{backoff_initial_sec}
|
||||
};
|
||||
}
|
||||
else
|
||||
failed_dictionaries.emplace(name, failed_dictionary_info{
|
||||
std::move(dict_ptr),
|
||||
std::chrono::system_clock::now() + std::chrono::seconds{backoff_initial_sec}
|
||||
});
|
||||
|
||||
std::rethrow_exception(exception_ptr);
|
||||
}
|
||||
|
||||
if (!dict_ptr->isCached())
|
||||
{
|
||||
const auto & lifetime = dict_ptr->getLifetime();
|
||||
@ -183,42 +268,38 @@ void ExternalDictionaries::reloadFromFile(const std::string & config_path, const
|
||||
}
|
||||
}
|
||||
|
||||
const std::lock_guard<std::mutex> lock{dictionaries_mutex};
|
||||
|
||||
/// add new dictionary or update an existing version
|
||||
if (it == std::end(dictionaries))
|
||||
{
|
||||
const std::lock_guard<std::mutex> lock{dictionaries_mutex};
|
||||
if (dict_it == std::end(dictionaries))
|
||||
dictionaries.emplace(name, dictionary_info{
|
||||
std::make_shared<MultiVersion<IDictionary>>(dict_ptr.release()),
|
||||
config_path
|
||||
});
|
||||
}
|
||||
else
|
||||
{
|
||||
if (it->second.dict)
|
||||
it->second.dict->set(dict_ptr.release());
|
||||
if (dict_it->second.dict)
|
||||
dict_it->second.dict->set(dict_ptr.release());
|
||||
else
|
||||
{
|
||||
const std::lock_guard<std::mutex> lock{dictionaries_mutex};
|
||||
it->second.dict = std::make_shared<MultiVersion<IDictionary>>(dict_ptr.release());
|
||||
}
|
||||
dict_it->second.dict = std::make_shared<MultiVersion<IDictionary>>(dict_ptr.release());
|
||||
|
||||
/// erase stored exception on success
|
||||
it->second.exception = std::exception_ptr{};
|
||||
dict_it->second.exception = std::exception_ptr{};
|
||||
failed_dictionaries.erase(name);
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
if (!name.empty())
|
||||
{
|
||||
const std::lock_guard<std::mutex> lock{dictionaries_mutex};
|
||||
|
||||
const auto exception_ptr = std::current_exception();
|
||||
const auto it = dictionaries.find(name);
|
||||
if (it == std::end(dictionaries))
|
||||
{
|
||||
const std::lock_guard<std::mutex> lock{dictionaries_mutex};
|
||||
const auto dict_it = dictionaries.find(name);
|
||||
if (dict_it == std::end(dictionaries))
|
||||
dictionaries.emplace(name, dictionary_info{nullptr, config_path, exception_ptr});
|
||||
}
|
||||
else
|
||||
it->second.exception = exception_ptr;
|
||||
dict_it->second.exception = exception_ptr;
|
||||
}
|
||||
|
||||
try
|
||||
@ -253,16 +334,17 @@ void ExternalDictionaries::reloadFromFile(const std::string & config_path, const
|
||||
MultiVersion<IDictionary>::Version ExternalDictionaries::getDictionary(const std::string & name) const
|
||||
{
|
||||
const std::lock_guard<std::mutex> lock{dictionaries_mutex};
|
||||
const auto it = dictionaries.find(name);
|
||||
|
||||
const auto it = dictionaries.find(name);
|
||||
if (it == std::end(dictionaries))
|
||||
throw Exception{
|
||||
"No such dictionary: " + name,
|
||||
ErrorCodes::BAD_ARGUMENTS
|
||||
};
|
||||
|
||||
if (!it->second.dict && it->second.exception)
|
||||
std::rethrow_exception(it->second.exception);
|
||||
if (!it->second.dict)
|
||||
it->second.exception ? std::rethrow_exception(it->second.exception) :
|
||||
throw Exception{"No dictionary", ErrorCodes::LOGICAL_ERROR};
|
||||
|
||||
return it->second.dict->get();
|
||||
}
|
||||
|
@ -12,7 +12,6 @@ InterpreterCheckQuery::InterpreterCheckQuery(DB::ASTPtr query_ptr_, DB::Context&
|
||||
|
||||
BlockInputStreamPtr InterpreterCheckQuery::execute()
|
||||
{
|
||||
/// @TODO
|
||||
ASTCheckQuery & alter = typeid_cast<ASTCheckQuery &>(*query_ptr);
|
||||
String & table_name = alter.table;
|
||||
String database_name = alter.database.empty() ? context.getCurrentDatabase() : alter.database;
|
||||
|
@ -339,15 +339,19 @@ void Join::setSampleBlock(const Block & block)
|
||||
/// Выберем, какую структуру данных для множества использовать.
|
||||
init(chooseMethod(key_columns, keys_fit_128_bits, key_sizes));
|
||||
|
||||
sample_block = block;
|
||||
sample_block_with_columns_to_add = block;
|
||||
|
||||
/// Удаляем из sample_block ключевые столбцы, так как они не нужны.
|
||||
/// Удаляем из sample_block_with_columns_to_add ключевые столбцы.
|
||||
for (const auto & name : key_names_right)
|
||||
sample_block.erase(sample_block.getPositionByName(name));
|
||||
|
||||
for (size_t i = 0, size = sample_block.columns(); i < size; ++i)
|
||||
{
|
||||
auto & column = sample_block.unsafeGetByPosition(i);
|
||||
size_t pos = sample_block_with_columns_to_add.getPositionByName(name);
|
||||
sample_block_with_keys.insert(sample_block_with_columns_to_add.unsafeGetByPosition(pos));
|
||||
sample_block_with_columns_to_add.erase(pos);
|
||||
}
|
||||
|
||||
for (size_t i = 0, size = sample_block_with_columns_to_add.columns(); i < size; ++i)
|
||||
{
|
||||
auto & column = sample_block_with_columns_to_add.unsafeGetByPosition(i);
|
||||
if (!column.column)
|
||||
column.column = column.type->createColumn();
|
||||
}
|
||||
@ -365,10 +369,21 @@ bool Join::insertFromBlock(const Block & block)
|
||||
size_t keys_size = key_names_right.size();
|
||||
ConstColumnPlainPtrs key_columns(keys_size);
|
||||
|
||||
/// Редкий случай, когда ключи являются константами. Чтобы не поддерживать отдельный код, материализуем их.
|
||||
Columns materialized_columns;
|
||||
|
||||
/// Запоминаем столбцы ключей, с которыми будем работать
|
||||
for (size_t i = 0; i < keys_size; ++i)
|
||||
{
|
||||
key_columns[i] = block.getByName(key_names_right[i]).column;
|
||||
|
||||
if (key_columns[i]->isConst())
|
||||
{
|
||||
materialized_columns.emplace_back(dynamic_cast<const IColumnConst &>(*key_columns[i]).convertToFullColumn());
|
||||
key_columns[i] = materialized_columns.back();
|
||||
}
|
||||
}
|
||||
|
||||
size_t rows = block.rows();
|
||||
|
||||
blocks.push_back(block);
|
||||
@ -378,7 +393,7 @@ bool Join::insertFromBlock(const Block & block)
|
||||
for (const auto & name : key_names_right)
|
||||
stored_block->erase(stored_block->getPositionByName(name));
|
||||
|
||||
/// Редкий случай, когда соединяемые столбцы являеются константами. Чтобы не поддерживать отдельный код, материализуем их.
|
||||
/// Редкий случай, когда соединяемые столбцы являются константами. Чтобы не поддерживать отдельный код, материализуем их.
|
||||
for (size_t i = 0, size = stored_block->columns(); i < size; ++i)
|
||||
{
|
||||
ColumnPtr col = stored_block->getByPosition(i).column;
|
||||
@ -515,19 +530,30 @@ void Join::joinBlockImpl(Block & block, const Maps & maps) const
|
||||
size_t keys_size = key_names_left.size();
|
||||
ConstColumnPlainPtrs key_columns(keys_size);
|
||||
|
||||
/// Редкий случай, когда ключи являются константами. Чтобы не поддерживать отдельный код, материализуем их.
|
||||
Columns materialized_columns;
|
||||
|
||||
/// Запоминаем столбцы ключей, с которыми будем работать
|
||||
for (size_t i = 0; i < keys_size; ++i)
|
||||
{
|
||||
key_columns[i] = block.getByName(key_names_left[i]).column;
|
||||
|
||||
if (key_columns[i]->isConst())
|
||||
{
|
||||
materialized_columns.emplace_back(dynamic_cast<const IColumnConst &>(*key_columns[i]).convertToFullColumn());
|
||||
key_columns[i] = materialized_columns.back();
|
||||
}
|
||||
}
|
||||
|
||||
/// Добавляем в блок новые столбцы.
|
||||
size_t num_columns_to_add = sample_block.columns();
|
||||
size_t num_columns_to_add = sample_block_with_columns_to_add.columns();
|
||||
ColumnPlainPtrs added_columns(num_columns_to_add);
|
||||
|
||||
size_t existing_columns = block.columns();
|
||||
|
||||
for (size_t i = 0; i < num_columns_to_add; ++i)
|
||||
{
|
||||
const ColumnWithNameAndType & src_column = sample_block.getByPosition(i);
|
||||
const ColumnWithNameAndType & src_column = sample_block_with_columns_to_add.getByPosition(i);
|
||||
ColumnWithNameAndType new_column = src_column.cloneEmpty();
|
||||
block.insert(new_column);
|
||||
added_columns[i] = new_column.column;
|
||||
@ -628,10 +654,25 @@ void Join::joinBlockImpl(Block & block, const Maps & maps) const
|
||||
}
|
||||
|
||||
|
||||
void Join::checkTypesOfKeys(const Block & block_left, const Block & block_right) const
|
||||
{
|
||||
size_t keys_size = key_names_left.size();
|
||||
|
||||
for (size_t i = 0; i < keys_size; ++i)
|
||||
if (block_left.getByName(key_names_left[i]).type->getName() != block_right.getByName(key_names_right[i]).type->getName())
|
||||
throw Exception("Type mismatch of columns to JOIN by: "
|
||||
+ key_names_left[i] + " " + block_left.getByName(key_names_left[i]).type->getName() + " at left, "
|
||||
+ key_names_right[i] + " " + block_right.getByName(key_names_right[i]).type->getName() + " at right",
|
||||
ErrorCodes::TYPE_MISMATCH);
|
||||
}
|
||||
|
||||
|
||||
void Join::joinBlock(Block & block) const
|
||||
{
|
||||
Poco::ScopedReadRWLock lock(rwlock);
|
||||
|
||||
checkTypesOfKeys(block, sample_block_with_keys);
|
||||
|
||||
if (kind == ASTJoin::Left && strictness == ASTJoin::Any)
|
||||
joinBlockImpl<ASTJoin::Left, ASTJoin::Any>(block, maps_any);
|
||||
else if (kind == ASTJoin::Inner && strictness == ASTJoin::Any)
|
||||
@ -666,7 +707,7 @@ void Join::joinTotals(Block & block) const
|
||||
else
|
||||
{
|
||||
/// Будем присоединять пустые totals - из одной строчки со значениями по-умолчанию.
|
||||
totals_without_keys = sample_block.cloneEmpty();
|
||||
totals_without_keys = sample_block_with_columns_to_add.cloneEmpty();
|
||||
|
||||
for (size_t i = 0; i < totals_without_keys.columns(); ++i)
|
||||
{
|
||||
@ -771,12 +812,12 @@ private:
|
||||
}
|
||||
|
||||
/// Добавляем в блок новые столбцы.
|
||||
size_t num_columns_right = parent.sample_block.columns();
|
||||
size_t num_columns_right = parent.sample_block_with_columns_to_add.columns();
|
||||
ColumnPlainPtrs columns_right(num_columns_right);
|
||||
|
||||
for (size_t i = 0; i < num_columns_right; ++i)
|
||||
{
|
||||
const ColumnWithNameAndType & src_column = parent.sample_block.getByPosition(i);
|
||||
const ColumnWithNameAndType & src_column = parent.sample_block_with_columns_to_add.getByPosition(i);
|
||||
ColumnWithNameAndType new_column = src_column.cloneEmpty();
|
||||
block.insert(new_column);
|
||||
columns_right[i] = new_column.column;
|
||||
|
@ -12,10 +12,15 @@
|
||||
#include <DB/Parsers/ASTLiteral.h>
|
||||
|
||||
#include <DB/Interpreters/Set.h>
|
||||
#include <DB/Interpreters/ExpressionAnalyzer.h>
|
||||
#include <DB/Interpreters/ExpressionActions.h>
|
||||
|
||||
#include <DB/DataTypes/DataTypeArray.h>
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
#include <DB/DataTypes/DataTypeFixedString.h>
|
||||
#include <DB/DataTypes/DataTypeDate.h>
|
||||
#include <DB/DataTypes/DataTypeDateTime.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -231,7 +236,7 @@ bool Set::insertFromBlock(const Block & block, bool create_ordered_set)
|
||||
*/
|
||||
static Field convertToType(const Field & src, const IDataType & type)
|
||||
{
|
||||
if (type.behavesAsNumber())
|
||||
if (type.isNumeric())
|
||||
{
|
||||
bool is_uint8 = false;
|
||||
bool is_uint16 = false;
|
||||
@ -243,6 +248,8 @@ static Field convertToType(const Field & src, const IDataType & type)
|
||||
bool is_int64 = false;
|
||||
bool is_float32 = false;
|
||||
bool is_float64 = false;
|
||||
bool is_date = false;
|
||||
bool is_datetime = false;
|
||||
|
||||
false
|
||||
|| (is_uint8 = typeid_cast<const DataTypeUInt8 * >(&type))
|
||||
@ -254,15 +261,18 @@ static Field convertToType(const Field & src, const IDataType & type)
|
||||
|| (is_int32 = typeid_cast<const DataTypeInt32 * >(&type))
|
||||
|| (is_int64 = typeid_cast<const DataTypeInt64 * >(&type))
|
||||
|| (is_float32 = typeid_cast<const DataTypeFloat32 * >(&type))
|
||||
|| (is_float64 = typeid_cast<const DataTypeFloat64 * >(&type));
|
||||
|| (is_float64 = typeid_cast<const DataTypeFloat64 * >(&type))
|
||||
|| (is_date = typeid_cast<const DataTypeDate * >(&type))
|
||||
|| (is_datetime = typeid_cast<const DataTypeDateTime * >(&type))
|
||||
;
|
||||
|
||||
if (is_uint8 || is_uint16 || is_uint32 || is_uint64)
|
||||
{
|
||||
if (src.getType() == Field::Types::Int64)
|
||||
throw Exception("Type mismatch in IN section: " + type.getName() + " at left, signed literal at right");
|
||||
throw Exception("Type mismatch in IN section: " + type.getName() + " at left, signed at right");
|
||||
|
||||
if (src.getType() == Field::Types::Float64)
|
||||
throw Exception("Type mismatch in IN section: " + type.getName() + " at left, floating point literal at right");
|
||||
throw Exception("Type mismatch in IN section: " + type.getName() + " at left, floating point at right");
|
||||
|
||||
if (src.getType() == Field::Types::UInt64)
|
||||
{
|
||||
@ -276,12 +286,12 @@ static Field convertToType(const Field & src, const IDataType & type)
|
||||
}
|
||||
|
||||
throw Exception("Type mismatch in IN section: " + type.getName() + " at left, "
|
||||
+ Field::Types::toString(src.getType()) + " literal at right");
|
||||
+ Field::Types::toString(src.getType()) + " at right");
|
||||
}
|
||||
else if (is_int8 || is_int16 || is_int32 || is_int64)
|
||||
{
|
||||
if (src.getType() == Field::Types::Float64)
|
||||
throw Exception("Type mismatch in IN section: " + type.getName() + " at left, floating point literal at right");
|
||||
throw Exception("Type mismatch in IN section: " + type.getName() + " at left, floating point at right");
|
||||
|
||||
if (src.getType() == Field::Types::UInt64)
|
||||
{
|
||||
@ -308,7 +318,7 @@ static Field convertToType(const Field & src, const IDataType & type)
|
||||
}
|
||||
|
||||
throw Exception("Type mismatch in IN section: " + type.getName() + " at left, "
|
||||
+ Field::Types::toString(src.getType()) + " literal at right");
|
||||
+ Field::Types::toString(src.getType()) + " at right");
|
||||
}
|
||||
else if (is_float32 || is_float64)
|
||||
{
|
||||
@ -322,24 +332,77 @@ static Field convertToType(const Field & src, const IDataType & type)
|
||||
return src;
|
||||
|
||||
throw Exception("Type mismatch in IN section: " + type.getName() + " at left, "
|
||||
+ Field::Types::toString(src.getType()) + " literal at right");
|
||||
+ Field::Types::toString(src.getType()) + " at right");
|
||||
}
|
||||
else if (is_date || is_datetime)
|
||||
{
|
||||
if (src.getType() != Field::Types::UInt64)
|
||||
throw Exception("Type mismatch in IN section: " + type.getName() + " at left, "
|
||||
+ Field::Types::toString(src.getType()) + " at right");
|
||||
|
||||
return src;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
if (src.getType() == Field::Types::UInt64
|
||||
|| src.getType() == Field::Types::Int64
|
||||
|| src.getType() == Field::Types::Float64
|
||||
|| src.getType() == Field::Types::Null
|
||||
|| (src.getType() == Field::Types::String
|
||||
&& !typeid_cast<const DataTypeString *>(&type)
|
||||
&& !typeid_cast<const DataTypeFixedString *>(&type))
|
||||
|| (src.getType() == Field::Types::Array
|
||||
&& !typeid_cast<const DataTypeArray *>(&type)))
|
||||
throw Exception("Type mismatch in IN section: " + type.getName() + " at left, "
|
||||
+ Field::Types::toString(src.getType()) + " at right");
|
||||
}
|
||||
|
||||
/// В остальных случаях, приведение типа не осуществляется.
|
||||
return src;
|
||||
}
|
||||
|
||||
|
||||
void Set::createFromAST(DataTypes & types, ASTPtr node, bool create_ordered_set)
|
||||
/** Выполнить константное выражение (для элемента множества в IN). Весьма неоптимально. */
|
||||
static Field evaluateConstantExpression(ASTPtr & node, const Context & context)
|
||||
{
|
||||
/** NOTE:
|
||||
* На данный момент в секции IN не поддерживаются выражения (вызовы функций), кроме кортежей.
|
||||
* То есть, например, не поддерживаются массивы. А по хорошему, хотелось бы поддерживать.
|
||||
* Для этого можно сделать constant folding с помощью ExpressionAnalyzer/ExpressionActions.
|
||||
* Но при этом, конечно же, не забыть про производительность работы с крупными множествами.
|
||||
*/
|
||||
ExpressionActionsPtr expr_for_constant_folding = ExpressionAnalyzer(
|
||||
node, context, NamesAndTypesList{{ "_dummy", new DataTypeUInt8 }}).getConstActions();
|
||||
|
||||
/// В блоке должен быть хотя бы один столбец, чтобы у него было известно число строк.
|
||||
Block block_with_constants{{ new ColumnConstUInt8(1, 0), new DataTypeUInt8, "_dummy" }};
|
||||
|
||||
expr_for_constant_folding->execute(block_with_constants);
|
||||
|
||||
if (!block_with_constants || block_with_constants.rows() == 0)
|
||||
throw Exception("Logical error: empty block after evaluation constant expression for IN", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
String name = node->getColumnName();
|
||||
|
||||
if (!block_with_constants.has(name))
|
||||
throw Exception("Element of set in IN is not a constant expression: " + name, ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
const IColumn & result_column = *block_with_constants.getByName(name).column;
|
||||
|
||||
if (!result_column.isConst())
|
||||
throw Exception("Element of set in IN is not a constant expression: " + name, ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
return result_column[0];
|
||||
}
|
||||
|
||||
|
||||
static Field extractValueFromNode(ASTPtr & node, const IDataType & type, const Context & context)
|
||||
{
|
||||
if (ASTLiteral * lit = typeid_cast<ASTLiteral *>(node.get()))
|
||||
return convertToType(lit->value, type);
|
||||
else if (typeid_cast<ASTFunction *>(node.get()))
|
||||
return convertToType(evaluateConstantExpression(node, context), type);
|
||||
else
|
||||
throw Exception("Incorrect element of set. Must be literal or constant expression.", ErrorCodes::INCORRECT_ELEMENT_OF_SET);
|
||||
}
|
||||
|
||||
|
||||
void Set::createFromAST(DataTypes & types, ASTPtr node, const Context & context, bool create_ordered_set)
|
||||
{
|
||||
data_types = types;
|
||||
|
||||
/// Засунем множество в блок.
|
||||
@ -359,10 +422,7 @@ void Set::createFromAST(DataTypes & types, ASTPtr node, bool create_ordered_set)
|
||||
{
|
||||
if (data_types.size() == 1)
|
||||
{
|
||||
if (ASTLiteral * lit = typeid_cast<ASTLiteral *>(&**it))
|
||||
block.getByPosition(0).column->insert(convertToType(lit->value, *data_types[0]));
|
||||
else
|
||||
throw Exception("Incorrect element of set. Must be literal.", ErrorCodes::INCORRECT_ELEMENT_OF_SET);
|
||||
block.getByPosition(0).column->insert(extractValueFromNode(*it, *data_types[0], context));
|
||||
}
|
||||
else if (ASTFunction * func = typeid_cast<ASTFunction *>(&**it))
|
||||
{
|
||||
@ -375,16 +435,11 @@ void Set::createFromAST(DataTypes & types, ASTPtr node, bool create_ordered_set)
|
||||
|
||||
for (size_t j = 0; j < tuple_size; ++j)
|
||||
{
|
||||
if (ASTLiteral * lit = typeid_cast<ASTLiteral *>(&*func->arguments->children[j]))
|
||||
block.getByPosition(j).column->insert(convertToType(lit->value, *data_types[j]));
|
||||
else
|
||||
throw Exception("Incorrect element of tuple in set. Must be literal.", ErrorCodes::INCORRECT_ELEMENT_OF_SET);
|
||||
block.getByPosition(j).column->insert(extractValueFromNode(func->arguments->children[j], *data_types[j], context));
|
||||
}
|
||||
}
|
||||
else
|
||||
throw Exception("Incorrect element of set", ErrorCodes::INCORRECT_ELEMENT_OF_SET);
|
||||
|
||||
/// NOTE: Потом можно реализовать возможность задавать константные выражения в множествах.
|
||||
}
|
||||
|
||||
if (create_ordered_set)
|
||||
|
@ -220,10 +220,12 @@ bool MergeTreeDataMerger::selectPartsToMerge(MergeTreeData::DataPartsVector & pa
|
||||
{
|
||||
disk_space_warning_time = now;
|
||||
LOG_WARNING(log, "Won't merge parts from " << first_part->name << " to " << last_part->name
|
||||
<< " because not enough free space: " << available_disk_space << " free and unreserved "
|
||||
<< "(" << DiskSpaceMonitor::getReservedSpace() << " reserved in "
|
||||
<< " because not enough free space: "
|
||||
<< formatReadableSizeWithBinarySuffix(available_disk_space) << " free and unreserved "
|
||||
<< "(" << formatReadableSizeWithBinarySuffix(DiskSpaceMonitor::getReservedSpace()) << " reserved in "
|
||||
<< DiskSpaceMonitor::getReservationCount() << " chunks), "
|
||||
<< cur_sum << " required now (+" << static_cast<int>((DISK_USAGE_COEFFICIENT_TO_SELECT - 1.0) * 100)
|
||||
<< formatReadableSizeWithBinarySuffix(cur_sum)
|
||||
<< " required now (+" << static_cast<int>((DISK_USAGE_COEFFICIENT_TO_SELECT - 1.0) * 100)
|
||||
<< "% on overhead); suppressing similar warnings for the next hour");
|
||||
}
|
||||
break;
|
||||
@ -461,9 +463,8 @@ size_t MergeTreeDataMerger::estimateDiskSpaceForMerge(const MergeTreeData::DataP
|
||||
{
|
||||
size_t res = 0;
|
||||
for (const MergeTreeData::DataPartPtr & part : parts)
|
||||
{
|
||||
res += part->size_in_bytes;
|
||||
}
|
||||
|
||||
return static_cast<size_t>(res * DISK_USAGE_COEFFICIENT_TO_RESERVE);
|
||||
}
|
||||
|
||||
|
@ -13,19 +13,29 @@ namespace DB
|
||||
{
|
||||
|
||||
|
||||
StoragePtr StorageBuffer::create(const std::string & name_, NamesAndTypesListPtr columns_, Context & context_,
|
||||
StoragePtr StorageBuffer::create(const std::string & name_, NamesAndTypesListPtr columns_,
|
||||
const NamesAndTypesList & materialized_columns_,
|
||||
const NamesAndTypesList & alias_columns_,
|
||||
const ColumnDefaults & column_defaults_,
|
||||
Context & context_,
|
||||
size_t num_shards_, const Thresholds & min_thresholds_, const Thresholds & max_thresholds_,
|
||||
const String & destination_database_, const String & destination_table_)
|
||||
{
|
||||
return (new StorageBuffer{
|
||||
name_, columns_, context_, num_shards_, min_thresholds_, max_thresholds_, destination_database_, destination_table_})->thisPtr();
|
||||
name_, columns_, materialized_columns_, alias_columns_, column_defaults_,
|
||||
context_, num_shards_, min_thresholds_, max_thresholds_, destination_database_, destination_table_})->thisPtr();
|
||||
}
|
||||
|
||||
|
||||
StorageBuffer::StorageBuffer(const std::string & name_, NamesAndTypesListPtr columns_, Context & context_,
|
||||
StorageBuffer::StorageBuffer(const std::string & name_, NamesAndTypesListPtr columns_,
|
||||
const NamesAndTypesList & materialized_columns_,
|
||||
const NamesAndTypesList & alias_columns_,
|
||||
const ColumnDefaults & column_defaults_,
|
||||
Context & context_,
|
||||
size_t num_shards_, const Thresholds & min_thresholds_, const Thresholds & max_thresholds_,
|
||||
const String & destination_database_, const String & destination_table_)
|
||||
: name(name_), columns(columns_), context(context_),
|
||||
: IStorage{materialized_columns_, alias_columns_, column_defaults_},
|
||||
name(name_), columns(columns_), context(context_),
|
||||
num_shards(num_shards_), buffers(num_shards_),
|
||||
min_thresholds(min_thresholds_), max_thresholds(max_thresholds_),
|
||||
destination_database(destination_database_), destination_table(destination_table_),
|
||||
|
@ -373,7 +373,9 @@ StoragePtr StorageFactory::get(
|
||||
size_t max_bytes = apply_visitor(FieldVisitorConvertToNumber<size_t>(), typeid_cast<ASTLiteral &>(*args[8]).value);
|
||||
|
||||
return StorageBuffer::create(
|
||||
table_name, columns, context,
|
||||
table_name, columns,
|
||||
materialized_columns, alias_columns, column_defaults,
|
||||
context,
|
||||
num_buckets, {min_time, min_rows, min_bytes}, {max_time, max_rows, max_bytes},
|
||||
destination_database, destination_table);
|
||||
}
|
||||
|
@ -229,7 +229,7 @@ bool StorageMergeTree::merge(size_t aio_threshold, bool aggressive, BackgroundPr
|
||||
return false;
|
||||
}
|
||||
|
||||
merging_tagger = new CurrentlyMergingPartsTagger(parts, merger.estimateDiskSpaceForMerge(parts), *this);
|
||||
merging_tagger = new CurrentlyMergingPartsTagger(parts, MergeTreeDataMerger::estimateDiskSpaceForMerge(parts), *this);
|
||||
|
||||
/// Если собираемся сливать большие куски, увеличим счетчик потоков, сливающих большие куски.
|
||||
if (pool_context)
|
||||
|
@ -14,6 +14,7 @@
|
||||
#include <DB/Parsers/ASTInsertQuery.h>
|
||||
#include <DB/DataStreams/AddingConstColumnBlockInputStream.h>
|
||||
#include <DB/Common/Macros.h>
|
||||
#include <DB/Common/formatReadable.h>
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
#include <time.h>
|
||||
|
||||
@ -860,12 +861,15 @@ bool StorageReplicatedMergeTree::executeLogEntry(const LogEntry & entry, Backgro
|
||||
}
|
||||
}
|
||||
|
||||
size_t sum_parts_size_in_bytes = MergeTreeDataMerger::estimateDiskSpaceForMerge(parts);
|
||||
DiskSpaceMonitor::ReservationPtr reserved_space = DiskSpaceMonitor::reserve(full_path, sum_parts_size_in_bytes); /// Может бросить исключение.
|
||||
|
||||
auto table_lock = lockStructure(false);
|
||||
|
||||
const auto & merge_entry = context.getMergeList().insert(database_name, table_name, entry.new_part_name);
|
||||
MergeTreeData::Transaction transaction;
|
||||
size_t aio_threshold = context.getSettings().min_bytes_to_use_direct_io;
|
||||
MergeTreeData::DataPartPtr part = merger.mergeParts(parts, entry.new_part_name, *merge_entry, aio_threshold, &transaction);
|
||||
MergeTreeData::DataPartPtr part = merger.mergeParts(parts, entry.new_part_name, *merge_entry, aio_threshold, &transaction, reserved_space);
|
||||
|
||||
zkutil::Ops ops;
|
||||
checkPartAndAddToZooKeeper(part, ops);
|
||||
@ -1360,8 +1364,10 @@ void StorageReplicatedMergeTree::mergeSelectingThread()
|
||||
|
||||
String merged_name;
|
||||
|
||||
if ( !merger.selectPartsToMerge(parts, merged_name, MergeTreeDataMerger::NO_LIMIT, false, false, only_small, can_merge)
|
||||
&& !merger.selectPartsToMerge(parts, merged_name, MergeTreeDataMerger::NO_LIMIT, true, false, only_small, can_merge))
|
||||
size_t disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);
|
||||
|
||||
if ( !merger.selectPartsToMerge(parts, merged_name, disk_space, false, false, only_small, can_merge)
|
||||
&& !merger.selectPartsToMerge(parts, merged_name, disk_space, true, false, only_small, can_merge))
|
||||
{
|
||||
break;
|
||||
}
|
||||
|
@ -1,6 +1,6 @@
|
||||
DROP TABLE IF EXISTS test.join;
|
||||
|
||||
CREATE TABLE test.join (k UInt8, s String) ENGINE = Join(ANY, LEFT, k);
|
||||
CREATE TABLE test.join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k);
|
||||
|
||||
USE test;
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
DROP TABLE IF EXISTS test.join;
|
||||
|
||||
CREATE TABLE test.join (s String, x Array(UInt8), k UInt8) ENGINE = Join(ANY, LEFT, k);
|
||||
CREATE TABLE test.join (s String, x Array(UInt8), k UInt64) ENGINE = Join(ANY, LEFT, k);
|
||||
|
||||
USE test;
|
||||
|
||||
|
@ -0,0 +1,9 @@
|
||||
EventDate Date
|
||||
UTCEventTime DateTime
|
||||
MoscowEventDate Date DEFAULT toDate(UTCEventTime)
|
||||
EventDate Date
|
||||
UTCEventTime DateTime
|
||||
MoscowEventDate Date DEFAULT toDate(UTCEventTime)
|
||||
2015-06-09 2015-06-09 01:02:03 2015-06-09
|
||||
2015-06-09 2015-06-09 01:02:03 2015-06-09
|
||||
2015-06-09 2015-06-09 01:02:03 2015-06-09
|
12
dbms/tests/queries/0_stateless/00168_buffer_defaults.sql
Normal file
12
dbms/tests/queries/0_stateless/00168_buffer_defaults.sql
Normal file
@ -0,0 +1,12 @@
|
||||
DROP TABLE IF EXISTS test.mt;
|
||||
DROP TABLE IF EXISTS test.mt_buffer;
|
||||
CREATE TABLE test.mt (EventDate Date, UTCEventTime DateTime, MoscowEventDate Date DEFAULT toDate(UTCEventTime)) ENGINE = MergeTree(EventDate, UTCEventTime, 8192);
|
||||
CREATE TABLE test.mt_buffer AS test.mt ENGINE = Buffer(test, mt, 16, 10, 100, 10000, 1000000, 10000000, 100000000);
|
||||
DESC TABLE test.mt;
|
||||
DESC TABLE test.mt_buffer;
|
||||
INSERT INTO test.mt (EventDate, UTCEventTime) VALUES ('2015-06-09', '2015-06-09 01:02:03');
|
||||
SELECT * FROM test.mt_buffer;
|
||||
INSERT INTO test.mt_buffer (EventDate, UTCEventTime) VALUES ('2015-06-09', '2015-06-09 01:02:03');
|
||||
SELECT * FROM test.mt_buffer;
|
||||
DROP TABLE test.mt_buffer;
|
||||
DROP TABLE test.mt;
|
@ -0,0 +1,3 @@
|
||||
1 0 999
|
||||
2 0 999
|
||||
3 0 999
|
17
dbms/tests/queries/0_stateless/00169_join_constant_keys.sql
Normal file
17
dbms/tests/queries/0_stateless/00169_join_constant_keys.sql
Normal file
@ -0,0 +1,17 @@
|
||||
SELECT
|
||||
key1,
|
||||
key2,
|
||||
table_1
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
arrayJoin([1, 2, 3]) AS key1,
|
||||
0 AS key2,
|
||||
999 AS table_1
|
||||
) ALL INNER JOIN
|
||||
(
|
||||
SELECT
|
||||
arrayJoin([1, 3, 2]) AS key1,
|
||||
0 AS key2,
|
||||
999 AS table_1
|
||||
) USING key2, key1;
|
@ -0,0 +1,24 @@
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
29
dbms/tests/queries/0_stateless/00170_lower_upper_utf8.sql
Normal file
29
dbms/tests/queries/0_stateless/00170_lower_upper_utf8.sql
Normal file
@ -0,0 +1,29 @@
|
||||
select lower('aaaaaaaaaaaaaaa012345789,.!aaaa' as str) = str;
|
||||
select lowerUTF8('aaaaaaaaaaaaaaa012345789,.!aaaa' as str) = str;
|
||||
select lower('AaAaAaAaAaAaAaA012345789,.!aAaA') = 'aaaaaaaaaaaaaaa012345789,.!aaaa';
|
||||
select lowerUTF8('AaAaAaAaAaAaAaA012345789,.!aAaA') = 'aaaaaaaaaaaaaaa012345789,.!aaaa';
|
||||
|
||||
select upper('AAAAAAAAAAAAAAA012345789,.!AAAA' as str) = str;
|
||||
select upperUTF8('AAAAAAAAAAAAAAA012345789,.!AAAA' as str) = str;
|
||||
select upper('AaAaAaAaAaAaAaA012345789,.!aAaA') = 'AAAAAAAAAAAAAAA012345789,.!AAAA';
|
||||
select upperUTF8('AaAaAaAaAaAaAaA012345789,.!aAaA') = 'AAAAAAAAAAAAAAA012345789,.!AAAA';
|
||||
|
||||
select sum(lower(materialize('aaaaaaaaaaaaaaa012345789,.!aaaa') as str) = str) = count() array join range(16384) as n;
|
||||
select sum(lowerUTF8(materialize('aaaaaaaaaaaaaaa012345789,.!aaaa') as str) = str) = count() array join range(16384) as n;
|
||||
select sum(lower(materialize('AaAaAaAaAaAaAaA012345789,.!aAaA')) = materialize('aaaaaaaaaaaaaaa012345789,.!aaaa')) = count() array join range(16384) as n;
|
||||
select sum(lowerUTF8(materialize('AaAaAaAaAaAaAaA012345789,.!aAaA')) = materialize('aaaaaaaaaaaaaaa012345789,.!aaaa')) = count() array join range(16384) as n;
|
||||
|
||||
select sum(upper(materialize('AAAAAAAAAAAAAAA012345789,.!AAAA') as str) = str) = count() array join range(16384) as n;
|
||||
select sum(upperUTF8(materialize('AAAAAAAAAAAAAAA012345789,.!AAAA') as str) = str) = count() array join range(16384) as n;
|
||||
select sum(upper(materialize('AaAaAaAaAaAaAaA012345789,.!aAaA')) = materialize('AAAAAAAAAAAAAAA012345789,.!AAAA')) = count() array join range(16384) as n;
|
||||
select sum(upperUTF8(materialize('AaAaAaAaAaAaAaA012345789,.!aAaA')) = materialize('AAAAAAAAAAAAAAA012345789,.!AAAA')) = count() array join range(16384) as n;
|
||||
|
||||
select lower('aaaaАБВГAAAAaaAA') = 'aaaaАБВГaaaaaaaa';
|
||||
select upper('aaaaАБВГAAAAaaAA') = 'AAAAАБВГAAAAAAAA';
|
||||
select lowerUTF8('aaaaАБВГAAAAaaAA') = 'aaaaабвгaaaaaaaa';
|
||||
select upperUTF8('aaaaАБВГAAAAaaAA') = 'AAAAАБВГAAAAAAAA';
|
||||
|
||||
select sum(lower(materialize('aaaaАБВГAAAAaaAA')) = materialize('aaaaАБВГaaaaaaaa')) = count() array join range(16384) as n;
|
||||
select sum(upper(materialize('aaaaАБВГAAAAaaAA')) = materialize('AAAAАБВГAAAAAAAA')) = count() array join range(16384) as n;
|
||||
select sum(lowerUTF8(materialize('aaaaАБВГAAAAaaAA')) = materialize('aaaaабвгaaaaaaaa')) = count() array join range(16384) as n;
|
||||
select sum(upperUTF8(materialize('aaaaАБВГAAAAaaAA')) = materialize('AAAAАБВГAAAAAAAA')) = count() array join range(16384) as n;
|
@ -0,0 +1,2 @@
|
||||
[(1,4),(2,5),(3,6)]
|
||||
[(1,4),(2,5),(3,6)]
|
@ -0,0 +1 @@
|
||||
SELECT arrayMap((x, y) -> (x, y), [1, 2, 3], [4, 5, 6]) FROM remote('127.0.0.{1,2}', system.one) ORDER BY rand();
|
@ -0,0 +1,6 @@
|
||||
14 3
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
@ -0,0 +1,6 @@
|
||||
SELECT sumIf(number, x), sum(x) FROM (SELECT number, number IN (0 + 1, 2 + 3, toUInt64(concat('8', ''))) AS x FROM system.numbers LIMIT 10);
|
||||
SELECT toDate('2015-06-12') IN toDate('2015-06-12');
|
||||
SELECT toDate('2015-06-12') IN (toDate('2015-06-12'));
|
||||
SELECT today() IN (toDate('2014-01-01'), toDate(now()));
|
||||
SELECT - -1 IN (2 - 1);
|
||||
SELECT - -1 IN (2 - 1, 3);
|
@ -0,0 +1,80 @@
|
||||
1
|
||||
1
|
||||
0
|
||||
0
|
||||
1
|
||||
1
|
||||
0
|
||||
0
|
||||
1
|
||||
1
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
1
|
||||
1
|
||||
0
|
||||
0
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
0
|
||||
0
|
||||
1
|
||||
1
|
||||
0
|
||||
0
|
||||
1
|
||||
1
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
1
|
||||
1
|
||||
0
|
||||
0
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
0
|
||||
0
|
||||
1
|
||||
1
|
||||
0
|
||||
0
|
||||
1
|
||||
1
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
1
|
||||
1
|
||||
0
|
||||
0
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
0
|
||||
0
|
||||
1
|
||||
1
|
||||
0
|
||||
0
|
||||
1
|
||||
1
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
1
|
||||
1
|
||||
0
|
||||
0
|
||||
1
|
||||
1
|
@ -0,0 +1,83 @@
|
||||
SELECT toDate('2015-02-03') = '2015-02-03';
|
||||
SELECT '2015-02-03' = toDate('2015-02-03');
|
||||
SELECT toDate('2015-02-03') = '2015-02-04';
|
||||
SELECT '2015-02-03' = toDate('2015-02-04');
|
||||
SELECT toDate('2015-02-03') < '2015-02-04';
|
||||
SELECT '2015-02-03' < toDate('2015-02-04');
|
||||
SELECT toDate('2015-02-03') > '2015-02-04';
|
||||
SELECT '2015-02-03' > toDate('2015-02-04');
|
||||
SELECT toDate('2015-02-03') <= '2015-02-04';
|
||||
SELECT '2015-02-03' <= toDate('2015-02-04');
|
||||
SELECT toDate('2015-02-03') >= '2015-02-04';
|
||||
SELECT '2015-02-03' >= toDate('2015-02-04');
|
||||
SELECT toDate('2015-02-05') < '2015-02-04';
|
||||
SELECT '2015-02-05' < toDate('2015-02-04');
|
||||
SELECT toDate('2015-02-05') > '2015-02-04';
|
||||
SELECT '2015-02-05' > toDate('2015-02-04');
|
||||
SELECT toDate('2015-02-05') <= '2015-02-04';
|
||||
SELECT '2015-02-05' <= toDate('2015-02-04');
|
||||
SELECT toDate('2015-02-05') >= '2015-02-04';
|
||||
SELECT '2015-02-05' >= toDate('2015-02-04');
|
||||
|
||||
SELECT materialize(toDate('2015-02-03')) = '2015-02-03';
|
||||
SELECT '2015-02-03' = materialize(toDate('2015-02-03'));
|
||||
SELECT materialize(toDate('2015-02-03')) = '2015-02-04';
|
||||
SELECT '2015-02-03' = materialize(toDate('2015-02-04'));
|
||||
SELECT materialize(toDate('2015-02-03')) < '2015-02-04';
|
||||
SELECT '2015-02-03' < materialize(toDate('2015-02-04'));
|
||||
SELECT materialize(toDate('2015-02-03')) > '2015-02-04';
|
||||
SELECT '2015-02-03' > materialize(toDate('2015-02-04'));
|
||||
SELECT materialize(toDate('2015-02-03')) <= '2015-02-04';
|
||||
SELECT '2015-02-03' <= materialize(toDate('2015-02-04'));
|
||||
SELECT materialize(toDate('2015-02-03')) >= '2015-02-04';
|
||||
SELECT '2015-02-03' >= materialize(toDate('2015-02-04'));
|
||||
SELECT materialize(toDate('2015-02-05')) < '2015-02-04';
|
||||
SELECT '2015-02-05' < materialize(toDate('2015-02-04'));
|
||||
SELECT materialize(toDate('2015-02-05')) > '2015-02-04';
|
||||
SELECT '2015-02-05' > materialize(toDate('2015-02-04'));
|
||||
SELECT materialize(toDate('2015-02-05')) <= '2015-02-04';
|
||||
SELECT '2015-02-05' <= materialize(toDate('2015-02-04'));
|
||||
SELECT materialize(toDate('2015-02-05')) >= '2015-02-04';
|
||||
SELECT '2015-02-05' >= materialize(toDate('2015-02-04'));
|
||||
|
||||
SELECT toDateTime('2015-02-03 04:05:06') = '2015-02-03 04:05:06';
|
||||
SELECT '2015-02-03 04:05:06' = toDateTime('2015-02-03 04:05:06');
|
||||
SELECT toDateTime('2015-02-03 04:05:06') = '2015-02-03 05:06:07';
|
||||
SELECT '2015-02-03 04:05:06' = toDateTime('2015-02-03 05:06:07');
|
||||
SELECT toDateTime('2015-02-03 04:05:06') < '2015-02-03 05:06:07';
|
||||
SELECT '2015-02-03 04:05:06' < toDateTime('2015-02-03 05:06:07');
|
||||
SELECT toDateTime('2015-02-03 04:05:06') > '2015-02-03 05:06:07';
|
||||
SELECT '2015-02-03 04:05:06' > toDateTime('2015-02-03 05:06:07');
|
||||
SELECT toDateTime('2015-02-03 04:05:06') <= '2015-02-03 05:06:07';
|
||||
SELECT '2015-02-03 04:05:06' <= toDateTime('2015-02-03 05:06:07');
|
||||
SELECT toDateTime('2015-02-03 04:05:06') >= '2015-02-03 05:06:07';
|
||||
SELECT '2015-02-03 04:05:06' >= toDateTime('2015-02-03 05:06:07');
|
||||
SELECT toDateTime('2015-02-03 06:07:08') < '2015-02-03 05:06:07';
|
||||
SELECT '2015-02-03 06:07:08' < toDateTime('2015-02-03 05:06:07');
|
||||
SELECT toDateTime('2015-02-03 06:07:08') > '2015-02-03 05:06:07';
|
||||
SELECT '2015-02-03 06:07:08' > toDateTime('2015-02-03 05:06:07');
|
||||
SELECT toDateTime('2015-02-03 06:07:08') <= '2015-02-03 05:06:07';
|
||||
SELECT '2015-02-03 06:07:08' <= toDateTime('2015-02-03 05:06:07');
|
||||
SELECT toDateTime('2015-02-03 06:07:08') >= '2015-02-03 05:06:07';
|
||||
SELECT '2015-02-03 06:07:08' >= toDateTime('2015-02-03 05:06:07');
|
||||
|
||||
SELECT materialize(toDateTime('2015-02-03 04:05:06')) = '2015-02-03 04:05:06';
|
||||
SELECT '2015-02-03 04:05:06' = materialize(toDateTime('2015-02-03 04:05:06'));
|
||||
SELECT materialize(toDateTime('2015-02-03 04:05:06')) = '2015-02-03 05:06:07';
|
||||
SELECT '2015-02-03 04:05:06' = materialize(toDateTime('2015-02-03 05:06:07'));
|
||||
SELECT materialize(toDateTime('2015-02-03 04:05:06')) < '2015-02-03 05:06:07';
|
||||
SELECT '2015-02-03 04:05:06' < materialize(toDateTime('2015-02-03 05:06:07'));
|
||||
SELECT materialize(toDateTime('2015-02-03 04:05:06')) > '2015-02-03 05:06:07';
|
||||
SELECT '2015-02-03 04:05:06' > materialize(toDateTime('2015-02-03 05:06:07'));
|
||||
SELECT materialize(toDateTime('2015-02-03 04:05:06')) <= '2015-02-03 05:06:07';
|
||||
SELECT '2015-02-03 04:05:06' <= materialize(toDateTime('2015-02-03 05:06:07'));
|
||||
SELECT materialize(toDateTime('2015-02-03 04:05:06')) >= '2015-02-03 05:06:07';
|
||||
SELECT '2015-02-03 04:05:06' >= materialize(toDateTime('2015-02-03 05:06:07'));
|
||||
SELECT materialize(toDateTime('2015-02-03 06:07:08')) < '2015-02-03 05:06:07';
|
||||
SELECT '2015-02-03 06:07:08' < materialize(toDateTime('2015-02-03 05:06:07'));
|
||||
SELECT materialize(toDateTime('2015-02-03 06:07:08')) > '2015-02-03 05:06:07';
|
||||
SELECT '2015-02-03 06:07:08' > materialize(toDateTime('2015-02-03 05:06:07'));
|
||||
SELECT materialize(toDateTime('2015-02-03 06:07:08')) <= '2015-02-03 05:06:07';
|
||||
SELECT '2015-02-03 06:07:08' <= materialize(toDateTime('2015-02-03 05:06:07'));
|
||||
SELECT materialize(toDateTime('2015-02-03 06:07:08')) >= '2015-02-03 05:06:07';
|
||||
SELECT '2015-02-03 06:07:08' >= materialize(toDateTime('2015-02-03 05:06:07'));
|
@ -2,13 +2,13 @@
|
||||
|
||||
#include <zkutil/ZooKeeper.h>
|
||||
|
||||
namespace zkutil
|
||||
namespace zkutil
|
||||
{
|
||||
|
||||
class Increment
|
||||
{
|
||||
public:
|
||||
Increment(ZooKeeperPtr zk_, const std::string & path_)
|
||||
Increment(ZooKeeperPtr zk_, const std::string & path_)
|
||||
: zk(zk_), path(path_)
|
||||
{
|
||||
zk->createAncestors(path);
|
||||
@ -34,8 +34,8 @@ public:
|
||||
{
|
||||
success = zk->tryCreate(path, std::to_string(result), zkutil::CreateMode::Persistent) == ZOK;
|
||||
}
|
||||
}
|
||||
while(!success);
|
||||
}
|
||||
while (!success);
|
||||
|
||||
return result;
|
||||
}
|
||||
@ -45,4 +45,4 @@ private:
|
||||
Logger * log = &Logger::get("zkutil::Increment");
|
||||
};
|
||||
|
||||
}
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user