This commit is contained in:
Dmitry Novik 2024-09-18 13:33:43 -07:00 committed by GitHub
commit 95ae65f185
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
5 changed files with 84 additions and 25 deletions

View File

@ -12,6 +12,7 @@
#include <Interpreters/ExpressionAnalyzer.h> #include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/TreeRewriter.h> #include <Interpreters/TreeRewriter.h>
#include <Processors/QueryPlan/FilterStep.h> #include <Processors/QueryPlan/FilterStep.h>
#include <Common/Logger.h>
namespace DB namespace DB
@ -28,7 +29,13 @@ IInterpreterUnionOrSelectQuery::IInterpreterUnionOrSelectQuery(const ASTPtr & qu
/// it's possible that new analyzer will be enabled in ::getQueryProcessingStage method /// it's possible that new analyzer will be enabled in ::getQueryProcessingStage method
/// of the underlying storage when all other parts of infrastructure are not ready for it /// of the underlying storage when all other parts of infrastructure are not ready for it
/// (built with old analyzer). /// (built with old analyzer).
context->setSetting("allow_experimental_analyzer", false); if (context->getSettingsRef().allow_experimental_analyzer)
{
LOG_TRACE(getLogger("IInterpreterUnionOrSelectQuery"),
"The new analyzer is enabled, but the old interpreter is used. It can be a bug, please report it. Will disable 'allow_experimental_analyzer' setting (for query: {})",
query_ptr->formatForLogging());
context->setSetting("allow_experimental_analyzer", false);
}
if (options.shard_num) if (options.shard_num)
context->addSpecialScalar( context->addSpecialScalar(

View File

@ -2,7 +2,20 @@
#include <Columns/ColumnConst.h> #include <Columns/ColumnConst.h>
#include <Columns/ColumnSet.h> #include <Columns/ColumnSet.h>
#include "Common/logger_useful.h"
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include "Analyzer/ConstantNode.h"
#include "Analyzer/IQueryTreeNode.h"
#include "Analyzer/Passes/QueryAnalysisPass.h"
#include "Analyzer/QueryNode.h"
#include "Analyzer/QueryTreeBuilder.h"
#include "Analyzer/TableNode.h"
#include "Interpreters/SelectQueryOptions.h"
#include "Planner/PlannerActionsVisitor.h"
#include "Planner/PlannerContext.h"
#include "Planner/Utils.h"
#include "Storages/ColumnsDescription.h"
#include "Storages/StorageDummy.h"
#include <Core/Block.h> #include <Core/Block.h>
#include <Core/Settings.h> #include <Core/Settings.h>
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
@ -14,6 +27,7 @@
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/castColumn.h> #include <Interpreters/castColumn.h>
#include <Interpreters/convertFieldToType.h> #include <Interpreters/convertFieldToType.h>
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
#include <Interpreters/ExpressionAnalyzer.h> #include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/FunctionNameNormalizer.h> #include <Interpreters/FunctionNameNormalizer.h>
#include <Interpreters/ReplaceQueryParameterVisitor.h> #include <Interpreters/ReplaceQueryParameterVisitor.h>
@ -22,10 +36,13 @@
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h> #include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSubquery.h> #include <Parsers/ASTSubquery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Processors/QueryPlan/Optimizations/actionsDAGUtils.h> #include <Processors/QueryPlan/Optimizations/actionsDAGUtils.h>
#include <Processors/Executors/PullingPipelineExecutor.h>
#include <Storages/MergeTree/KeyCondition.h> #include <Storages/MergeTree/KeyCondition.h>
#include <TableFunctions/TableFunctionFactory.h> #include <TableFunctions/TableFunctionFactory.h>
#include <optional>
#include <unordered_map> #include <unordered_map>
@ -71,33 +88,66 @@ std::optional<EvaluateConstantExpressionResult> evaluateConstantExpressionImpl(c
ReplaceQueryParameterVisitor param_visitor(context->getQueryParameters()); ReplaceQueryParameterVisitor param_visitor(context->getQueryParameters());
param_visitor.visit(ast); param_visitor.visit(ast);
/// Notice: function name normalization is disabled when it's a secondary query, because queries are either String result_name = ast->getColumnName();
/// already normalized on initiator node, or not normalized and should remain unnormalized for
/// compatibility.
if (context->getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY && context->getSettingsRef().normalize_function_names)
FunctionNameNormalizer::visit(ast.get());
auto syntax_result = TreeRewriter(context, no_throw).analyze(ast, source_columns);
if (!syntax_result)
return {};
/// AST potentially could be transformed to literal during TreeRewriter analyze.
/// For example if we have SQL user defined function that return literal AS subquery.
if (ASTLiteral * literal = ast->as<ASTLiteral>())
return getFieldAndDataTypeFromLiteral(literal);
auto actions = ExpressionAnalyzer(ast, syntax_result, context).getConstActionsDAG();
ColumnPtr result_column; ColumnPtr result_column;
DataTypePtr result_type; DataTypePtr result_type;
String result_name = ast->getColumnName(); if (context->getSettingsRef().allow_experimental_analyzer)
for (const auto & action_node : actions.getOutputs())
{ {
if ((action_node->result_name == result_name) && action_node->column) auto execution_context = Context::createCopy(context);
auto expression = buildQueryTree(ast, execution_context);
ColumnsDescription fake_column_descriptions(source_columns);
auto storage = std::make_shared<StorageDummy>(StorageID{"dummy", "dummy"}, fake_column_descriptions);
QueryTreeNodePtr fake_table_expression = std::make_shared<TableNode>(storage, execution_context);
QueryAnalysisPass query_analysis_pass(fake_table_expression);
query_analysis_pass.run(expression, execution_context);
GlobalPlannerContextPtr global_planner_context = std::make_shared<GlobalPlannerContext>(nullptr, nullptr, FiltersForTableExpressionMap{});
auto planner_context = std::make_shared<PlannerContext>(execution_context, global_planner_context, SelectQueryOptions{});
auto actions = buildActionsDAGFromExpressionNode(expression, {}, planner_context);
if (actions.getOutputs().size() != 1)
{ {
result_column = action_node->column; throw Exception(ErrorCodes::LOGICAL_ERROR, "ActionsDAG contains more than 1 output for expression: {}", ast->formatForLogging());
result_type = action_node->result_type; }
break;
const auto & output = actions.getOutputs()[0];
if (output->column)
{
result_column = output->column;
result_type = output->result_type;
}
}
else
{
/// Notice: function name normalization is disabled when it's a secondary query, because queries are either
/// already normalized on initiator node, or not normalized and should remain unnormalized for
/// compatibility.
if (context->getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY && context->getSettingsRef().normalize_function_names)
FunctionNameNormalizer::visit(ast.get());
auto syntax_result = TreeRewriter(context, no_throw).analyze(ast, source_columns);
if (!syntax_result)
return {};
/// AST potentially could be transformed to literal during TreeRewriter analyze.
/// For example if we have SQL user defined function that return literal AS subquery.
if (ASTLiteral * literal = ast->as<ASTLiteral>())
return getFieldAndDataTypeFromLiteral(literal);
auto actions = ExpressionAnalyzer(ast, syntax_result, context).getConstActionsDAG();
for (const auto & action_node : actions.getOutputs())
{
if ((action_node->result_name == result_name) && action_node->column)
{
result_column = action_node->column;
result_type = action_node->result_type;
break;
}
} }
} }

View File

@ -462,7 +462,7 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(
} }
std::optional<QueryProcessingStage::Enum> optimized_stage; std::optional<QueryProcessingStage::Enum> optimized_stage;
if (query_info.query_tree) if (settings.allow_experimental_analyzer)
optimized_stage = getOptimizedQueryProcessingStageAnalyzer(query_info, settings); optimized_stage = getOptimizedQueryProcessingStageAnalyzer(query_info, settings);
else else
optimized_stage = getOptimizedQueryProcessingStage(query_info, settings); optimized_stage = getOptimizedQueryProcessingStage(query_info, settings);

View File

@ -1 +1,2 @@
Unknown function Unknown function
Function with name 'xyz' does not exist.

View File

@ -8,7 +8,8 @@ ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test"
${CLICKHOUSE_CLIENT} --query "CREATE TABLE test (s String) ENGINE = Memory" ${CLICKHOUSE_CLIENT} --query "CREATE TABLE test (s String) ENGINE = Memory"
# Calling an unknown function should not lead to creation of a 'user_defined' directory in the current directory # Calling an unknown function should not lead to creation of a 'user_defined' directory in the current directory
${CLICKHOUSE_CLIENT} --query "INSERT INTO test VALUES (xyz('abc'))" 2>&1 | grep -o -F 'Unknown function' ${CLICKHOUSE_CLIENT} --query "SET allow_experimental_analyzer = 0;INSERT INTO test VALUES (xyz('abc'))" 2>&1 | grep -o -F 'Unknown function'
${CLICKHOUSE_CLIENT} --query "SET allow_experimental_analyzer = 1;INSERT INTO test VALUES (xyz('abc'))" 2>&1 | grep -o -F "Function with name 'xyz' does not exist."
ls -ld user_defined 2> /dev/null ls -ld user_defined 2> /dev/null