mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
fix performance with large tuples, which are interpreted as functions in IN section
This commit is contained in:
parent
df1213f351
commit
fc515728f4
@ -64,6 +64,16 @@ static NamesAndTypesList::iterator findColumn(const String & name, NamesAndTypes
|
||||
[&](const NamesAndTypesList::value_type & val) { return val.name == name; });
|
||||
}
|
||||
|
||||
static size_t getTypeDepth(const DataTypePtr & type)
|
||||
{
|
||||
if (const auto * array_type = typeid_cast<const DataTypeArray *>(type.get()))
|
||||
return 1 + getTypeDepth(array_type->getNestedType());
|
||||
else if (const auto * tuple_type = typeid_cast<const DataTypeTuple *>(type.get()))
|
||||
return 1 + (tuple_type->getElements().empty() ? 0 : getTypeDepth(tuple_type->getElements().at(0)));
|
||||
|
||||
return 0;
|
||||
}
|
||||
|
||||
template<typename Collection>
|
||||
static Block createBlockFromCollection(const Collection & collection, const DataTypes & types, const Context & context)
|
||||
{
|
||||
@ -117,48 +127,119 @@ static Block createBlockFromCollection(const Collection & collection, const Data
|
||||
return res;
|
||||
}
|
||||
|
||||
SetPtr makeExplicitSet(
|
||||
const ASTFunction * node, const Block & sample_block, bool create_ordered_set,
|
||||
const Context & context, const SizeLimits & size_limits, PreparedSets & prepared_sets)
|
||||
static Field extractValueFromNode(const ASTPtr & node, const IDataType & type, const Context & context)
|
||||
{
|
||||
const IAST & args = *node->arguments;
|
||||
if (const auto * lit = node->as<ASTLiteral>())
|
||||
{
|
||||
return convertFieldToType(lit->value, type);
|
||||
}
|
||||
else if (node->as<ASTFunction>())
|
||||
{
|
||||
std::pair<Field, DataTypePtr> value_raw = evaluateConstantExpression(node, context);
|
||||
return convertFieldToType(value_raw.first, type, value_raw.second.get());
|
||||
}
|
||||
else
|
||||
throw Exception("Incorrect element of set. Must be literal or constant expression.", ErrorCodes::INCORRECT_ELEMENT_OF_SET);
|
||||
}
|
||||
|
||||
if (args.children.size() != 2)
|
||||
throw Exception("Wrong number of arguments passed to function in", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
static Block createBlockFromAST(const ASTPtr & node, const DataTypes & types, const Context & context)
|
||||
{
|
||||
/// Will form a block with values from the set.
|
||||
|
||||
const ASTPtr & left_arg = args.children.at(0);
|
||||
const ASTPtr & right_arg = args.children.at(1);
|
||||
Block header;
|
||||
size_t num_columns = types.size();
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
header.insert(ColumnWithTypeAndName(types[i]->createColumn(), types[i], "_" + toString(i)));
|
||||
|
||||
const DataTypePtr & left_arg_type = sample_block.getByName(left_arg->getColumnName()).type;
|
||||
MutableColumns columns = header.cloneEmptyColumns();
|
||||
|
||||
DataTypes set_element_types = {left_arg_type};
|
||||
const auto * left_tuple_type = typeid_cast<const DataTypeTuple *>(left_arg_type.get());
|
||||
if (left_tuple_type && left_tuple_type->getElements().size() != 1)
|
||||
set_element_types = left_tuple_type->getElements();
|
||||
DataTypePtr tuple_type;
|
||||
Row tuple_values;
|
||||
const auto & list = node->as<ASTExpressionList &>();
|
||||
for (const auto & elem : list.children)
|
||||
{
|
||||
if (num_columns == 1)
|
||||
{
|
||||
Field value = extractValueFromNode(elem, *types[0], context);
|
||||
|
||||
for (auto & element_type : set_element_types)
|
||||
if (const auto * low_cardinality_type = typeid_cast<const DataTypeLowCardinality *>(element_type.get()))
|
||||
element_type = low_cardinality_type->getDictionaryType();
|
||||
if (!value.isNull() || context.getSettingsRef().transform_null_in)
|
||||
columns[0]->insert(value);
|
||||
}
|
||||
else if (elem->as<ASTFunction>() || elem->as<ASTLiteral>())
|
||||
{
|
||||
Field function_result;
|
||||
const Tuple * tuple = nullptr;
|
||||
|
||||
auto set_key = PreparedSetKey::forLiteral(*right_arg, set_element_types);
|
||||
if (prepared_sets.count(set_key))
|
||||
return prepared_sets.at(set_key); /// Already prepared.
|
||||
auto * func = elem->as<ASTFunction>();
|
||||
if (func && func->name != "tuple")
|
||||
{
|
||||
if (!tuple_type)
|
||||
tuple_type = std::make_shared<DataTypeTuple>(types);
|
||||
|
||||
function_result = extractValueFromNode(elem, *tuple_type, context);
|
||||
if (function_result.getType() != Field::Types::Tuple)
|
||||
throw Exception("Invalid type of set. Expected tuple, got " + String(function_result.getTypeName()),
|
||||
ErrorCodes::INCORRECT_ELEMENT_OF_SET);
|
||||
|
||||
tuple = &function_result.get<Tuple>();
|
||||
}
|
||||
|
||||
auto * literal = elem->as<ASTLiteral>();
|
||||
if (literal)
|
||||
{
|
||||
if (literal->value.getType() != Field::Types::Tuple)
|
||||
throw Exception("Invalid type in set. Expected tuple, got "
|
||||
+ String(literal->value.getTypeName()), ErrorCodes::INCORRECT_ELEMENT_OF_SET);
|
||||
|
||||
tuple = &literal->value.get<Tuple>();
|
||||
}
|
||||
|
||||
size_t tuple_size = tuple ? tuple->size() : func->arguments->children.size();
|
||||
if (tuple_size != num_columns)
|
||||
throw Exception("Incorrect size of tuple in set: " + toString(tuple_size) + " instead of " + toString(num_columns),
|
||||
ErrorCodes::INCORRECT_ELEMENT_OF_SET);
|
||||
|
||||
if (tuple_values.empty())
|
||||
tuple_values.resize(tuple_size);
|
||||
|
||||
size_t i = 0;
|
||||
for (; i < tuple_size; ++i)
|
||||
{
|
||||
Field value = tuple ? (*tuple)[i]
|
||||
: extractValueFromNode(func->arguments->children[i], *types[i], context);
|
||||
|
||||
/// If at least one of the elements of the tuple has an impossible (outside the range of the type) value, then the entire tuple too.
|
||||
if (value.isNull() && !context.getSettings().transform_null_in)
|
||||
break;
|
||||
|
||||
tuple_values[i] = value;
|
||||
}
|
||||
|
||||
if (i == tuple_size)
|
||||
for (i = 0; i < tuple_size; ++i)
|
||||
columns[i]->insert(tuple_values[i]);
|
||||
}
|
||||
else
|
||||
throw Exception("Incorrect element of set", ErrorCodes::INCORRECT_ELEMENT_OF_SET);
|
||||
}
|
||||
|
||||
return header.cloneWithColumns(std::move(columns));
|
||||
}
|
||||
|
||||
/** Create a block for set from literal.
|
||||
* 'set_element_types' - types of what are on the left hand side of IN.
|
||||
* 'right_arg' - Literal - Tuple or Array.
|
||||
*/
|
||||
static Block createBlockForSet(
|
||||
const DataTypePtr & left_arg_type,
|
||||
const std::shared_ptr<ASTLiteral> & right_arg,
|
||||
const DataTypes & set_element_types,
|
||||
const Context & context)
|
||||
{
|
||||
auto [right_arg_value, right_arg_type] = evaluateConstantExpression(right_arg, context);
|
||||
|
||||
std::function<size_t(const DataTypePtr &)> get_type_depth;
|
||||
get_type_depth = [&get_type_depth](const DataTypePtr & type) -> size_t
|
||||
{
|
||||
if (const auto * array_type = typeid_cast<const DataTypeArray *>(type.get()))
|
||||
return 1 + get_type_depth(array_type->getNestedType());
|
||||
else if (const auto * tuple_type = typeid_cast<const DataTypeTuple *>(type.get()))
|
||||
return 1 + (tuple_type->getElements().empty() ? 0 : get_type_depth(tuple_type->getElements().at(0)));
|
||||
|
||||
return 0;
|
||||
};
|
||||
|
||||
const size_t left_type_depth = get_type_depth(left_arg_type);
|
||||
const size_t right_type_depth = get_type_depth(right_arg_type);
|
||||
const size_t left_type_depth = getTypeDepth(left_arg_type);
|
||||
const size_t right_type_depth = getTypeDepth(right_arg_type);
|
||||
|
||||
auto throw_unsupported_type = [](const auto & type)
|
||||
{
|
||||
@ -187,10 +268,105 @@ SetPtr makeExplicitSet(
|
||||
else
|
||||
throw_unsupported_type(right_arg_type);
|
||||
|
||||
SetPtr set = std::make_shared<Set>(size_limits, create_ordered_set, context.getSettingsRef().transform_null_in);
|
||||
return block;
|
||||
}
|
||||
|
||||
set->setHeader(block);
|
||||
/** Create a block for set from expression.
|
||||
* 'set_element_types' - types of what are on the left hand side of IN.
|
||||
* 'right_arg' - list of values: 1, 2, 3 or list of tuples: (1, 2), (3, 4), (5, 6).
|
||||
*
|
||||
* We need special implementation for common AST, because in case, when we interpret tuple or array
|
||||
* as function, `evaluateConstantExpression` works extremely slow.
|
||||
*/
|
||||
static Block createBlockForSet(
|
||||
const DataTypePtr & left_arg_type,
|
||||
const ASTPtr & right_arg,
|
||||
const DataTypes & set_element_types,
|
||||
const Context & context)
|
||||
{
|
||||
auto get_tuple_type_from_ast = [&context](const ASTPtr & tuple_ast) -> DataTypePtr
|
||||
{
|
||||
const auto * func = tuple_ast->as<ASTFunction>();
|
||||
if (func && (func->name == "tuple" || func->name == "array") && !func->arguments->children.empty())
|
||||
{
|
||||
/// Won't parse all values of outer tuple.
|
||||
auto element = func->arguments->children.at(0);
|
||||
std::pair<Field, DataTypePtr> value_raw = evaluateConstantExpression(element, context);
|
||||
return std::make_shared<DataTypeTuple>(DataTypes({value_raw.second}));
|
||||
}
|
||||
|
||||
return evaluateConstantExpression(tuple_ast, context).second;
|
||||
};
|
||||
|
||||
const DataTypePtr & right_arg_type = get_tuple_type_from_ast(right_arg);
|
||||
|
||||
size_t left_tuple_depth = getTypeDepth(left_arg_type);
|
||||
size_t right_tuple_depth = getTypeDepth(right_arg_type);
|
||||
ASTPtr elements_ast;
|
||||
|
||||
/// 1 in 1; (1, 2) in (1, 2); identity(tuple(tuple(tuple(1)))) in tuple(tuple(tuple(1))); etc.
|
||||
if (left_tuple_depth == right_tuple_depth)
|
||||
{
|
||||
ASTPtr exp_list = std::make_shared<ASTExpressionList>();
|
||||
exp_list->children.push_back(right_arg);
|
||||
elements_ast = exp_list;
|
||||
}
|
||||
/// 1 in (1, 2); (1, 2) in ((1, 2), (3, 4)); etc.
|
||||
else if (left_tuple_depth + 1 == right_tuple_depth)
|
||||
{
|
||||
const auto * set_func = right_arg->as<ASTFunction>();
|
||||
if (!set_func || (set_func->name != "tuple" && set_func->name != "array"))
|
||||
throw Exception("Incorrect type of 2nd argument for function 'in'"
|
||||
". Must be subquery or set of elements with type " + left_arg_type->getName() + ".",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
elements_ast = set_func->arguments;
|
||||
}
|
||||
else
|
||||
throw Exception("Invalid types for IN function: "
|
||||
+ left_arg_type->getName() + " and " + right_arg_type->getName() + ".",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return createBlockFromAST(elements_ast, set_element_types, context);
|
||||
}
|
||||
|
||||
SetPtr makeExplicitSet(
|
||||
const ASTFunction * node, const Block & sample_block, bool create_ordered_set,
|
||||
const Context & context, const SizeLimits & size_limits, PreparedSets & prepared_sets)
|
||||
{
|
||||
const IAST & args = *node->arguments;
|
||||
|
||||
if (args.children.size() != 2)
|
||||
throw Exception("Wrong number of arguments passed to function in", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const ASTPtr & left_arg = args.children.at(0);
|
||||
const ASTPtr & right_arg = args.children.at(1);
|
||||
|
||||
const DataTypePtr & left_arg_type = sample_block.getByName(left_arg->getColumnName()).type;
|
||||
|
||||
DataTypes set_element_types = {left_arg_type};
|
||||
const auto * left_tuple_type = typeid_cast<const DataTypeTuple *>(left_arg_type.get());
|
||||
if (left_tuple_type && left_tuple_type->getElements().size() != 1)
|
||||
set_element_types = left_tuple_type->getElements();
|
||||
|
||||
for (auto & element_type : set_element_types)
|
||||
if (const auto * low_cardinality_type = typeid_cast<const DataTypeLowCardinality *>(element_type.get()))
|
||||
element_type = low_cardinality_type->getDictionaryType();
|
||||
|
||||
auto set_key = PreparedSetKey::forLiteral(*right_arg, set_element_types);
|
||||
if (prepared_sets.count(set_key))
|
||||
return prepared_sets.at(set_key); /// Already prepared.
|
||||
|
||||
Block block;
|
||||
if (const auto & right_arg_literal = std::dynamic_pointer_cast<ASTLiteral>(right_arg))
|
||||
block = createBlockForSet(left_arg_type, right_arg_literal, set_element_types, context);
|
||||
else
|
||||
block = createBlockForSet(left_arg_type, right_arg, set_element_types, context);
|
||||
|
||||
SetPtr set = std::make_shared<Set>(size_limits, create_ordered_set, context.getSettingsRef().transform_null_in);
|
||||
set->setHeader(block.cloneEmpty());
|
||||
set->insertFromBlock(block);
|
||||
set->finishInsert();
|
||||
|
||||
prepared_sets[set_key] = set;
|
||||
return set;
|
||||
|
@ -39,7 +39,6 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int SET_SIZE_LIMIT_EXCEEDED;
|
||||
extern const int TYPE_MISMATCH;
|
||||
extern const int INCORRECT_ELEMENT_OF_SET;
|
||||
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
@ -216,97 +215,6 @@ bool Set::insertFromBlock(const Block & block)
|
||||
}
|
||||
|
||||
|
||||
static Field extractValueFromNode(const ASTPtr & node, const IDataType & type, const Context & context)
|
||||
{
|
||||
if (const auto * lit = node->as<ASTLiteral>())
|
||||
{
|
||||
return convertFieldToType(lit->value, type);
|
||||
}
|
||||
else if (node->as<ASTFunction>())
|
||||
{
|
||||
std::pair<Field, DataTypePtr> value_raw = evaluateConstantExpression(node, context);
|
||||
return convertFieldToType(value_raw.first, type, value_raw.second.get());
|
||||
}
|
||||
else
|
||||
throw Exception("Incorrect element of set. Must be literal or constant expression.", ErrorCodes::INCORRECT_ELEMENT_OF_SET);
|
||||
}
|
||||
|
||||
void Set::createFromAST(const DataTypes & types, ASTPtr node, const Context & context)
|
||||
{
|
||||
/// Will form a block with values from the set.
|
||||
|
||||
Block header;
|
||||
size_t num_columns = types.size();
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
header.insert(ColumnWithTypeAndName(types[i]->createColumn(), types[i], "_" + toString(i)));
|
||||
setHeader(header);
|
||||
|
||||
MutableColumns columns = header.cloneEmptyColumns();
|
||||
|
||||
DataTypePtr tuple_type;
|
||||
Row tuple_values;
|
||||
const auto & list = node->as<ASTExpressionList &>();
|
||||
for (const auto & elem : list.children)
|
||||
{
|
||||
if (num_columns == 1)
|
||||
{
|
||||
Field value = extractValueFromNode(elem, *types[0], context);
|
||||
|
||||
if (!value.isNull() || context.getSettingsRef().transform_null_in)
|
||||
columns[0]->insert(value);
|
||||
}
|
||||
else if (const auto * func = elem->as<ASTFunction>())
|
||||
{
|
||||
Field function_result;
|
||||
const Tuple * tuple = nullptr;
|
||||
if (func->name != "tuple")
|
||||
{
|
||||
if (!tuple_type)
|
||||
tuple_type = std::make_shared<DataTypeTuple>(types);
|
||||
|
||||
function_result = extractValueFromNode(elem, *tuple_type, context);
|
||||
if (function_result.getType() != Field::Types::Tuple)
|
||||
throw Exception("Invalid type of set. Expected tuple, got " + String(function_result.getTypeName()),
|
||||
ErrorCodes::INCORRECT_ELEMENT_OF_SET);
|
||||
|
||||
tuple = &function_result.get<Tuple>();
|
||||
}
|
||||
|
||||
size_t tuple_size = tuple ? tuple->size() : func->arguments->children.size();
|
||||
if (tuple_size != num_columns)
|
||||
throw Exception("Incorrect size of tuple in set: " + toString(tuple_size) + " instead of " + toString(num_columns),
|
||||
ErrorCodes::INCORRECT_ELEMENT_OF_SET);
|
||||
|
||||
if (tuple_values.empty())
|
||||
tuple_values.resize(tuple_size);
|
||||
|
||||
size_t i = 0;
|
||||
for (; i < tuple_size; ++i)
|
||||
{
|
||||
Field value = tuple ? (*tuple)[i]
|
||||
: extractValueFromNode(func->arguments->children[i], *types[i], context);
|
||||
|
||||
/// If at least one of the elements of the tuple has an impossible (outside the range of the type) value, then the entire tuple too.
|
||||
if (value.isNull() && !context.getSettings().transform_null_in)
|
||||
break;
|
||||
|
||||
tuple_values[i] = value;
|
||||
}
|
||||
|
||||
if (i == tuple_size)
|
||||
for (i = 0; i < tuple_size; ++i)
|
||||
columns[i]->insert(tuple_values[i]);
|
||||
}
|
||||
else
|
||||
throw Exception("Incorrect element of set", ErrorCodes::INCORRECT_ELEMENT_OF_SET);
|
||||
}
|
||||
|
||||
Block block = header.cloneWithColumns(std::move(columns));
|
||||
insertFromBlock(block);
|
||||
finishInsert();
|
||||
}
|
||||
|
||||
|
||||
ColumnPtr Set::execute(const Block & block, bool negative) const
|
||||
{
|
||||
size_t num_key_columns = block.columns();
|
||||
|
@ -41,12 +41,6 @@ public:
|
||||
/** Set can be created either from AST or from a stream of data (subquery result).
|
||||
*/
|
||||
|
||||
/** Create a Set from expression (specified literally in the query).
|
||||
* 'types' - types of what are on the left hand side of IN.
|
||||
* 'node' - list of values: 1, 2, 3 or list of tuples: (1, 2), (3, 4), (5, 6).
|
||||
*/
|
||||
void createFromAST(const DataTypes & types, ASTPtr node, const Context & context);
|
||||
|
||||
/** Create a Set from stream.
|
||||
* Call setHeader, then call insertFromBlock for each block.
|
||||
*/
|
||||
|
File diff suppressed because one or more lines are too long
13
tests/queries/0_stateless/01398_in_tuple_func.reference
Normal file
13
tests/queries/0_stateless/01398_in_tuple_func.reference
Normal file
@ -0,0 +1,13 @@
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
0
|
||||
1
|
||||
1
|
||||
1
|
||||
0
|
||||
1
|
16
tests/queries/0_stateless/01398_in_tuple_func.sql
Normal file
16
tests/queries/0_stateless/01398_in_tuple_func.sql
Normal file
@ -0,0 +1,16 @@
|
||||
select 1 in tuple(1, 2, 3, 4, 5) settings max_temporary_columns = 2;
|
||||
select (1, 2) in tuple(tuple(1, 2), tuple(3, 4), tuple(5, 6), tuple(7, 8), tuple(9, 10)) settings max_temporary_columns = 4;
|
||||
|
||||
select 1 in array(1, 2, 3, 4, 5) settings max_temporary_columns = 3;
|
||||
select (1, 2) in array(tuple(1, 2), tuple(3, 4), tuple(5, 6), tuple(7, 8), tuple(9, 10)) settings max_temporary_columns = 4;
|
||||
|
||||
select (1, 2) in tuple(1, 2);
|
||||
select (1, 2) in array((1, 3), (1, 2));
|
||||
select [1] in array([1], [2, 3]);
|
||||
select ([1], [2]) in tuple([NULL], [NULL]);
|
||||
select ([1], [2]) in tuple(([NULL], [NULL]), ([1], [2]));
|
||||
|
||||
select 4 in plus(2, 2);
|
||||
select (1, 'a') in tuple((1, 'a'), (2, 'b'), (3, 'c'));
|
||||
select (1, 'a') in tuple((2, 'b'), (3, 'c'), (4, 'd'));
|
||||
select (1, (2, 'foo')) in tuple((1, (3, 'b')), (1, (2, 'foo')));
|
Loading…
Reference in New Issue
Block a user