diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp index 0034b50f02c..27716aeca57 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace DB @@ -28,7 +29,13 @@ IInterpreterUnionOrSelectQuery::IInterpreterUnionOrSelectQuery(const ASTPtr & qu /// 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 /// (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) context->addSpecialScalar( diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index d4bb0cc2f8a..efe0da108ad 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -2,7 +2,20 @@ #include #include +#include "Common/logger_useful.h" #include +#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 #include #include @@ -14,6 +27,7 @@ #include #include #include +#include #include #include #include @@ -22,10 +36,13 @@ #include #include #include +#include #include +#include #include #include +#include #include @@ -71,33 +88,66 @@ std::optional evaluateConstantExpressionImpl(c ReplaceQueryParameterVisitor param_visitor(context->getQueryParameters()); param_visitor.visit(ast); - /// 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()) - return getFieldAndDataTypeFromLiteral(literal); - - auto actions = ExpressionAnalyzer(ast, syntax_result, context).getConstActionsDAG(); + String result_name = ast->getColumnName(); ColumnPtr result_column; DataTypePtr result_type; - String result_name = ast->getColumnName(); - for (const auto & action_node : actions.getOutputs()) + if (context->getSettingsRef().allow_experimental_analyzer) { - 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(StorageID{"dummy", "dummy"}, fake_column_descriptions); + QueryTreeNodePtr fake_table_expression = std::make_shared(storage, execution_context); + + QueryAnalysisPass query_analysis_pass(fake_table_expression); + query_analysis_pass.run(expression, execution_context); + + GlobalPlannerContextPtr global_planner_context = std::make_shared(nullptr, nullptr, FiltersForTableExpressionMap{}); + auto planner_context = std::make_shared(execution_context, global_planner_context, SelectQueryOptions{}); + + auto actions = buildActionsDAGFromExpressionNode(expression, {}, planner_context); + + if (actions.getOutputs().size() != 1) { - result_column = action_node->column; - result_type = action_node->result_type; - break; + throw Exception(ErrorCodes::LOGICAL_ERROR, "ActionsDAG contains more than 1 output for expression: {}", ast->formatForLogging()); + } + + 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()) + 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; + } } } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 0b80858800b..f54a0015999 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -462,7 +462,7 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( } std::optional optimized_stage; - if (query_info.query_tree) + if (settings.allow_experimental_analyzer) optimized_stage = getOptimizedQueryProcessingStageAnalyzer(query_info, settings); else optimized_stage = getOptimizedQueryProcessingStage(query_info, settings); diff --git a/tests/queries/0_stateless/03036_udf_user_defined_directory_in_client.reference b/tests/queries/0_stateless/03036_udf_user_defined_directory_in_client.reference index 251d054748a..76800809c34 100644 --- a/tests/queries/0_stateless/03036_udf_user_defined_directory_in_client.reference +++ b/tests/queries/0_stateless/03036_udf_user_defined_directory_in_client.reference @@ -1 +1,2 @@ Unknown function +Function with name 'xyz' does not exist. diff --git a/tests/queries/0_stateless/03036_udf_user_defined_directory_in_client.sh b/tests/queries/0_stateless/03036_udf_user_defined_directory_in_client.sh index e0a145d8456..3e93920e94c 100755 --- a/tests/queries/0_stateless/03036_udf_user_defined_directory_in_client.sh +++ b/tests/queries/0_stateless/03036_udf_user_defined_directory_in_client.sh @@ -8,7 +8,8 @@ ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test" ${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 -${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