mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge
This commit is contained in:
commit
7e5801e303
@ -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);
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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)
|
||||
|
@ -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'));
|
Loading…
Reference in New Issue
Block a user