mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
Merge 7dcaacab9d
into 44b4bd38b9
This commit is contained in:
commit
0b57b8562d
@ -96,7 +96,7 @@ private:
|
||||
|
||||
auto [_, inserted] = aliases.alias_name_to_lambda_node.insert(std::make_pair(alias, node));
|
||||
if (!inserted)
|
||||
addDuplicatingAlias(node);
|
||||
addDuplicatingAlias(node);
|
||||
|
||||
return;
|
||||
}
|
||||
|
@ -11,7 +11,9 @@
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Functions/UserDefined/UserDefinedSQLFunctionFactory.h>
|
||||
#include "Parsers/ASTColumnDeclaration.h"
|
||||
#include <Interpreters/QueryAliasesVisitor.h>
|
||||
#include <Interpreters/MarkTableIdentifiersVisitor.h>
|
||||
#include <Interpreters/QueryNormalizer.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -22,14 +24,14 @@ namespace ErrorCodes
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
}
|
||||
|
||||
void UserDefinedSQLFunctionVisitor::visit(ASTPtr & ast)
|
||||
void UserDefinedSQLFunctionVisitor::visit(ASTPtr & ast, ContextPtr context_)
|
||||
{
|
||||
chassert(ast);
|
||||
|
||||
if (const auto * function = ast->template as<ASTFunction>())
|
||||
{
|
||||
std::unordered_set<std::string> udf_in_replace_process;
|
||||
auto replace_result = tryToReplaceFunction(*function, udf_in_replace_process);
|
||||
auto replace_result = tryToReplaceFunction(*function, udf_in_replace_process, context_);
|
||||
if (replace_result)
|
||||
ast = replace_result;
|
||||
}
|
||||
@ -40,7 +42,7 @@ void UserDefinedSQLFunctionVisitor::visit(ASTPtr & ast)
|
||||
return;
|
||||
|
||||
auto * old_ptr = child.get();
|
||||
visit(child);
|
||||
visit(child, context_);
|
||||
auto * new_ptr = child.get();
|
||||
|
||||
/// Some AST classes have naked pointers to children elements as members.
|
||||
@ -50,16 +52,16 @@ void UserDefinedSQLFunctionVisitor::visit(ASTPtr & ast)
|
||||
}
|
||||
}
|
||||
|
||||
void UserDefinedSQLFunctionVisitor::visit(IAST * ast)
|
||||
void UserDefinedSQLFunctionVisitor::visit(IAST * ast, ContextPtr context_)
|
||||
{
|
||||
if (!ast)
|
||||
return;
|
||||
|
||||
for (auto & child : ast->children)
|
||||
visit(child);
|
||||
visit(child, context_);
|
||||
}
|
||||
|
||||
ASTPtr UserDefinedSQLFunctionVisitor::tryToReplaceFunction(const ASTFunction & function, std::unordered_set<std::string> & udf_in_replace_process)
|
||||
ASTPtr UserDefinedSQLFunctionVisitor::tryToReplaceFunction(const ASTFunction & function, std::unordered_set<std::string> & udf_in_replace_process, ContextPtr context_)
|
||||
{
|
||||
if (udf_in_replace_process.find(function.name) != udf_in_replace_process.end())
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
|
||||
@ -101,6 +103,17 @@ ASTPtr UserDefinedSQLFunctionVisitor::tryToReplaceFunction(const ASTFunction & f
|
||||
|
||||
auto function_body_to_update = function_core_expression->children.at(1)->clone();
|
||||
|
||||
Aliases aliases;
|
||||
QueryAliasesVisitor(aliases).visit(function_body_to_update);
|
||||
|
||||
/// Mark table ASTIdentifiers with not a column marker
|
||||
MarkTableIdentifiersVisitor::Data identifiers_data{aliases};
|
||||
MarkTableIdentifiersVisitor(identifiers_data).visit(function_body_to_update);
|
||||
|
||||
/// Common subexpression elimination. Rewrite rules.
|
||||
QueryNormalizer::Data normalizer_data(aliases, {}, true, context_->getSettingsRef(), true, false);
|
||||
QueryNormalizer(normalizer_data).visit(function_body_to_update);
|
||||
|
||||
auto expression_list = std::make_shared<ASTExpressionList>();
|
||||
expression_list->children.emplace_back(std::move(function_body_to_update));
|
||||
|
||||
@ -116,7 +129,7 @@ ASTPtr UserDefinedSQLFunctionVisitor::tryToReplaceFunction(const ASTFunction & f
|
||||
{
|
||||
if (auto * inner_function = child->as<ASTFunction>())
|
||||
{
|
||||
auto replace_result = tryToReplaceFunction(*inner_function, udf_in_replace_process);
|
||||
auto replace_result = tryToReplaceFunction(*inner_function, udf_in_replace_process, context_);
|
||||
if (replace_result)
|
||||
child = replace_result;
|
||||
}
|
||||
|
@ -22,10 +22,10 @@ class ASTFunction;
|
||||
class UserDefinedSQLFunctionVisitor
|
||||
{
|
||||
public:
|
||||
static void visit(ASTPtr & ast);
|
||||
static void visit(ASTPtr & ast, ContextPtr context_);
|
||||
private:
|
||||
static void visit(IAST *);
|
||||
static ASTPtr tryToReplaceFunction(const ASTFunction & function, std::unordered_set<std::string> & udf_in_replace_process);
|
||||
static void visit(IAST *, ContextPtr context_);
|
||||
static ASTPtr tryToReplaceFunction(const ASTFunction & function, std::unordered_set<std::string> & udf_in_replace_process, ContextPtr context_);
|
||||
|
||||
};
|
||||
|
||||
|
@ -99,7 +99,7 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter)
|
||||
BlockIO res;
|
||||
|
||||
if (!UserDefinedSQLFunctionFactory::instance().empty())
|
||||
UserDefinedSQLFunctionVisitor::visit(query_ptr);
|
||||
UserDefinedSQLFunctionVisitor::visit(query_ptr, getContext());
|
||||
|
||||
auto table_id = getContext()->tryResolveStorageID(alter);
|
||||
StoragePtr table;
|
||||
|
@ -1599,7 +1599,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
|
||||
// substitute possible UDFs with their definitions
|
||||
if (!UserDefinedSQLFunctionFactory::instance().empty())
|
||||
UserDefinedSQLFunctionVisitor::visit(query_ptr);
|
||||
UserDefinedSQLFunctionVisitor::visit(query_ptr, getContext());
|
||||
|
||||
/// Set and retrieve list of columns, indices and constraints. Set table engine if needed. Rewrite query in canonical way.
|
||||
TableProperties properties = getTablePropertiesAndNormalizeCreateQuery(create, mode);
|
||||
|
@ -122,7 +122,7 @@ void QueryAliasesMatcher<T>::visitOther(const ASTPtr & ast, Data & data)
|
||||
if (!alias.empty())
|
||||
{
|
||||
if (aliases.contains(alias) && ast->getTreeHash(/*ignore_aliases=*/ true) != aliases[alias]->getTreeHash(/*ignore_aliases=*/ true))
|
||||
throw Exception(wrongAliasMessage(ast, aliases[alias], alias), ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS);
|
||||
throw Exception(wrongAliasMessage(ast, aliases[alias], alias), ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS);
|
||||
|
||||
aliases[alias] = ast;
|
||||
}
|
||||
|
@ -85,10 +85,10 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data)
|
||||
}
|
||||
|
||||
/// If it is an alias, but not a parent alias (for constructs like "SELECT column + 1 AS column").
|
||||
auto it_alias = data.aliases.find(node.name());
|
||||
if (!data.allow_self_aliases && current_alias == node.name())
|
||||
throw Exception(ErrorCodes::CYCLIC_ALIASES, "Self referencing of {} to {}. Cyclic alias",
|
||||
backQuote(current_alias), backQuote(node.name()));
|
||||
auto it_alias = data.aliases.find(node.name());
|
||||
|
||||
if (it_alias != data.aliases.end() && current_alias != node.name())
|
||||
{
|
||||
|
@ -1571,7 +1571,7 @@ void TreeRewriter::normalize(
|
||||
ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, bool ignore_alias, const Settings & settings, bool allow_self_aliases, ContextPtr context_, bool is_create_parameterized_view)
|
||||
{
|
||||
if (!UserDefinedSQLFunctionFactory::instance().empty())
|
||||
UserDefinedSQLFunctionVisitor::visit(query);
|
||||
UserDefinedSQLFunctionVisitor::visit(query, context_);
|
||||
|
||||
CustomizeCountDistinctVisitor::Data data_count_distinct{settings[Setting::count_distinct_implementation]};
|
||||
CustomizeCountDistinctVisitor(data_count_distinct).visit(query);
|
||||
|
5
tests/queries/0_stateless/03274_aliases_in_udf.reference
Normal file
5
tests/queries/0_stateless/03274_aliases_in_udf.reference
Normal file
@ -0,0 +1,5 @@
|
||||
FIX ISSUE #69143
|
||||
a
|
||||
b
|
||||
EXPAIN SYNTAX OF UDF
|
||||
SELECT ((4 + 2) + 1, ((4 + 2) + 1) + 2)
|
40
tests/queries/0_stateless/03274_aliases_in_udf.sql
Normal file
40
tests/queries/0_stateless/03274_aliases_in_udf.sql
Normal file
@ -0,0 +1,40 @@
|
||||
SELECT 'FIX ISSUE #69143';
|
||||
|
||||
DROP FUNCTION IF EXISTS test_function;
|
||||
DROP FUNCTION IF EXISTS test_03274;
|
||||
DROP TABLE IF EXISTS test_table;
|
||||
|
||||
CREATE OR REPLACE FUNCTION test_function AS ( input_column_name ) -> ((
|
||||
'1' AS a,
|
||||
input_column_name AS input_column_name
|
||||
).2);
|
||||
|
||||
CREATE TABLE IF NOT EXISTS test_table
|
||||
(
|
||||
`metadata_a` String,
|
||||
`metadata_b` String
|
||||
)
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY tuple();
|
||||
|
||||
|
||||
ALTER TABLE test_table ADD COLUMN mat_a String MATERIALIZED test_function(metadata_a);
|
||||
ALTER TABLE test_table MATERIALIZE COLUMN `mat_a`;
|
||||
|
||||
ALTER TABLE test_table ADD COLUMN mat_b String MATERIALIZED test_function(metadata_b);
|
||||
ALTER TABLE test_table MATERIALIZE COLUMN `mat_b`;
|
||||
|
||||
INSERT INTO test_table SELECT 'a', 'b';
|
||||
|
||||
SELECT mat_a FROM test_table;
|
||||
SELECT mat_b FROM test_table;
|
||||
|
||||
SELECT 'EXPAIN SYNTAX OF UDF';
|
||||
|
||||
CREATE OR REPLACE FUNCTION test_03274 AS ( x ) -> ((x + 1 as y, y + 2));
|
||||
|
||||
EXPAIN SYNTAX SELECT test_03274(4 + 2);
|
||||
|
||||
DROP FUNCTION IF EXISTS test_function;
|
||||
DROP FUNCTION IF EXISTS test_03274;
|
||||
DROP TABLE IF EXISTS test_table;
|
Loading…
Reference in New Issue
Block a user