ClickHouse/src/Interpreters/ActionsVisitor.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

1479 lines
57 KiB
C++
Raw Normal View History

2022-05-18 15:23:31 +00:00
#include <memory>
2020-08-13 20:17:18 +00:00
#include <Common/quoteString.h>
#include <Common/typeid_cast.h>
#include <Common/FieldVisitorsAccurateComparison.h>
2023-10-09 00:40:08 +00:00
#include <Common/checkStackSize.h>
2022-05-13 14:55:50 +00:00
#include <Core/ColumnNumbers.h>
2022-05-12 16:39:50 +00:00
#include <Core/ColumnWithTypeAndName.h>
2022-05-18 15:23:31 +00:00
#include <Functions/grouping.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsMiscellaneous.h>
2021-06-23 12:19:22 +00:00
#include <Functions/indexHint.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <DataTypes/DataTypeSet.h>
#include <DataTypes/DataTypeFunction.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeTuple.h>
2020-01-24 00:33:38 +00:00
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/FieldToDataType.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeFactory.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnSet.h>
#include <Storages/StorageSet.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTQueryParameter.h>
2021-09-08 18:29:38 +00:00
#include <Processors/QueryPlan/QueryPlan.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/misc.h>
#include <Interpreters/ActionsVisitor.h>
#include <Interpreters/Set.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/convertFieldToType.h>
#include <Interpreters/interpretSubquery.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Functions/UserDefined/UserDefinedExecutableFunctionFactory.h>
#include <Parsers/QueryParameterVisitor.h>
2021-09-09 13:47:48 +00:00
#include <Analyzer/QueryNode.h>
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
#include <Parsers/queryToString.h>
2023-07-17 02:49:04 +00:00
namespace DB
{
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int UNKNOWN_IDENTIFIER;
extern const int NOT_AN_AGGREGATE;
extern const int UNEXPECTED_EXPRESSION;
extern const int TYPE_MISMATCH;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
2020-01-24 00:33:38 +00:00
extern const int INCORRECT_ELEMENT_OF_SET;
2020-09-13 13:51:31 +00:00
extern const int BAD_ARGUMENTS;
2021-03-03 20:01:07 +00:00
extern const int DUPLICATE_COLUMN;
extern const int LOGICAL_ERROR;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
2023-07-07 19:51:22 +00:00
extern const int FUNCTION_CANNOT_HAVE_PARAMETERS;
2023-11-30 13:20:32 +00:00
extern const int SYNTAX_ERROR;
}
2019-12-15 06:34:43 +00:00
static NamesAndTypesList::iterator findColumn(const String & name, NamesAndTypesList & cols)
2018-11-02 18:53:23 +00:00
{
return std::find_if(cols.begin(), cols.end(),
[&](const NamesAndTypesList::value_type & val) { return val.name == name; });
}
2020-07-31 11:39:09 +00:00
/// Recursion is limited in query parser and we did not check for too large depth here.
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;
}
/// The `convertFieldToTypeStrict` is used to prevent unexpected results in case of conversion with loss of precision.
/// Example: `SELECT 33.3 :: Decimal(9, 1) AS a WHERE a IN (33.33 :: Decimal(9, 2))`
/// 33.33 in the set is converted to 33.3, but it is not equal to 33.3 in the column, so the result should still be empty.
/// We can not include values that don't represent any possible value from the type of filtered column to the set.
2020-01-24 21:36:28 +00:00
template<typename Collection>
2020-10-07 15:36:34 +00:00
static Block createBlockFromCollection(const Collection & collection, const DataTypes & types, bool transform_null_in)
2020-01-24 00:33:38 +00:00
{
size_t columns_num = types.size();
MutableColumns columns(columns_num);
for (size_t i = 0; i < columns_num; ++i)
2022-03-16 11:21:18 +00:00
{
2020-01-24 00:33:38 +00:00
columns[i] = types[i]->createColumn();
2022-03-16 11:21:18 +00:00
columns[i]->reserve(collection.size());
}
2020-01-24 00:33:38 +00:00
Row tuple_values;
2020-01-24 21:36:28 +00:00
for (const auto & value : collection)
2020-01-24 00:33:38 +00:00
{
if (columns_num == 1)
{
2023-05-04 14:23:24 +00:00
auto field = convertFieldToTypeStrict(value, *types[0]);
2020-10-07 15:36:34 +00:00
bool need_insert_null = transform_null_in && types[0]->isNullable();
2023-05-04 14:23:24 +00:00
if (field && (!field->isNull() || need_insert_null))
columns[0]->insert(*field);
2020-01-24 00:33:38 +00:00
}
else
{
if (value.getType() != Field::Types::Tuple)
throw Exception(ErrorCodes::INCORRECT_ELEMENT_OF_SET, "Invalid type in set. Expected tuple, got {}",
String(value.getTypeName()));
2020-01-24 00:33:38 +00:00
2022-09-10 03:04:40 +00:00
const auto & tuple = value.template get<const Tuple &>();
2020-01-24 00:33:38 +00:00
size_t tuple_size = tuple.size();
if (tuple_size != columns_num)
throw Exception(ErrorCodes::INCORRECT_ELEMENT_OF_SET, "Incorrect size of tuple in set: {} instead of {}",
tuple_size, columns_num);
2020-01-24 00:33:38 +00:00
if (tuple_values.empty())
tuple_values.resize(tuple_size);
size_t i = 0;
for (; i < tuple_size; ++i)
{
2023-05-04 14:23:24 +00:00
auto converted_field = convertFieldToTypeStrict(tuple[i], *types[i]);
if (!converted_field)
break;
2023-05-04 14:23:24 +00:00
tuple_values[i] = std::move(*converted_field);
2020-10-07 15:36:34 +00:00
bool need_insert_null = transform_null_in && types[i]->isNullable();
if (tuple_values[i].isNull() && !need_insert_null)
2020-01-24 00:33:38 +00:00
break;
}
if (i == tuple_size)
for (i = 0; i < tuple_size; ++i)
columns[i]->insert(tuple_values[i]);
2020-01-24 00:33:38 +00:00
}
}
Block res;
for (size_t i = 0; i < columns_num; ++i)
res.insert(ColumnWithTypeAndName{std::move(columns[i]), types[i], "_" + toString(i)});
return res;
}
static Field extractValueFromNode(const ASTPtr & node, const IDataType & type, ContextPtr 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(ErrorCodes::INCORRECT_ELEMENT_OF_SET, "Incorrect element of set. Must be literal or constant expression.");
}
static Block createBlockFromAST(const ASTPtr & node, const DataTypes & types, ContextPtr context)
{
2020-08-01 23:06:08 +00:00
/// 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)));
MutableColumns columns = header.cloneEmptyColumns();
DataTypePtr tuple_type;
Row tuple_values;
const auto & list = node->as<ASTExpressionList &>();
bool transform_null_in = context->getSettingsRef().transform_null_in;
for (const auto & elem : list.children)
{
if (num_columns == 1)
{
/// One column at the left of IN.
Field value = extractValueFromNode(elem, *types[0], context);
2020-10-07 15:36:34 +00:00
bool need_insert_null = transform_null_in && types[0]->isNullable();
2020-10-07 15:36:34 +00:00
if (!value.isNull() || need_insert_null)
columns[0]->insert(value);
}
else if (elem->as<ASTFunction>() || elem->as<ASTLiteral>())
{
/// Multiple columns at the left of IN.
/// The right hand side of in should be a set of tuples.
Field function_result;
const Tuple * tuple = nullptr;
/// Tuple can be represented as a function in AST.
auto * func = elem->as<ASTFunction>();
if (func && func->name != "tuple")
{
if (!tuple_type)
tuple_type = std::make_shared<DataTypeTuple>(types);
/// If the function is not a tuple, treat it as a constant expression that returns tuple and extract it.
function_result = extractValueFromNode(elem, *tuple_type, context);
2021-09-06 15:59:46 +00:00
if (function_result.getType() != Field::Types::Tuple)
2021-09-06 15:59:46 +00:00
throw Exception(ErrorCodes::INCORRECT_ELEMENT_OF_SET,
"Invalid type of set. Expected tuple, got {}",
function_result.getTypeName());
tuple = &function_result.get<Tuple>();
}
/// Tuple can be represented as a literal in AST.
auto * literal = elem->as<ASTLiteral>();
if (literal)
{
/// The literal must be tuple.
if (literal->value.getType() != Field::Types::Tuple)
2021-09-06 15:59:46 +00:00
throw Exception(ErrorCodes::INCORRECT_ELEMENT_OF_SET,
"Invalid type in set. Expected tuple, got {}",
literal->value.getTypeName());
tuple = &literal->value.get<Tuple>();
}
2021-05-08 15:35:09 +00:00
assert(tuple || func);
2023-02-19 22:15:09 +00:00
size_t tuple_size = tuple ? tuple->size() : func->arguments->children.size();
if (tuple_size != num_columns)
throw Exception(ErrorCodes::INCORRECT_ELEMENT_OF_SET, "Incorrect size of tuple in set: {} instead of {}",
tuple_size, num_columns);
if (tuple_values.empty())
tuple_values.resize(tuple_size);
/// Fill tuple values by evaluation of constant expressions.
size_t i = 0;
for (; i < tuple_size; ++i)
{
Field value = tuple ? convertFieldToType((*tuple)[i], *types[i])
: extractValueFromNode(func->arguments->children[i], *types[i], context);
2020-10-07 15:36:34 +00:00
bool need_insert_null = transform_null_in && types[i]->isNullable();
/// 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.
2020-10-07 15:36:34 +00:00
if (value.isNull() && !need_insert_null)
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(ErrorCodes::INCORRECT_ELEMENT_OF_SET, "Incorrect element of set");
}
return header.cloneWithColumns(std::move(columns));
}
2022-04-15 21:59:49 +00:00
namespace
{
/** 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 ASTFunction, because in case, when we interpret
* large tuple or array as function, `evaluateConstantExpression` works extremely slow.
*/
Block createBlockForSet(
const DataTypePtr & left_arg_type,
2020-07-23 10:49:50 +00:00
const ASTPtr & right_arg,
const DataTypes & set_element_types,
ContextPtr context)
{
auto [right_arg_value, right_arg_type] = evaluateConstantExpression(right_arg, context);
const size_t left_type_depth = getTypeDepth(left_arg_type);
const size_t right_type_depth = getTypeDepth(right_arg_type);
2020-01-24 21:36:28 +00:00
auto throw_unsupported_type = [](const auto & type)
{
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Unsupported value type at the right-side of IN: {}.",
type->getName());
2020-01-24 21:36:28 +00:00
};
Block block;
bool tranform_null_in = context->getSettingsRef().transform_null_in;
2020-10-07 15:36:34 +00:00
/// 1 in 1; (1, 2) in (1, 2); identity(tuple(tuple(tuple(1)))) in tuple(tuple(tuple(1))); etc.
2020-01-24 00:33:38 +00:00
if (left_type_depth == right_type_depth)
{
2020-01-24 00:33:38 +00:00
Array array{right_arg_value};
2020-10-07 15:36:34 +00:00
block = createBlockFromCollection(array, set_element_types, tranform_null_in);
}
/// 1 in (1, 2); (1, 2) in ((1, 2), (3, 4)); etc.
2020-01-24 00:33:38 +00:00
else if (left_type_depth + 1 == right_type_depth)
{
2020-01-24 21:36:28 +00:00
auto type_index = right_arg_type->getTypeId();
if (type_index == TypeIndex::Tuple)
2022-09-10 03:04:40 +00:00
block = createBlockFromCollection(right_arg_value.get<const Tuple &>(), set_element_types, tranform_null_in);
2020-01-24 21:36:28 +00:00
else if (type_index == TypeIndex::Array)
2022-09-10 03:04:40 +00:00
block = createBlockFromCollection(right_arg_value.get<const Array &>(), set_element_types, tranform_null_in);
2020-01-24 21:36:28 +00:00
else
throw_unsupported_type(right_arg_type);
}
else
2020-01-24 21:36:28 +00:00
throw_unsupported_type(right_arg_type);
return block;
}
2022-04-15 21:59:49 +00:00
/** 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.
*/
Block createBlockForSet(
const DataTypePtr & left_arg_type,
2020-07-23 10:49:50 +00:00
const std::shared_ptr<ASTFunction> & right_arg,
const DataTypes & set_element_types,
ContextPtr context)
{
auto get_tuple_type_from_ast = [context](const auto & func) -> DataTypePtr
{
if ((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}));
}
2020-07-23 10:49:50 +00:00
return evaluateConstantExpression(func, context).second;
};
assert(right_arg);
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(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Incorrect type of 2nd argument for function 'in'. "
"Must be subquery or set of elements with type {}.", left_arg_type->getName());
elements_ast = set_func->arguments;
}
else
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Invalid types for IN function: {} and {}.",
left_arg_type->getName(), right_arg_type->getName());
return createBlockFromAST(elements_ast, set_element_types, context);
}
2022-04-15 21:59:49 +00:00
}
2023-04-25 18:14:08 +00:00
FutureSetPtr makeExplicitSet(
const ASTFunction * node, const ActionsDAG & actions, ContextPtr context, PreparedSets & prepared_sets)
{
const IAST & args = *node->arguments;
if (args.children.size() != 2)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Wrong number of arguments passed to function in");
const ASTPtr & left_arg = args.children.at(0);
const ASTPtr & right_arg = args.children.at(1);
auto column_name = left_arg->getColumnName();
2022-08-08 15:54:51 +00:00
const auto & dag_node = actions.findInOutputs(column_name);
2021-03-11 17:03:39 +00:00
const DataTypePtr & left_arg_type = dag_node.result_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();
2023-06-20 19:12:22 +00:00
auto set_element_keys = Set::getElementTypes(set_element_types, context->getSettingsRef().transform_null_in);
auto set_key = right_arg->getTreeHash(/*ignore_aliases=*/ true);
2023-06-20 19:12:22 +00:00
if (auto set = prepared_sets.findTuple(set_key, set_element_keys))
2022-07-18 15:53:30 +00:00
return set; /// Already prepared.
2023-06-22 14:23:04 +00:00
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();
Block block;
2020-07-23 10:49:50 +00:00
const auto & right_arg_func = std::dynamic_pointer_cast<ASTFunction>(right_arg);
if (right_arg_func && (right_arg_func->name == "tuple" || right_arg_func->name == "array"))
block = createBlockForSet(left_arg_type, right_arg_func, set_element_types, context);
else
block = createBlockForSet(left_arg_type, right_arg, set_element_types, context);
2023-05-25 13:33:52 +00:00
return prepared_sets.addFromTuple(set_key, block, context->getSettings());
}
2021-03-03 20:01:07 +00:00
class ScopeStack::Index
{
/// Map column name -> Node.
/// Use string_view as key which always points to Node::result_name.
std::unordered_map<std::string_view, const ActionsDAG::Node *> map;
ActionsDAG::NodeRawConstPtrs & index;
public:
2021-03-04 17:38:12 +00:00
explicit Index(ActionsDAG::NodeRawConstPtrs & index_) : index(index_)
{
for (const auto * node : index)
map.emplace(node->result_name, node);
}
2021-03-03 20:01:07 +00:00
void addNode(const ActionsDAG::Node * node)
{
bool inserted = map.emplace(node->result_name, node).second;
if (!inserted)
throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Column '{}' already exists", node->result_name);
2021-03-03 20:01:07 +00:00
index.push_back(node);
}
const ActionsDAG::Node * tryGetNode(const std::string & name) const
{
auto it = map.find(name);
if (it == map.end())
return nullptr;
return it->second;
}
const ActionsDAG::Node & getNode(const std::string & name) const
{
const auto * node = tryGetNode(name);
if (!node)
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown identifier: '{}'", name);
2021-03-03 20:01:07 +00:00
return *node;
}
bool contains(const std::string & name) const { return map.contains(name); }
2022-06-13 22:39:18 +00:00
std::vector<std::string_view> getAllNames() const
{
std::vector<std::string_view> result;
result.reserve(map.size());
for (auto const & e : map)
result.emplace_back(e.first);
return result;
}
2021-03-03 20:01:07 +00:00
};
ScopeStack::Level::~Level() = default;
ScopeStack::Level::Level() = default;
ScopeStack::Level::Level(Level &&) noexcept = default;
2020-09-30 12:32:37 +00:00
ActionsMatcher::Data::Data(
2022-05-12 16:39:50 +00:00
ContextPtr context_,
SizeLimits set_size_limit_,
size_t subquery_depth_,
2022-05-25 23:22:29 +00:00
std::reference_wrapper<const NamesAndTypesList> source_columns_,
2022-05-12 16:39:50 +00:00
ActionsDAGPtr actions_dag,
2022-07-18 15:53:30 +00:00
PreparedSetsPtr prepared_sets_,
2022-05-12 16:39:50 +00:00
bool no_subqueries_,
bool no_makeset_,
bool only_consts_,
AggregationKeysInfo aggregation_keys_info_,
bool build_expression_with_window_functions_,
bool is_create_parameterized_view_)
: WithContext(context_)
2020-09-30 12:32:37 +00:00
, set_size_limit(set_size_limit_)
, subquery_depth(subquery_depth_)
, source_columns(source_columns_)
, prepared_sets(prepared_sets_)
, no_subqueries(no_subqueries_)
, no_makeset(no_makeset_)
, only_consts(only_consts_)
, visit_depth(0)
, actions_stack(std::move(actions_dag), context_)
2022-05-25 23:22:29 +00:00
, aggregation_keys_info(aggregation_keys_info_)
, build_expression_with_window_functions(build_expression_with_window_functions_)
, is_create_parameterized_view(is_create_parameterized_view_)
2022-08-08 15:54:51 +00:00
, next_unique_suffix(actions_stack.getLastActions().getOutputs().size() + 1)
2020-09-30 12:32:37 +00:00
{
}
bool ActionsMatcher::Data::hasColumn(const String & column_name) const
{
2021-03-03 20:01:07 +00:00
return actions_stack.getLastActionsIndex().contains(column_name);
2020-09-30 12:32:37 +00:00
}
2022-06-13 22:39:18 +00:00
std::vector<std::string_view> ActionsMatcher::Data::getAllColumnNames() const
{
const auto & index = actions_stack.getLastActionsIndex();
return index.getAllNames();
}
ScopeStack::ScopeStack(ActionsDAGPtr actions_dag, ContextPtr context_) : WithContext(context_)
{
2020-09-13 13:51:31 +00:00
auto & level = stack.emplace_back();
2020-11-11 16:52:27 +00:00
level.actions_dag = std::move(actions_dag);
2022-08-08 15:54:51 +00:00
level.index = std::make_unique<ScopeStack::Index>(level.actions_dag->getOutputs());
2020-09-13 13:51:31 +00:00
2022-08-08 15:54:51 +00:00
for (const auto & node : level.actions_dag->getOutputs())
2020-11-10 14:54:59 +00:00
if (node->type == ActionsDAG::ActionType::INPUT)
2020-11-03 11:28:28 +00:00
level.inputs.emplace(node->result_name);
}
void ScopeStack::pushLevel(const NamesAndTypesList & input_columns)
{
2020-09-13 13:51:31 +00:00
auto & level = stack.emplace_back();
2020-11-11 16:52:27 +00:00
level.actions_dag = std::make_shared<ActionsDAG>();
2022-08-08 15:54:51 +00:00
level.index = std::make_unique<ScopeStack::Index>(level.actions_dag->getOutputs());
2020-09-10 07:30:03 +00:00
const auto & prev = stack[stack.size() - 2];
2020-03-08 23:48:08 +00:00
for (const auto & input_column : input_columns)
2020-09-13 13:51:31 +00:00
{
2021-03-03 20:01:07 +00:00
const auto & node = level.actions_dag->addInput(input_column.name, input_column.type);
level.index->addNode(&node);
2020-09-13 13:51:31 +00:00
level.inputs.emplace(input_column.name);
}
2022-08-08 15:54:51 +00:00
for (const auto & node : prev.actions_dag->getOutputs())
{
2021-03-03 20:01:07 +00:00
if (!level.index->contains(node->result_name))
{
const auto & input = level.actions_dag->addInput({node->column, node->result_type, node->result_name});
level.index->addNode(&input);
}
}
}
size_t ScopeStack::getColumnLevel(const std::string & name)
{
2020-09-30 11:45:47 +00:00
for (size_t i = stack.size(); i > 0;)
2020-09-13 13:51:31 +00:00
{
2020-09-30 11:45:47 +00:00
--i;
if (stack[i].inputs.contains(name))
return i;
2021-03-03 20:01:07 +00:00
const auto * node = stack[i].index->tryGetNode(name);
if (node && node->type != ActionsDAG::ActionType::INPUT)
2020-09-13 13:51:31 +00:00
return i;
}
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown identifier: {}", name);
}
2020-09-10 16:01:41 +00:00
void ScopeStack::addColumn(ColumnWithTypeAndName column)
{
2020-11-11 16:52:27 +00:00
const auto & node = stack[0].actions_dag->addColumn(std::move(column));
2021-03-03 20:01:07 +00:00
stack[0].index->addNode(&node);
2020-09-10 16:01:41 +00:00
2020-09-13 13:51:31 +00:00
for (size_t j = 1; j < stack.size(); ++j)
2021-03-03 20:01:07 +00:00
{
const auto & input = stack[j].actions_dag->addInput({node.column, node.result_type, node.result_name});
stack[j].index->addNode(&input);
}
2020-09-10 16:01:41 +00:00
}
2020-09-10 07:30:03 +00:00
void ScopeStack::addAlias(const std::string & name, std::string alias)
{
2020-09-13 13:51:31 +00:00
auto level = getColumnLevel(name);
2021-03-03 20:01:07 +00:00
const auto & source = stack[level].index->getNode(name);
const auto & node = stack[level].actions_dag->addAlias(source, std::move(alias));
stack[level].index->addNode(&node);
2020-09-10 07:30:03 +00:00
for (size_t j = level + 1; j < stack.size(); ++j)
2021-03-03 20:01:07 +00:00
{
const auto & input = stack[j].actions_dag->addInput({node.column, node.result_type, node.result_name});
stack[j].index->addNode(&input);
}
2020-09-10 07:30:03 +00:00
}
2020-11-03 11:28:28 +00:00
void ScopeStack::addArrayJoin(const std::string & source_name, std::string result_name)
2020-09-10 07:30:03 +00:00
{
2020-09-13 13:51:31 +00:00
getColumnLevel(source_name);
2020-09-10 07:30:03 +00:00
2021-03-03 20:01:07 +00:00
const auto * source_node = stack.front().index->tryGetNode(source_name);
if (!source_node)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression with arrayJoin cannot depend on lambda argument: {}",
source_name);
2020-09-13 13:51:31 +00:00
2021-03-03 20:01:07 +00:00
const auto & node = stack.front().actions_dag->addArrayJoin(*source_node, std::move(result_name));
stack.front().index->addNode(&node);
2020-09-13 13:51:31 +00:00
for (size_t j = 1; j < stack.size(); ++j)
2021-03-03 20:01:07 +00:00
{
const auto & input = stack[j].actions_dag->addInput({node.column, node.result_type, node.result_name});
stack[j].index->addNode(&input);
}
2020-09-10 07:30:03 +00:00
}
2020-09-10 16:01:41 +00:00
void ScopeStack::addFunction(
const FunctionOverloadResolverPtr & function,
const Names & argument_names,
2020-11-03 11:28:28 +00:00
std::string result_name)
{
size_t level = 0;
2020-09-10 16:01:41 +00:00
for (const auto & argument : argument_names)
level = std::max(level, getColumnLevel(argument));
2021-03-03 20:01:07 +00:00
ActionsDAG::NodeRawConstPtrs children;
children.reserve(argument_names.size());
for (const auto & argument : argument_names)
children.push_back(&stack[level].index->getNode(argument));
const auto & node = stack[level].actions_dag->addFunction(function, std::move(children), std::move(result_name));
stack[level].index->addNode(&node);
2020-09-10 16:01:41 +00:00
for (size_t j = level + 1; j < stack.size(); ++j)
2021-03-03 20:01:07 +00:00
{
const auto & input = stack[j].actions_dag->addInput({node.column, node.result_type, node.result_name});
stack[j].index->addNode(&input);
}
}
2020-09-10 07:30:03 +00:00
ActionsDAGPtr ScopeStack::popLevel()
{
2021-03-03 20:01:07 +00:00
auto res = std::move(stack.back().actions_dag);
stack.pop_back();
2021-03-03 20:01:07 +00:00
return res;
}
2020-09-10 16:01:41 +00:00
std::string ScopeStack::dumpNames() const
{
2020-11-11 16:52:27 +00:00
return stack.back().actions_dag->dumpNames();
2020-09-10 16:01:41 +00:00
}
2020-09-30 11:45:47 +00:00
const ActionsDAG & ScopeStack::getLastActions() const
{
2020-11-11 16:52:27 +00:00
return *stack.back().actions_dag;
}
2021-03-03 20:01:07 +00:00
const ScopeStack::Index & ScopeStack::getLastActionsIndex() const
{
return *stack.back().index;
}
bool ActionsMatcher::needChildVisit(const ASTPtr & node, const ASTPtr & child)
{
/// Visit children themself
if (node->as<ASTIdentifier>() ||
2020-11-13 14:13:27 +00:00
node->as<ASTTableIdentifier>() ||
node->as<ASTFunction>() ||
2020-10-21 18:17:27 +00:00
node->as<ASTLiteral>() ||
node->as<ASTExpressionList>())
return false;
2023-07-17 02:49:04 +00:00
/// Do not go to FROM, JOIN, UNION
if (child->as<ASTTableExpression>() ||
child->as<ASTSelectQuery>())
return false;
return true;
}
void ActionsMatcher::visit(const ASTPtr & ast, Data & data)
{
2023-10-09 00:40:08 +00:00
checkStackSize();
if (const auto * identifier = ast->as<ASTIdentifier>())
visit(*identifier, ast, data);
2020-11-13 14:13:27 +00:00
else if (const auto * table = ast->as<ASTTableIdentifier>())
visit(*table, ast, data);
else if (const auto * node = ast->as<ASTFunction>())
visit(*node, ast, data);
else if (const auto * literal = ast->as<ASTLiteral>())
visit(*literal, ast, data);
2020-10-21 18:17:27 +00:00
else if (auto * expression_list = ast->as<ASTExpressionList>())
visit(*expression_list, ast, data);
2020-10-22 06:58:20 +00:00
else
{
for (auto & child : ast->children)
2020-10-22 08:25:24 +00:00
if (needChildVisit(ast, child))
visit(child, data);
2020-10-22 06:58:20 +00:00
}
2020-10-21 18:17:27 +00:00
}
std::optional<NameAndTypePair> ActionsMatcher::getNameAndTypeFromAST(const ASTPtr & ast, Data & data)
{
// If the argument is a literal, we generated a unique column name for it.
// Use it instead of a generic display name.
auto child_column_name = ast->getColumnName();
2020-10-21 18:17:27 +00:00
const auto * as_literal = ast->as<ASTLiteral>();
if (as_literal)
{
assert(!as_literal->unique_column_name.empty());
child_column_name = as_literal->unique_column_name;
}
2021-03-03 20:01:07 +00:00
const auto & index = data.actions_stack.getLastActionsIndex();
if (const auto * node = index.tryGetNode(child_column_name))
return NameAndTypePair(child_column_name, node->result_type);
2020-10-21 18:17:27 +00:00
if (!data.only_consts)
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown identifier: {}; there are columns: {}",
child_column_name, data.actions_stack.dumpNames());
2020-10-21 18:17:27 +00:00
return {};
}
ASTs ActionsMatcher::doUntuple(const ASTFunction * function, ActionsMatcher::Data & data)
{
if (function->arguments->children.size() != 1)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Number of arguments for function untuple doesn't match. Passed {}, should be 1",
function->arguments->children.size());
auto & child = function->arguments->children[0];
/// Calculate nested function.
visit(child, data);
/// Get type and name for tuple argument
auto tuple_name_type = getNameAndTypeFromAST(child, data);
if (!tuple_name_type)
return {};
const auto * tuple_type = typeid_cast<const DataTypeTuple *>(tuple_name_type->type.get());
if (!tuple_type)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Function untuple expect tuple argument, got {}",
tuple_name_type->type->getName());
ASTs columns;
size_t tid = 0;
2023-09-30 11:27:32 +00:00
auto untuple_alias = function->tryGetAlias();
2023-10-08 09:32:02 +00:00
for (const auto & element_name : tuple_type->getElementNames())
2020-10-21 18:17:27 +00:00
{
auto tuple_ast = function->arguments->children[0];
/// This transformation can lead to exponential growth of AST size, let's check it.
tuple_ast->checkSize(data.getContext()->getSettingsRef().max_ast_elements);
2020-10-21 18:17:27 +00:00
if (tid != 0)
tuple_ast = tuple_ast->clone();
auto literal = std::make_shared<ASTLiteral>(UInt64{++tid});
2020-10-21 18:17:27 +00:00
visit(*literal, literal, data);
auto func = makeASTFunction("tupleElement", tuple_ast, literal);
2023-09-30 12:11:10 +00:00
if (!untuple_alias.empty())
2023-09-30 11:14:09 +00:00
{
auto element_alias = tuple_type->haveExplicitNames() ? element_name : toString(tid);
func->setAlias(untuple_alias + "." + element_alias);
}
auto function_builder = FunctionFactory::instance().get(func->name, data.getContext());
data.addFunction(function_builder, {tuple_name_type->name, literal->getColumnName()}, func->getColumnName());
2020-10-21 18:17:27 +00:00
columns.push_back(std::move(func));
}
return columns;
}
void ActionsMatcher::visit(ASTExpressionList & expression_list, const ASTPtr &, Data & data)
{
size_t num_children = expression_list.children.size();
for (size_t i = 0; i < num_children; ++i)
{
if (const auto * function = expression_list.children[i]->as<ASTFunction>())
{
if (function->name == "untuple")
{
auto columns = doUntuple(function, data);
if (columns.empty())
continue;
expression_list.children.erase(expression_list.children.begin() + i);
expression_list.children.insert(expression_list.children.begin() + i, columns.begin(), columns.end());
num_children += columns.size() - 1;
i += columns.size() - 1;
}
else
visit(expression_list.children[i], data);
}
else
visit(expression_list.children[i], data);
}
}
2020-11-13 14:13:27 +00:00
void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr &, Data & data)
{
2020-11-13 14:13:27 +00:00
auto column_name = identifier.getColumnName();
2020-10-23 16:56:11 +00:00
if (data.hasColumn(column_name))
2019-08-15 14:22:33 +00:00
return;
2019-08-15 14:22:33 +00:00
if (!data.only_consts)
{
/// The requested column is not in the block.
/// If such a column exists in the table, then the user probably forgot to surround it with an aggregate function or add it to GROUP BY.
for (const auto & column_name_type : data.source_columns)
2020-10-01 11:35:56 +00:00
{
2020-10-23 16:56:11 +00:00
if (column_name_type.name == column_name)
2020-10-01 11:35:56 +00:00
{
2022-06-13 22:39:18 +00:00
throw Exception(ErrorCodes::NOT_AN_AGGREGATE,
"Column {} is not under aggregate function and not in GROUP BY. Have columns: {}",
backQuote(column_name), toString(data.getAllColumnNames()));
}
}
/// Special check for WITH statement alias. Add alias action to be able to use this alias.
if (identifier.prefer_alias_to_column_name && !identifier.alias.empty())
data.addAlias(identifier.name(), identifier.alias);
}
}
namespace
{
void checkFunctionHasEmptyNullsAction(const ASTFunction & node)
{
if (node.nulls_action != NullsAction::EMPTY)
throw Exception(
ErrorCodes::SYNTAX_ERROR,
"Function {} cannot use {} NULLS",
node.name,
node.nulls_action == NullsAction::IGNORE_NULLS ? "IGNORE" : "RESPECT");
}
}
void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & data)
{
auto column_name = ast->getColumnName();
2020-10-23 16:56:11 +00:00
if (data.hasColumn(column_name))
return;
2018-11-12 18:19:16 +00:00
if (node.name == "lambda")
throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION, "Unexpected lambda expression");
/// Function arrayJoin.
if (node.name == "arrayJoin")
{
if (node.arguments->children.size() != 1)
throw Exception(ErrorCodes::TYPE_MISMATCH, "arrayJoin requires exactly 1 argument");
checkFunctionHasEmptyNullsAction(node);
ASTPtr arg = node.arguments->children.at(0);
visit(arg, data);
if (!data.only_consts)
data.addArrayJoin(arg->getColumnName(), column_name);
return;
}
2022-05-12 16:39:50 +00:00
if (node.name == "grouping")
{
checkFunctionHasEmptyNullsAction(node);
if (data.only_consts)
return; // Can not perform constant folding, because this function can be executed only after GROUP BY
size_t arguments_size = node.arguments->children.size();
if (arguments_size == 0)
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function GROUPING expects at least one argument");
if (arguments_size > 64)
throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION,
"Function GROUPING can have up to 64 arguments, but {} provided", arguments_size);
2022-05-25 23:22:29 +00:00
auto keys_info = data.aggregation_keys_info;
auto aggregation_keys_number = keys_info.aggregation_keys.size();
2022-05-18 15:23:31 +00:00
ColumnNumbers arguments_indexes;
for (auto const & arg : node.arguments->children)
2022-05-12 16:39:50 +00:00
{
2022-05-25 23:22:29 +00:00
size_t pos = keys_info.aggregation_keys.getPosByName(arg->getColumnName());
2022-05-18 15:23:31 +00:00
if (pos == aggregation_keys_number)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument of GROUPING function {} is not a part of GROUP BY clause", arg->getColumnName());
arguments_indexes.push_back(pos);
}
2022-05-25 23:22:29 +00:00
switch (keys_info.group_by_kind)
2022-05-18 15:23:31 +00:00
{
case GroupByKind::GROUPING_SETS:
2022-05-13 14:55:50 +00:00
{
2022-08-30 18:49:40 +00:00
data.addFunction(std::make_shared<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionGroupingForGroupingSets>(std::move(arguments_indexes), keys_info.grouping_set_keys, data.getContext()->getSettingsRef().force_grouping_standard_compatibility)), { "__grouping_set" }, column_name);
2022-05-18 15:23:31 +00:00
break;
2022-05-13 14:55:50 +00:00
}
2022-05-18 15:23:31 +00:00
case GroupByKind::ROLLUP:
2022-08-30 18:49:40 +00:00
data.addFunction(std::make_shared<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionGroupingForRollup>(std::move(arguments_indexes), aggregation_keys_number, data.getContext()->getSettingsRef().force_grouping_standard_compatibility)), { "__grouping_set" }, column_name);
2022-05-18 15:23:31 +00:00
break;
case GroupByKind::CUBE:
2022-05-12 16:39:50 +00:00
{
2022-08-30 18:49:40 +00:00
data.addFunction(std::make_shared<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionGroupingForCube>(std::move(arguments_indexes), aggregation_keys_number, data.getContext()->getSettingsRef().force_grouping_standard_compatibility)), { "__grouping_set" }, column_name);
2022-05-18 15:23:31 +00:00
break;
2022-05-12 16:39:50 +00:00
}
2022-05-18 15:23:31 +00:00
case GroupByKind::ORDINARY:
{
2022-08-30 18:49:40 +00:00
data.addFunction(std::make_shared<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionGroupingOrdinary>(std::move(arguments_indexes), data.getContext()->getSettingsRef().force_grouping_standard_compatibility)), {}, column_name);
2022-05-18 15:23:31 +00:00
break;
}
default:
throw Exception(ErrorCodes::LOGICAL_ERROR,
2022-05-25 23:22:29 +00:00
"Unexpected kind of GROUP BY clause for GROUPING function: {}", keys_info.group_by_kind);
2022-05-13 23:04:12 +00:00
}
2022-05-12 16:39:50 +00:00
return;
}
2023-04-25 18:14:08 +00:00
FutureSetPtr prepared_set;
if (checkFunctionIsInOrGlobalInOperator(node))
{
checkFunctionHasEmptyNullsAction(node);
/// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything).
visit(node.arguments->children.at(0), data);
2023-04-04 10:01:01 +00:00
if (!data.no_makeset && !(data.is_create_parameterized_view && !analyzeReceiveQueryParams(ast).empty()))
prepared_set = makeSet(node, data, data.no_subqueries);
2023-04-25 18:14:08 +00:00
if (prepared_set)
{
/// Transform tuple or subquery into a set.
}
else
{
if (!data.only_consts)
{
/// We are in the part of the tree that we are not going to compute. You just need to define types.
/// Do not evaluate subquery and create sets. We replace "in*" function to "in*IgnoreSet".
2020-05-14 14:21:38 +00:00
auto argument_name = node.arguments->children.at(0)->getColumnName();
2020-09-10 16:01:41 +00:00
data.addFunction(
FunctionFactory::instance().get(node.name + "IgnoreSet", data.getContext()),
{argument_name, argument_name},
column_name);
}
return;
}
}
2021-02-28 07:25:56 +00:00
/// A special function `indexHint`. Everything that is inside it is not calculated
if (node.name == "indexHint")
{
checkFunctionHasEmptyNullsAction(node);
2021-06-23 12:19:22 +00:00
if (data.only_consts)
{
/// We need to collect constants inside `indexHint` for index analysis.
if (node.arguments)
{
for (const auto & arg : node.arguments->children)
visit(arg, data);
}
2021-06-23 12:19:22 +00:00
return;
}
2021-06-23 12:19:22 +00:00
/// Here we create a separate DAG for indexHint condition.
/// It will be used only for index analysis.
Data index_hint_data(
data.getContext(),
data.set_size_limit,
data.subquery_depth,
data.source_columns,
std::make_shared<ActionsDAG>(data.source_columns),
data.prepared_sets,
data.no_subqueries,
data.no_makeset,
data.only_consts,
2022-07-19 20:19:57 +00:00
data.aggregation_keys_info);
2021-06-23 12:19:22 +00:00
NamesWithAliases args;
if (node.arguments)
{
for (const auto & arg : node.arguments->children)
{
visit(arg, index_hint_data);
args.push_back({arg->getColumnNameWithoutAlias(), {}});
}
}
auto dag = index_hint_data.getActions();
dag->project(args);
auto index_hint = std::make_shared<FunctionIndexHint>();
index_hint->setActions(std::move(dag));
2021-02-28 07:25:56 +00:00
// Arguments are removed. We add function instead of constant column to avoid constant folding.
2021-06-23 12:19:22 +00:00
data.addFunction(std::make_unique<FunctionToOverloadResolverAdaptor>(index_hint), {}, column_name);
2021-02-28 07:25:56 +00:00
return;
}
2022-06-16 15:11:16 +00:00
// Now we need to correctly process window functions and any expression which depend on them.
if (node.is_window_function)
2020-12-09 11:14:40 +00:00
{
2020-12-16 21:44:05 +00:00
// Also add columns from PARTITION BY and ORDER BY of window functions.
2021-01-13 19:29:52 +00:00
if (node.window_definition)
2020-12-18 00:21:23 +00:00
{
2021-01-13 19:29:52 +00:00
visit(node.window_definition, data);
2020-12-18 00:21:23 +00:00
}
// Also manually add columns for arguments of the window function itself.
// ActionVisitor is written in such a way that this method must itself
// descend into all needed function children. Window functions can't have
// any special functions as argument, so the code below that handles
// special arguments is not needed. This is analogous to the
// appendWindowFunctionsArguments() in SelectQueryExpressionAnalyzer and
// partially duplicates its code. Probably we can remove most of the
// logic from that function, but I don't yet have it all figured out...
for (const auto & arg : node.arguments->children)
{
visit(arg, data);
}
2020-12-16 21:44:05 +00:00
// Don't need to do anything more for window functions here -- the
// resulting column is added in ExpressionAnalyzer, similar to the
// aggregate functions.
return;
}
else if (node.compute_after_window_functions)
{
if (!data.build_expression_with_window_functions)
{
2022-07-11 23:18:44 +00:00
for (const auto & arg : node.arguments->children)
{
if (auto const * function = arg->as<ASTFunction>();
function && function->name == "lambda")
{
// Lambda function is a special case. It shouldn't be visited here.
continue;
}
2022-07-11 23:18:44 +00:00
visit(arg, data);
}
return;
2022-07-11 23:18:44 +00:00
}
}
2020-12-09 11:14:40 +00:00
2020-12-16 21:44:05 +00:00
// An aggregate function can also be calculated as a window function, but we
// checked for it above, so no need to do anything more.
2022-06-16 15:11:16 +00:00
if (AggregateUtils::isAggregateFunction(node))
2020-12-16 21:44:05 +00:00
return;
FunctionOverloadResolverPtr function_builder;
auto current_context = data.getContext();
if (UserDefinedExecutableFunctionFactory::instance().has(node.name, current_context))
{
Array parameters;
if (node.parameters)
{
auto & node_parameters = node.parameters->children;
size_t parameters_size = node_parameters.size();
parameters.resize(parameters_size);
for (size_t i = 0; i < parameters_size; ++i)
{
ASTPtr literal = evaluateConstantExpressionAsLiteral(node_parameters[i], current_context);
parameters[i] = literal->as<ASTLiteral>()->value;
}
}
function_builder = UserDefinedExecutableFunctionFactory::instance().tryGet(node.name, current_context, parameters);
}
2021-09-09 13:47:48 +00:00
if (!function_builder)
{
2021-09-09 13:47:48 +00:00
try
{
function_builder = FunctionFactory::instance().get(node.name, current_context);
2021-09-09 13:47:48 +00:00
}
catch (Exception & e)
{
auto hints = AggregateFunctionFactory::instance().getHints(node.name);
if (!hints.empty())
e.addMessage("Or unknown aggregate function " + node.name + ". Maybe you meant: " + toString(hints));
throw;
}
2023-07-07 19:51:22 +00:00
/// Normal functions are not parametric for now.
if (node.parameters)
throw Exception(ErrorCodes::FUNCTION_CANNOT_HAVE_PARAMETERS, "Function {} is not parametric", node.name);
}
checkFunctionHasEmptyNullsAction(node);
Names argument_names;
DataTypes argument_types;
bool arguments_present = true;
/// If the function has an argument-lambda expression, you need to determine its type before the recursive call.
bool has_lambda_arguments = false;
if (node.arguments)
{
size_t num_arguments = node.arguments->children.size();
for (size_t arg = 0; arg < num_arguments; ++arg)
{
auto & child = node.arguments->children[arg];
const auto * function = child->as<ASTFunction>();
const auto * identifier = child->as<ASTTableIdentifier>();
const auto * query_parameter = child->as<ASTQueryParameter>();
if (function && function->name == "lambda")
{
/// If the argument is a lambda expression, just remember its approximate type.
if (function->arguments->children.size() != 2)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "lambda requires two arguments");
const auto * lambda_args_tuple = function->arguments->children.at(0)->as<ASTFunction>();
if (!lambda_args_tuple || lambda_args_tuple->name != "tuple")
throw Exception(ErrorCodes::TYPE_MISMATCH, "First argument of lambda must be a tuple");
2020-10-21 18:17:27 +00:00
has_lambda_arguments = true;
argument_types.emplace_back(std::make_shared<DataTypeFunction>(DataTypes(lambda_args_tuple->arguments->children.size())));
/// Select the name in the next cycle.
argument_names.emplace_back();
2020-10-21 18:17:27 +00:00
}
else if (function && function->name == "untuple")
{
auto columns = doUntuple(function, data);
2020-10-21 18:17:27 +00:00
if (columns.empty())
continue;
for (const auto & column : columns)
{
if (auto name_type = getNameAndTypeFromAST(column, data))
{
argument_types.push_back(name_type->type);
argument_names.push_back(name_type->name);
}
else
arguments_present = false;
}
node.arguments->children.erase(node.arguments->children.begin() + arg);
node.arguments->children.insert(node.arguments->children.begin() + arg, columns.begin(), columns.end());
num_arguments += columns.size() - 1;
arg += columns.size() - 1;
}
2023-04-25 18:14:08 +00:00
else if (checkFunctionIsInOrGlobalInOperator(node) && arg == 1 && prepared_set)
{
ColumnWithTypeAndName column;
column.type = std::make_shared<DataTypeSet>();
/// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name,
/// so that sets with the same literal representation do not fuse together (they can have different types).
2023-04-25 18:14:08 +00:00
const bool is_constant_set = typeid_cast<const FutureSetFromSubquery *>(prepared_set.get()) == nullptr;
2023-04-11 21:19:44 +00:00
if (is_constant_set)
column.name = data.getUniqueName("__set");
2019-10-27 18:12:40 +00:00
else
column.name = child->getColumnName();
if (!data.hasColumn(column.name))
{
auto column_set = ColumnSet::create(1, prepared_set);
/// If prepared_set is not empty, we have a set made with literals.
/// Create a const ColumnSet to make constant folding work
2023-04-04 10:01:01 +00:00
if (is_constant_set)
2023-04-11 21:19:44 +00:00
column.column = ColumnConst::create(std::move(column_set), 1);
else
column.column = std::move(column_set);
data.addColumn(column);
}
argument_types.push_back(column.type);
argument_names.push_back(column.name);
}
else if (identifier && (functionIsJoinGet(node.name) || functionIsDictGet(node.name)) && arg == 0)
{
auto table_id = identifier->getTableId();
table_id = data.getContext()->resolveStorageID(table_id, Context::ResolveOrdinary);
auto column_string = ColumnString::create();
column_string->insert(table_id.getDatabaseName() + "." + table_id.getTableName());
ColumnWithTypeAndName column(
ColumnConst::create(std::move(column_string), 1),
std::make_shared<DataTypeString>(),
data.getUniqueName("__" + node.name));
data.addColumn(column);
argument_types.push_back(column.type);
argument_names.push_back(column.name);
}
else if (data.is_create_parameterized_view && query_parameter)
{
const auto data_type = DataTypeFactory::instance().get(query_parameter->type);
/// During analysis for CREATE VIEW of a parameterized view, if parameter is
/// used multiple times, column is only added once
if (!data.hasColumn(query_parameter->name))
{
ColumnWithTypeAndName column(data_type, query_parameter->name);
data.addColumn(column);
}
argument_types.push_back(data_type);
argument_names.push_back(query_parameter->name);
}
else
{
/// If the argument is not a lambda expression, call it recursively and find out its type.
visit(child, data);
if (auto name_type = getNameAndTypeFromAST(child, data))
{
argument_types.push_back(name_type->type);
argument_names.push_back(name_type->name);
}
else
arguments_present = false;
}
}
if (data.only_consts && !arguments_present)
return;
if (has_lambda_arguments && !data.only_consts)
{
function_builder->getLambdaArgumentTypes(argument_types);
/// Call recursively for lambda expressions.
for (size_t i = 0; i < node.arguments->children.size(); ++i)
{
ASTPtr child = node.arguments->children[i];
const auto * lambda = child->as<ASTFunction>();
if (lambda && lambda->name == "lambda")
{
const DataTypeFunction * lambda_type = typeid_cast<const DataTypeFunction *>(argument_types[i].get());
const auto * lambda_args_tuple = lambda->arguments->children.at(0)->as<ASTFunction>();
const ASTs & lambda_arg_asts = lambda_args_tuple->arguments->children;
NamesAndTypesList lambda_arguments;
for (size_t j = 0; j < lambda_arg_asts.size(); ++j)
{
auto opt_arg_name = tryGetIdentifierName(lambda_arg_asts[j]);
if (!opt_arg_name)
throw Exception(ErrorCodes::TYPE_MISMATCH, "lambda argument declarations must be identifiers");
lambda_arguments.emplace_back(*opt_arg_name, lambda_type->getArgumentTypes()[j]);
}
data.actions_stack.pushLevel(lambda_arguments);
visit(lambda->arguments->children.at(1), data);
auto lambda_dag = data.actions_stack.popLevel();
String result_name = lambda->arguments->children.at(1)->getColumnName();
lambda_dag->removeUnusedActions(Names(1, result_name));
2020-11-03 11:28:28 +00:00
2021-03-04 17:38:12 +00:00
auto lambda_actions = std::make_shared<ExpressionActions>(
lambda_dag,
2021-05-19 14:32:07 +00:00
ExpressionActionsSettings::fromContext(data.getContext(), CompileExpressions::yes));
2020-11-03 11:28:28 +00:00
DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type;
Names captured;
Names required = lambda_actions->getRequiredColumns();
for (const auto & required_arg : required)
if (findColumn(required_arg, lambda_arguments) == lambda_arguments.end())
captured.push_back(required_arg);
/// We can not name `getColumnName()`,
/// because it does not uniquely define the expression (the types of arguments can be different).
String lambda_name = data.getUniqueName("__lambda");
2021-05-15 17:33:15 +00:00
auto function_capture = std::make_shared<FunctionCaptureOverloadResolver>(
lambda_actions, captured, lambda_arguments, result_type, result_name);
2021-05-15 17:33:15 +00:00
data.addFunction(function_capture, captured, lambda_name);
argument_types[i] = std::make_shared<DataTypeFunction>(lambda_type->getArgumentTypes(), result_type);
argument_names[i] = lambda_name;
}
}
}
}
if (data.only_consts)
{
for (const auto & argument_name : argument_names)
{
2019-08-15 14:22:33 +00:00
if (!data.hasColumn(argument_name))
{
arguments_present = false;
break;
}
}
}
if (arguments_present)
{
2020-10-23 16:56:11 +00:00
/// Calculate column name here again, because AST may be changed here (in case of untuple).
data.addFunction(function_builder, argument_names, ast->getColumnName());
}
}
void ActionsMatcher::visit(const ASTLiteral & literal, const ASTPtr & /* ast */,
Data & data)
{
DataTypePtr type = applyVisitor(FieldToDataType(), literal.value);
2020-04-01 12:16:39 +00:00
const auto value = convertFieldToType(literal.value, *type);
// FIXME why do we have a second pass with a clean sample block over the same
// AST here? Anyway, do not modify the column name if it is set already.
if (literal.unique_column_name.empty())
{
const auto default_name = literal.getColumnName();
2021-03-03 20:01:07 +00:00
const auto & index = data.actions_stack.getLastActionsIndex();
const auto * existing_column = index.tryGetNode(default_name);
2020-04-01 12:16:39 +00:00
/*
2020-04-01 12:21:13 +00:00
* To approximate CSE, bind all identical literals to a single temporary
2020-04-01 12:16:39 +00:00
* columns. We try to find the column by its default name, but after that
* we have to check that it contains the correct data. This might not be
* the case if it is a user-supplied column, or it is from under a join,
* etc.
* Overall, this is a hack around a generally poor name-based notion of
* column identity we currently use.
*/
if (existing_column
&& existing_column->column
&& isColumnConst(*existing_column->column)
&& existing_column->column->size() == 1
2020-04-15 14:10:33 +00:00
&& existing_column->column->operator[](0) == value)
2020-04-01 12:16:39 +00:00
{
const_cast<ASTLiteral &>(literal).unique_column_name = default_name;
}
else
{
const_cast<ASTLiteral &>(literal).unique_column_name
2020-04-14 17:41:06 +00:00
= data.getUniqueName(default_name);
2020-04-01 12:16:39 +00:00
}
}
if (data.hasColumn(literal.unique_column_name))
{
return;
}
ColumnWithTypeAndName column;
2020-04-01 12:16:39 +00:00
column.name = literal.unique_column_name;
column.column = type->createColumnConst(1, value);
column.type = type;
2020-09-10 16:01:41 +00:00
data.addColumn(std::move(column));
}
2023-04-25 18:14:08 +00:00
FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_subqueries)
{
2022-07-18 15:53:30 +00:00
if (!data.prepared_sets)
2023-04-11 21:19:44 +00:00
return {};
2022-07-18 15:53:30 +00:00
/** You need to convert the right argument to a set.
* This can be a table name, a value, a value enumeration, or a subquery.
* The enumeration of values is parsed as a function `tuple`.
*/
const IAST & args = *node.arguments;
2019-10-27 18:12:40 +00:00
const ASTPtr & left_in_operand = args.children.at(0);
const ASTPtr & right_in_operand = args.children.at(1);
/// If the subquery or table name for SELECT.
2020-11-13 14:13:27 +00:00
const auto * identifier = right_in_operand->as<ASTTableIdentifier>();
2019-10-27 18:12:40 +00:00
if (right_in_operand->as<ASTSubquery>() || identifier)
{
2019-10-27 18:12:40 +00:00
if (no_subqueries)
return {};
2023-06-19 19:01:56 +00:00
PreparedSets::Hash set_key;
2023-06-15 16:27:51 +00:00
if (data.getContext()->getSettingsRef().allow_experimental_analyzer && !identifier)
2023-06-14 18:10:30 +00:00
{
2023-06-22 14:23:04 +00:00
/// Here we can be only from mutation interpreter. Normal selects with analyzed use other interpreter.
/// This is a hacky way to allow reusing cache for prepared sets.
///
/// Mutation is executed in two stages:
/// * first, query 'SELECT count() FROM table WHERE ...' is executed to get the set of affected parts (using analyzer)
2023-06-22 18:16:48 +00:00
/// * second, every part is mutated separately, where plan is build "manually", using this code as well
2023-06-22 14:23:04 +00:00
/// To share the Set in between first and second stage, we should use the same hash.
/// New analyzer is uses a hash from query tree, so here we also build a query tree.
///
/// Note : this code can be safely removed, but the test 02581_share_big_sets will be too slow (and fail by timeout).
/// Note : we should use new analyzer for mutations and remove this hack.
2023-06-14 18:10:30 +00:00
InterpreterSelectQueryAnalyzer interpreter(right_in_operand, data.getContext(), SelectQueryOptions().analyze(true).subquery());
2023-06-21 20:55:27 +00:00
const auto & query_tree = interpreter.getQueryTree();
2023-06-14 18:10:30 +00:00
if (auto * query_node = query_tree->as<QueryNode>())
query_node->setIsSubquery(true);
2023-06-19 19:01:56 +00:00
set_key = query_tree->getTreeHash();
2023-06-14 18:10:30 +00:00
}
else
set_key = right_in_operand->getTreeHash(/*ignore_aliases=*/ true);
2023-06-19 19:01:56 +00:00
if (auto set = data.prepared_sets->findSubquery(set_key))
2023-04-25 18:14:08 +00:00
return set;
2024-01-05 10:53:07 +00:00
FutureSetFromSubqueryPtr external_table_set;
2023-05-25 19:18:11 +00:00
/// A special case is if the name of the table is specified on the right side of the IN statement,
/// and the table has the type Set (a previously prepared set).
if (identifier)
{
auto table_id = data.getContext()->resolveStorageID(right_in_operand);
StoragePtr table = DatabaseCatalog::instance().tryGetTable(table_id, data.getContext());
if (table)
{
if (auto set = data.prepared_sets->findStorage(set_key))
return set;
2023-04-25 18:14:08 +00:00
if (StorageSet * storage_set = dynamic_cast<StorageSet *>(table.get()))
return data.prepared_sets->addFromStorage(set_key, storage_set->getSet());
}
2023-05-25 19:18:11 +00:00
2023-05-30 18:51:18 +00:00
if (!data.getContext()->isGlobalContext())
2023-05-26 19:25:33 +00:00
{
2023-06-22 14:23:04 +00:00
/// If we are reading from storage, it can be an external table which is used for GLOBAL IN.
/// Here, we take FutureSet which is used to build external table.
/// It will be used if set is useful for primary key. During PK analysis
/// temporary table is not filled yet, so we need to fill it first.
2023-05-30 18:51:18 +00:00
if (auto tmp_table = data.getContext()->findExternalTable(identifier->getColumnName()))
external_table_set = tmp_table->future_set;
2023-05-26 19:25:33 +00:00
}
}
2023-06-21 20:55:27 +00:00
std::unique_ptr<QueryPlan> source = std::make_unique<QueryPlan>();
2020-10-23 19:08:38 +00:00
/** The following happens for GLOBAL INs or INs:
* - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1,
* in the subquery_for_set object, this subquery is set as source and the temporary table _data1 as the table.
* - this function shows the expression IN_data1.
2020-11-03 16:07:27 +00:00
*
2020-10-23 19:08:38 +00:00
* In case that we have HAVING with IN subquery, we have to force creating set for it.
2020-11-12 23:27:18 +00:00
* Also it doesn't make sense if it is GLOBAL IN or ordinary IN.
*/
{
auto interpreter = interpretSubquery(right_in_operand, data.getContext(), data.subquery_depth, {});
2023-06-21 20:55:27 +00:00
interpreter->buildQueryPlan(*source);
}
2023-12-22 17:26:31 +00:00
return data.prepared_sets->addFromSubquery(
set_key, std::move(source), nullptr, std::move(external_table_set), data.getContext()->getSettingsRef());
}
else
{
2020-09-30 11:45:47 +00:00
const auto & last_actions = data.actions_stack.getLastActions();
2021-03-03 20:01:07 +00:00
const auto & index = data.actions_stack.getLastActionsIndex();
2022-07-18 15:53:30 +00:00
if (data.prepared_sets && index.contains(left_in_operand->getColumnName()))
2019-10-27 18:12:40 +00:00
/// An explicit enumeration of values in parentheses.
2023-04-25 18:14:08 +00:00
return makeExplicitSet(&node, last_actions, data.getContext(), *data.prepared_sets);
2019-10-27 18:12:40 +00:00
else
return {};
}
}
}