mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-17 21:24:28 +00:00
Add untuple.
This commit is contained in:
parent
a7098c2557
commit
69ae6ace47
@ -44,6 +44,21 @@ DataTypeTuple::DataTypeTuple(const DataTypes & elems_)
|
||||
names[i] = toString(i + 1);
|
||||
}
|
||||
|
||||
static void checkTupleNames(const Strings & names, std::function<void(const char *, int)> on_error)
|
||||
{
|
||||
std::unordered_set<String> names_set;
|
||||
for (const auto & name : names)
|
||||
{
|
||||
if (name.empty())
|
||||
on_error("Names of tuple elements cannot be empty", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (isNumericASCII(name[0]))
|
||||
on_error("Explicitly specified names of tuple elements cannot start with digit", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (!names_set.insert(name).second)
|
||||
on_error("Names of tuple elements must be unique", ErrorCodes::DUPLICATE_COLUMN);
|
||||
}
|
||||
}
|
||||
|
||||
DataTypeTuple::DataTypeTuple(const DataTypes & elems_, const Strings & names_)
|
||||
: elems(elems_), names(names_), have_explicit_names(true)
|
||||
@ -52,20 +67,15 @@ DataTypeTuple::DataTypeTuple(const DataTypes & elems_, const Strings & names_)
|
||||
if (names.size() != size)
|
||||
throw Exception("Wrong number of names passed to constructor of DataTypeTuple", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
std::unordered_set<String> names_set;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (names[i].empty())
|
||||
throw Exception("Names of tuple elements cannot be empty", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (isNumericASCII(names[i][0]))
|
||||
throw Exception("Explicitly specified names of tuple elements cannot start with digit", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (!names_set.insert(names[i]).second)
|
||||
throw Exception("Names of tuple elements must be unique", ErrorCodes::DUPLICATE_COLUMN);
|
||||
}
|
||||
checkTupleNames(names, [](const char * msg, int code) { throw Exception(msg, code); });
|
||||
}
|
||||
|
||||
bool DataTypeTuple::canBeCreatedWithNames(const Strings & names)
|
||||
{
|
||||
bool has_error = false;
|
||||
checkTupleNames(names, [&](const char *, int) { has_error = true; });
|
||||
return !has_error;
|
||||
}
|
||||
|
||||
std::string DataTypeTuple::doGetName() const
|
||||
{
|
||||
|
@ -28,6 +28,8 @@ public:
|
||||
DataTypeTuple(const DataTypes & elems);
|
||||
DataTypeTuple(const DataTypes & elems, const Strings & names);
|
||||
|
||||
static bool canBeCreatedWithNames(const Strings & names);
|
||||
|
||||
TypeIndex getTypeId() const override { return TypeIndex::Tuple; }
|
||||
std::string doGetName() const override;
|
||||
const char * getFamilyName() const override { return "Tuple"; }
|
||||
|
@ -52,12 +52,25 @@ public:
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
if (arguments.empty())
|
||||
throw Exception("Function " + getName() + " requires at least one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
return std::make_shared<DataTypeTuple>(arguments);
|
||||
DataTypes types;
|
||||
Strings names;
|
||||
|
||||
for (const auto & argument : arguments)
|
||||
{
|
||||
types.emplace_back(argument.type);
|
||||
names.emplace_back(argument.name);
|
||||
}
|
||||
|
||||
/// Create named tuple if possible.
|
||||
if (DataTypeTuple::canBeCreatedWithNames(names))
|
||||
return std::make_shared<DataTypeTuple>(types, names);
|
||||
|
||||
return std::make_shared<DataTypeTuple>(types);
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
|
@ -524,11 +524,14 @@ const ActionsDAG & ScopeStack::getLastActions() const
|
||||
|
||||
struct CachedColumnName
|
||||
{
|
||||
bool & skip_cache;
|
||||
String cached;
|
||||
|
||||
CachedColumnName(bool & skip_cache_) : skip_cache(skip_cache_) {}
|
||||
|
||||
const String & get(const ASTPtr & ast)
|
||||
{
|
||||
if (cached.empty())
|
||||
if (cached.empty() || skip_cache)
|
||||
cached = ast->getColumnName();
|
||||
return cached;
|
||||
}
|
||||
@ -539,7 +542,8 @@ bool ActionsMatcher::needChildVisit(const ASTPtr & node, const ASTPtr & child)
|
||||
/// Visit children themself
|
||||
if (node->as<ASTIdentifier>() ||
|
||||
node->as<ASTFunction>() ||
|
||||
node->as<ASTLiteral>())
|
||||
node->as<ASTLiteral>() ||
|
||||
node->as<ASTExpressionList>())
|
||||
return false;
|
||||
|
||||
/// Do not go to FROM, JOIN, UNION.
|
||||
@ -558,11 +562,115 @@ void ActionsMatcher::visit(const ASTPtr & ast, Data & data)
|
||||
visit(*node, ast, data);
|
||||
else if (const auto * literal = ast->as<ASTLiteral>())
|
||||
visit(*literal, ast, data);
|
||||
else if (auto * expression_list = ast->as<ASTExpressionList>())
|
||||
visit(*expression_list, ast, data);
|
||||
}
|
||||
|
||||
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();
|
||||
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;
|
||||
}
|
||||
|
||||
const auto & index = data.actions_stack.getLastActions().getIndex();
|
||||
auto it = index.find(child_column_name);
|
||||
if (it != index.end())
|
||||
return NameAndTypePair(child_column_name, it->second->result_type);
|
||||
|
||||
if (!data.only_consts)
|
||||
throw Exception("Unknown identifier: " + child_column_name + " there are columns: " + data.actions_stack.dumpNames(),
|
||||
ErrorCodes::UNKNOWN_IDENTIFIER);
|
||||
|
||||
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;
|
||||
for (const auto & name : tuple_type->getElementNames())
|
||||
{
|
||||
auto tuple_ast = function->arguments->children[0];
|
||||
if (tid != 0)
|
||||
tuple_ast = tuple_ast->clone();
|
||||
|
||||
auto literal = std::make_shared<ASTLiteral>(UInt64(++tid));
|
||||
visit(*literal, literal, data);
|
||||
|
||||
auto func = makeASTFunction("tupleElement", tuple_ast, literal);
|
||||
|
||||
if (tuple_type->haveExplicitNames())
|
||||
func->setAlias(name);
|
||||
else
|
||||
func->setAlias(data.getUniqueName("_ut" + name));
|
||||
|
||||
auto function_builder = FunctionFactory::instance().get(func->name, data.context);
|
||||
data.addFunction(function_builder, {tuple_name_type->name, literal->getColumnName()}, func->getColumnName());
|
||||
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
||||
void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr & ast, Data & data)
|
||||
{
|
||||
CachedColumnName column_name;
|
||||
CachedColumnName column_name(data.has_untuple);
|
||||
if (data.hasColumn(column_name.get(ast)))
|
||||
return;
|
||||
|
||||
@ -588,7 +696,7 @@ void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr & ast,
|
||||
|
||||
void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & data)
|
||||
{
|
||||
CachedColumnName column_name;
|
||||
CachedColumnName column_name(data.has_untuple);
|
||||
if (data.hasColumn(column_name.get(ast)))
|
||||
return;
|
||||
|
||||
@ -662,20 +770,20 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
||||
|
||||
/// If the function has an argument-lambda expression, you need to determine its type before the recursive call.
|
||||
bool has_lambda_arguments = false;
|
||||
|
||||
for (size_t arg = 0; arg < node.arguments->children.size(); ++arg)
|
||||
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 * lambda = child->as<ASTFunction>();
|
||||
const auto * function = child->as<ASTFunction>();
|
||||
const auto * identifier = child->as<ASTIdentifier>();
|
||||
if (lambda && lambda->name == "lambda")
|
||||
if (function && function->name == "lambda")
|
||||
{
|
||||
/// If the argument is a lambda expression, just remember its approximate type.
|
||||
if (lambda->arguments->children.size() != 2)
|
||||
if (function->arguments->children.size() != 2)
|
||||
throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const auto * lambda_args_tuple = lambda->arguments->children.at(0)->as<ASTFunction>();
|
||||
const auto * lambda_args_tuple = function->arguments->children.at(0)->as<ASTFunction>();
|
||||
|
||||
if (!lambda_args_tuple || lambda_args_tuple->name != "tuple")
|
||||
throw Exception("First argument of lambda must be a tuple", ErrorCodes::TYPE_MISMATCH);
|
||||
@ -685,6 +793,30 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
||||
/// Select the name in the next cycle.
|
||||
argument_names.emplace_back();
|
||||
}
|
||||
else if (function && function->name == "untuple")
|
||||
{
|
||||
data.has_untuple = true;
|
||||
auto columns = doUntuple(function, data);
|
||||
|
||||
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;
|
||||
}
|
||||
else if (checkFunctionIsInOrGlobalInOperator(node) && arg == 1 && prepared_set)
|
||||
{
|
||||
ColumnWithTypeAndName column;
|
||||
@ -731,32 +863,13 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
||||
/// If the argument is not a lambda expression, call it recursively and find out its type.
|
||||
visit(child, data);
|
||||
|
||||
// In the above visit() call, 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 = child->getColumnName();
|
||||
const auto * as_literal = child->as<ASTLiteral>();
|
||||
if (as_literal)
|
||||
if (auto name_type = getNameAndTypeFromAST(child, data))
|
||||
{
|
||||
assert(!as_literal->unique_column_name.empty());
|
||||
child_column_name = as_literal->unique_column_name;
|
||||
}
|
||||
|
||||
const auto & index = data.actions_stack.getLastActions().getIndex();
|
||||
auto it = index.find(child_column_name);
|
||||
if (it != index.end())
|
||||
{
|
||||
argument_types.push_back(it->second->result_type);
|
||||
argument_names.push_back(child_column_name);
|
||||
argument_types.push_back(name_type->type);
|
||||
argument_names.push_back(name_type->name);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (data.only_consts)
|
||||
arguments_present = false;
|
||||
else
|
||||
throw Exception("Unknown identifier: " + child_column_name + " there are columns: " + data.actions_stack.dumpNames(),
|
||||
ErrorCodes::UNKNOWN_IDENTIFIER);
|
||||
}
|
||||
arguments_present = false;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -118,6 +118,7 @@ public:
|
||||
bool no_makeset;
|
||||
bool only_consts;
|
||||
bool no_storage_or_local;
|
||||
bool has_untuple = false;
|
||||
size_t visit_depth;
|
||||
ScopeStack actions_stack;
|
||||
|
||||
@ -191,8 +192,11 @@ private:
|
||||
static void visit(const ASTIdentifier & identifier, const ASTPtr & ast, Data & data);
|
||||
static void visit(const ASTFunction & node, const ASTPtr & ast, Data & data);
|
||||
static void visit(const ASTLiteral & literal, const ASTPtr & ast, Data & data);
|
||||
static void visit(ASTExpressionList & expression_list, const ASTPtr & ast, Data & data);
|
||||
|
||||
static SetPtr makeSet(const ASTFunction & node, Data & data, bool no_subqueries);
|
||||
static ASTs doUntuple(const ASTFunction * function, ActionsMatcher::Data & data);
|
||||
static std::optional<NameAndTypePair> getNameAndTypeFromAST(const ASTPtr & ast, Data & data);
|
||||
};
|
||||
|
||||
using ActionsVisitor = ActionsMatcher::Visitor;
|
||||
|
@ -1559,6 +1559,7 @@ const ActionsDAG::Node & ActionsDAG::addFunction(
|
||||
node.allow_constant_folding = node.allow_constant_folding && child.allow_constant_folding;
|
||||
|
||||
ColumnWithTypeAndName argument;
|
||||
argument.name = argument_names[i];
|
||||
argument.column = child.column;
|
||||
argument.type = child.result_type;
|
||||
|
||||
|
@ -418,6 +418,8 @@ bool ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions)
|
||||
for (const ASTFunction * node : aggregates())
|
||||
{
|
||||
AggregateDescription aggregate;
|
||||
getRootActionsNoMakeSet(node->arguments, true, actions);
|
||||
|
||||
aggregate.column_name = node->getColumnName();
|
||||
|
||||
const ASTs & arguments = node->arguments->children;
|
||||
@ -427,7 +429,6 @@ bool ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions)
|
||||
const auto & index = actions->getIndex();
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
getRootActionsNoMakeSet(arguments[i], true, actions);
|
||||
const std::string & name = arguments[i]->getColumnName();
|
||||
|
||||
auto it = index.find(name);
|
||||
@ -743,12 +744,12 @@ bool SelectQueryExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain,
|
||||
|
||||
ExpressionActionsChain::Step & step = chain.lastStep(columns_after_join);
|
||||
|
||||
getRootActions(select_query->where(), only_types, step.actions());
|
||||
|
||||
auto where_column_name = select_query->where()->getColumnName();
|
||||
step.required_output.push_back(where_column_name);
|
||||
step.can_remove_required_output = {true};
|
||||
|
||||
getRootActions(select_query->where(), only_types, step.actions());
|
||||
|
||||
auto filter_type = step.actions()->getIndex().find(where_column_name)->second->result_type;
|
||||
if (!filter_type->canBeUsedInBooleanContext())
|
||||
throw Exception("Invalid type for filter in WHERE: " + filter_type->getName(),
|
||||
@ -824,8 +825,8 @@ bool SelectQueryExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain,
|
||||
|
||||
ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns);
|
||||
|
||||
step.required_output.push_back(select_query->having()->getColumnName());
|
||||
getRootActions(select_query->having(), only_types, step.actions());
|
||||
step.required_output.push_back(select_query->having()->getColumnName());
|
||||
|
||||
return true;
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user