mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 00:22:29 +00:00
Modifiy visitor
This commit is contained in:
parent
e84e05d849
commit
981e8a3a27
@ -4,12 +4,13 @@
|
||||
#include <unordered_set>
|
||||
#include <stack>
|
||||
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTColumnDeclaration.h>
|
||||
#include <Parsers/ASTCreateFunctionQuery.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Functions/UserDefined/UserDefinedSQLFunctionFactory.h>
|
||||
#include "Parsers/ASTColumnDeclaration.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -20,24 +21,77 @@ namespace ErrorCodes
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
}
|
||||
|
||||
void UserDefinedSQLFunctionMatcher::visit(ASTPtr & ast, Data &)
|
||||
void UserDefinedSQLFunctionVisitor::visit(ASTPtr & ast)
|
||||
{
|
||||
auto * function = ast->as<ASTFunction>();
|
||||
if (!function)
|
||||
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);
|
||||
if (replace_result)
|
||||
ast = replace_result;
|
||||
|
||||
return;
|
||||
}
|
||||
|
||||
std::unordered_set<std::string> udf_in_replace_process;
|
||||
auto replace_result = tryToReplaceFunction(*function, udf_in_replace_process);
|
||||
if (replace_result)
|
||||
ast = replace_result;
|
||||
if (auto * col_decl = ast->as<ASTColumnDeclaration>())
|
||||
{
|
||||
const auto visit_child = [&](ASTPtr & child)
|
||||
{
|
||||
if (!child)
|
||||
return;
|
||||
|
||||
auto * old_value = child.get();
|
||||
visit(child);
|
||||
ast->setOrReplace(old_value, child);
|
||||
};
|
||||
|
||||
visit_child(col_decl->default_expression);
|
||||
visit_child(col_decl->ttl);
|
||||
return;
|
||||
}
|
||||
|
||||
if (auto * storage = ast->as<ASTStorage>())
|
||||
{
|
||||
const auto visit_child = [&](IAST * child)
|
||||
{
|
||||
if (!child)
|
||||
return;
|
||||
|
||||
if (const auto * function = child->template as<ASTFunction>())
|
||||
{
|
||||
std::unordered_set<std::string> udf_in_replace_process;
|
||||
auto replace_result = tryToReplaceFunction(*function, udf_in_replace_process);
|
||||
if (replace_result)
|
||||
ast->setOrReplace(child, replace_result);
|
||||
|
||||
return;
|
||||
}
|
||||
|
||||
visit(child);
|
||||
};
|
||||
|
||||
visit_child(storage->partition_by);
|
||||
visit_child(storage->primary_key);
|
||||
visit_child(storage->order_by);
|
||||
visit_child(storage->sample_by);
|
||||
visit_child(storage->ttl_table);
|
||||
|
||||
return;
|
||||
}
|
||||
|
||||
for (auto & child : ast->children)
|
||||
visit(child);
|
||||
}
|
||||
|
||||
bool UserDefinedSQLFunctionMatcher::needChildVisit(const ASTPtr &, const ASTPtr &)
|
||||
void UserDefinedSQLFunctionVisitor::visit(IAST * ast)
|
||||
{
|
||||
return true;
|
||||
assert(ast && !ast->as<ASTFunction>());
|
||||
|
||||
for (auto & child : ast->children)
|
||||
visit(child);
|
||||
}
|
||||
|
||||
ASTPtr UserDefinedSQLFunctionMatcher::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)
|
||||
{
|
||||
if (udf_in_replace_process.find(function.name) != udf_in_replace_process.end())
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
|
||||
|
@ -19,26 +19,14 @@ class ASTFunction;
|
||||
* After applying visitor:
|
||||
* SELECT number + 1 FROM system.numbers LIMIT 10;
|
||||
*/
|
||||
class UserDefinedSQLFunctionMatcher
|
||||
class UserDefinedSQLFunctionVisitor
|
||||
{
|
||||
public:
|
||||
using Visitor = InDepthNodeVisitor<UserDefinedSQLFunctionMatcher, true>;
|
||||
|
||||
struct Data
|
||||
{
|
||||
};
|
||||
|
||||
static void visit(ASTPtr & ast, Data & data);
|
||||
static bool needChildVisit(const ASTPtr & node, const ASTPtr & child);
|
||||
|
||||
static void visit(ASTPtr & ast);
|
||||
private:
|
||||
static void visit(ASTFunction & func, const Data & data);
|
||||
|
||||
static void visit(IAST *);
|
||||
static ASTPtr tryToReplaceFunction(const ASTFunction & function, std::unordered_set<std::string> & udf_in_replace_process);
|
||||
|
||||
};
|
||||
|
||||
/// Visits AST nodes and collect their aliases in one map (with links to source nodes).
|
||||
using UserDefinedSQLFunctionVisitor = UserDefinedSQLFunctionMatcher::Visitor;
|
||||
|
||||
}
|
||||
|
@ -4,9 +4,7 @@
|
||||
#include <vector>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/checkStackSize.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Parsers/DumpASTNode.h>
|
||||
#include <Parsers/ASTColumnDeclaration.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -82,7 +80,7 @@ private:
|
||||
template <bool with_dump>
|
||||
void visitChildren(T & ast)
|
||||
{
|
||||
const auto visit_child = [&](auto & child)
|
||||
for (auto & child : ast->children)
|
||||
{
|
||||
bool need_visit_child = false;
|
||||
if constexpr (need_child_accept_data)
|
||||
@ -92,23 +90,6 @@ private:
|
||||
|
||||
if (need_visit_child)
|
||||
visitImpl<with_dump>(child);
|
||||
};
|
||||
|
||||
for (auto & child : ast->children)
|
||||
{
|
||||
visit_child(child);
|
||||
}
|
||||
|
||||
if constexpr (std::same_as<T, ASTPtr>)
|
||||
{
|
||||
if (auto * col_decl = ast->template as<ASTColumnDeclaration>())
|
||||
{
|
||||
if (col_decl->default_expression)
|
||||
visit_child(col_decl->default_expression);
|
||||
|
||||
if (col_decl->default_expression)
|
||||
visit_child(col_decl->default_expression);
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
@ -1161,10 +1161,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
|
||||
// substitute possible UDFs with their definitions
|
||||
if (!UserDefinedSQLFunctionFactory::instance().empty())
|
||||
{
|
||||
UserDefinedSQLFunctionVisitor::Data data_user_defined_functions_visitor;
|
||||
UserDefinedSQLFunctionVisitor(data_user_defined_functions_visitor).visit(query_ptr);
|
||||
}
|
||||
UserDefinedSQLFunctionVisitor::visit(query_ptr);
|
||||
|
||||
/// Set and retrieve list of columns, indices and constraints. Set table engine if needed. Rewrite query in canonical way.
|
||||
TableProperties properties = getTablePropertiesAndNormalizeCreateQuery(create);
|
||||
|
@ -1408,10 +1408,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_)
|
||||
{
|
||||
if (!UserDefinedSQLFunctionFactory::instance().empty())
|
||||
{
|
||||
UserDefinedSQLFunctionVisitor::Data data_user_defined_functions_visitor;
|
||||
UserDefinedSQLFunctionVisitor(data_user_defined_functions_visitor).visit(query);
|
||||
}
|
||||
UserDefinedSQLFunctionVisitor::visit(query);
|
||||
|
||||
CustomizeCountDistinctVisitor::Data data_count_distinct{settings.count_distinct_implementation};
|
||||
CustomizeCountDistinctVisitor(data_count_distinct).visit(query);
|
||||
|
@ -1027,10 +1027,7 @@ void AlterCommands::prepare(const StorageInMemoryMetadata & metadata)
|
||||
}
|
||||
|
||||
if (command.default_expression && !UserDefinedSQLFunctionFactory::instance().empty())
|
||||
{
|
||||
UserDefinedSQLFunctionVisitor::Data data_user_defined_functions_visitor;
|
||||
UserDefinedSQLFunctionVisitor(data_user_defined_functions_visitor).visit(command.default_expression);
|
||||
}
|
||||
UserDefinedSQLFunctionVisitor::visit(command.default_expression);
|
||||
}
|
||||
|
||||
prepared = true;
|
||||
|
Loading…
Reference in New Issue
Block a user