mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 00:22:29 +00:00
dbms: additions to prev. revision [#METR-2944].
This commit is contained in:
parent
779d85afc9
commit
7b76ff3118
@ -922,16 +922,24 @@ void ExpressionAnalyzer::makeExplicitSet(ASTFunction * node, const Block & sampl
|
|||||||
|
|
||||||
if (ASTFunction * set_func = typeid_cast<ASTFunction *>(&*arg))
|
if (ASTFunction * set_func = typeid_cast<ASTFunction *>(&*arg))
|
||||||
{
|
{
|
||||||
if (set_func->name != "tuple")
|
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));
|
||||||
/// Отличм случай (x, y) in ((1, 2), (3, 4)) от случая (x, y) in (1, 2).
|
if (set_element_types.size() >= 2 && (!any_element || any_element->name != "tuple"))
|
||||||
ASTFunction * any_element = typeid_cast<ASTFunction *>(&*set_func->arguments->children.at(0));
|
single_value = true;
|
||||||
if (set_element_types.size() >= 2 && (!any_element || any_element->name != "tuple"))
|
else
|
||||||
single_value = true;
|
elements_ast = set_func->arguments;
|
||||||
|
}
|
||||||
else
|
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))
|
else if (typeid_cast<ASTLiteral *>(&*arg))
|
||||||
{
|
{
|
||||||
|
@ -19,6 +19,8 @@
|
|||||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||||
#include <DB/DataTypes/DataTypeString.h>
|
#include <DB/DataTypes/DataTypeString.h>
|
||||||
#include <DB/DataTypes/DataTypeFixedString.h>
|
#include <DB/DataTypes/DataTypeFixedString.h>
|
||||||
|
#include <DB/DataTypes/DataTypeDate.h>
|
||||||
|
#include <DB/DataTypes/DataTypeDateTime.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -234,7 +236,7 @@ bool Set::insertFromBlock(const Block & block, bool create_ordered_set)
|
|||||||
*/
|
*/
|
||||||
static Field convertToType(const Field & src, const IDataType & type)
|
static Field convertToType(const Field & src, const IDataType & type)
|
||||||
{
|
{
|
||||||
if (type.behavesAsNumber())
|
if (type.isNumeric())
|
||||||
{
|
{
|
||||||
bool is_uint8 = false;
|
bool is_uint8 = false;
|
||||||
bool is_uint16 = false;
|
bool is_uint16 = false;
|
||||||
@ -246,6 +248,8 @@ static Field convertToType(const Field & src, const IDataType & type)
|
|||||||
bool is_int64 = false;
|
bool is_int64 = false;
|
||||||
bool is_float32 = false;
|
bool is_float32 = false;
|
||||||
bool is_float64 = false;
|
bool is_float64 = false;
|
||||||
|
bool is_date = false;
|
||||||
|
bool is_datetime = false;
|
||||||
|
|
||||||
false
|
false
|
||||||
|| (is_uint8 = typeid_cast<const DataTypeUInt8 * >(&type))
|
|| (is_uint8 = typeid_cast<const DataTypeUInt8 * >(&type))
|
||||||
@ -257,7 +261,10 @@ static Field convertToType(const Field & src, const IDataType & type)
|
|||||||
|| (is_int32 = typeid_cast<const DataTypeInt32 * >(&type))
|
|| (is_int32 = typeid_cast<const DataTypeInt32 * >(&type))
|
||||||
|| (is_int64 = typeid_cast<const DataTypeInt64 * >(&type))
|
|| (is_int64 = typeid_cast<const DataTypeInt64 * >(&type))
|
||||||
|| (is_float32 = typeid_cast<const DataTypeFloat32 * >(&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 (is_uint8 || is_uint16 || is_uint32 || is_uint64)
|
||||||
{
|
{
|
||||||
@ -327,6 +334,14 @@ static Field convertToType(const Field & src, const IDataType & type)
|
|||||||
throw Exception("Type mismatch in IN section: " + type.getName() + " at left, "
|
throw Exception("Type mismatch in IN section: " + type.getName() + " at left, "
|
||||||
+ Field::Types::toString(src.getType()) + " 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
|
else
|
||||||
{
|
{
|
||||||
|
@ -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);
|
Loading…
Reference in New Issue
Block a user