mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-30 19:42:00 +00:00
Support DESCRIBE for parameterized view; Fix usage in scalars
This commit is contained in:
parent
967ba9d3d4
commit
79140a3589
@ -74,7 +74,17 @@ public:
|
||||
return query_tree_node;
|
||||
}
|
||||
|
||||
static QueryTreeNodePtr buildForTableFunction(
|
||||
const ASTTableExpression & table_expression,
|
||||
const ContextPtr & context)
|
||||
{
|
||||
QueryTreeBuilder builder;
|
||||
return builder.buildTableFunction(table_expression, context);
|
||||
}
|
||||
|
||||
private:
|
||||
QueryTreeBuilder() = default;
|
||||
|
||||
QueryTreeNodePtr buildSelectOrUnionExpression(const ASTPtr & select_or_union_query,
|
||||
bool is_subquery,
|
||||
const std::string & cte_name,
|
||||
@ -109,6 +119,11 @@ private:
|
||||
|
||||
QueryTreeNodePtr buildJoinTree(const ASTPtr & tables_in_select_query, const ContextPtr & context) const;
|
||||
|
||||
QueryTreeNodePtr buildTableFunction(
|
||||
const ASTTableExpression & table_expression,
|
||||
const ContextPtr & context,
|
||||
const std::optional<TableExpressionModifiers> & table_expression_modifiers = {}) const;
|
||||
|
||||
ColumnTransformersNodes buildColumnTransformers(const ASTPtr & matcher_expression, const ContextPtr & context) const;
|
||||
|
||||
ASTPtr query;
|
||||
@ -854,34 +869,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select
|
||||
}
|
||||
else if (table_expression.table_function)
|
||||
{
|
||||
auto & table_function_expression = table_expression.table_function->as<ASTFunction &>();
|
||||
|
||||
auto node = std::make_shared<TableFunctionNode>(table_function_expression.name);
|
||||
|
||||
if (table_function_expression.arguments)
|
||||
{
|
||||
const auto & function_arguments_list = table_function_expression.arguments->as<ASTExpressionList &>().children;
|
||||
for (const auto & argument : function_arguments_list)
|
||||
{
|
||||
if (!node->getSettingsChanges().empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Table function '{}' has arguments after SETTINGS",
|
||||
table_function_expression.formatForErrorMessage());
|
||||
|
||||
if (argument->as<ASTSelectQuery>() || argument->as<ASTSelectWithUnionQuery>() || argument->as<ASTSelectIntersectExceptQuery>())
|
||||
node->getArguments().getNodes().push_back(buildSelectOrUnionExpression(argument, false /*is_subquery*/, {} /*cte_name*/, context));
|
||||
else if (const auto * ast_set = argument->as<ASTSetQuery>())
|
||||
node->setSettingsChanges(ast_set->changes);
|
||||
else
|
||||
node->getArguments().getNodes().push_back(buildExpression(argument, context));
|
||||
}
|
||||
}
|
||||
|
||||
if (table_expression_modifiers)
|
||||
node->setTableExpressionModifiers(*table_expression_modifiers);
|
||||
node->setAlias(table_function_expression.tryGetAlias());
|
||||
node->setOriginalAST(table_expression.table_function);
|
||||
|
||||
table_expressions.push_back(std::move(node));
|
||||
table_expressions.push_back(buildTableFunction(table_expression, context, table_expression_modifiers));
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -983,6 +971,42 @@ QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select
|
||||
}
|
||||
|
||||
|
||||
QueryTreeNodePtr QueryTreeBuilder::buildTableFunction(
|
||||
const ASTTableExpression & table_expression,
|
||||
const ContextPtr & context,
|
||||
const std::optional<TableExpressionModifiers> & table_expression_modifiers) const
|
||||
{
|
||||
auto & table_function_expression = table_expression.table_function->as<ASTFunction &>();
|
||||
|
||||
auto node = std::make_shared<TableFunctionNode>(table_function_expression.name);
|
||||
|
||||
if (table_function_expression.arguments)
|
||||
{
|
||||
const auto & function_arguments_list = table_function_expression.arguments->as<ASTExpressionList &>().children;
|
||||
for (const auto & argument : function_arguments_list)
|
||||
{
|
||||
if (!node->getSettingsChanges().empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Table function '{}' has arguments after SETTINGS",
|
||||
table_function_expression.formatForErrorMessage());
|
||||
|
||||
if (argument->as<ASTSelectQuery>() || argument->as<ASTSelectWithUnionQuery>() || argument->as<ASTSelectIntersectExceptQuery>())
|
||||
node->getArguments().getNodes().push_back(buildSelectOrUnionExpression(argument, false /*is_subquery*/, {} /*cte_name*/, context));
|
||||
else if (const auto * ast_set = argument->as<ASTSetQuery>())
|
||||
node->setSettingsChanges(ast_set->changes);
|
||||
else
|
||||
node->getArguments().getNodes().push_back(buildExpression(argument, context));
|
||||
}
|
||||
}
|
||||
|
||||
if (table_expression_modifiers)
|
||||
node->setTableExpressionModifiers(*table_expression_modifiers);
|
||||
node->setAlias(table_function_expression.tryGetAlias());
|
||||
node->setOriginalAST(table_expression.table_function);
|
||||
|
||||
return node;
|
||||
}
|
||||
|
||||
|
||||
ColumnTransformersNodes QueryTreeBuilder::buildColumnTransformers(const ASTPtr & matcher_expression, const ContextPtr & context) const
|
||||
{
|
||||
ColumnTransformersNodes column_transformers;
|
||||
@ -1056,4 +1080,9 @@ QueryTreeNodePtr buildQueryTree(ASTPtr query, ContextPtr context)
|
||||
return builder.getQueryTreeNode();
|
||||
}
|
||||
|
||||
QueryTreeNodePtr buildQueryTreeForTableFunction(const ASTTableExpression & table_expression, ContextPtr context)
|
||||
{
|
||||
return QueryTreeBuilder::buildForTableFunction(table_expression, context);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -9,6 +9,8 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct ASTTableExpression;
|
||||
|
||||
/** Build query tree from AST.
|
||||
* AST that represent query ASTSelectWithUnionQuery, ASTSelectIntersectExceptQuery, ASTSelectQuery.
|
||||
* AST that represent a list of expressions ASTExpressionList.
|
||||
@ -18,4 +20,7 @@ namespace DB
|
||||
*/
|
||||
QueryTreeNodePtr buildQueryTree(ASTPtr query, ContextPtr context);
|
||||
|
||||
// Build query tree from AST of table function.
|
||||
QueryTreeNodePtr buildQueryTreeForTableFunction(const ASTTableExpression & table_expression, ContextPtr context);
|
||||
|
||||
}
|
||||
|
@ -64,6 +64,8 @@
|
||||
#include <Analyzer/Resolve/TableExpressionsAliasVisitor.h>
|
||||
#include <Analyzer/Resolve/ReplaceColumnsVisitor.h>
|
||||
|
||||
#include <Analyzer/createUniqueTableAliases.h>
|
||||
|
||||
#include <Planner/PlannerActionsVisitor.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
@ -515,7 +517,9 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden
|
||||
|
||||
auto options = SelectQueryOptions(QueryProcessingStage::Complete, scope.subquery_depth, true /*is_subquery*/);
|
||||
options.only_analyze = only_analyze;
|
||||
auto interpreter = std::make_unique<InterpreterSelectQueryAnalyzer>(node->toAST(), subquery_context, subquery_context->getViewSource(), options);
|
||||
auto subquery = node->clone();
|
||||
createUniqueTableAliases(subquery, {}, subquery_context);
|
||||
auto interpreter = std::make_unique<InterpreterSelectQueryAnalyzer>(subquery->toAST(), subquery_context, subquery_context->getViewSource(), options);
|
||||
|
||||
if (only_analyze)
|
||||
{
|
||||
@ -4566,9 +4570,9 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node,
|
||||
|
||||
if (parametrized_view_storage)
|
||||
{
|
||||
auto fake_table_node = std::make_shared<TableNode>(parametrized_view_storage, scope_context);
|
||||
fake_table_node->setAlias(table_function_node->getAlias());
|
||||
table_function_node = fake_table_node;
|
||||
std::vector<size_t> skip_analysis_arguments_indexes(table_function_node_typed.getArguments().getNodes().size());
|
||||
std::iota(skip_analysis_arguments_indexes.begin(), skip_analysis_arguments_indexes.end(), 0);
|
||||
table_function_node_typed.resolve({}, parametrized_view_storage, scope_context, std::move(skip_analysis_arguments_indexes));
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -73,7 +73,7 @@ public:
|
||||
/// Returns true, if table function is resolved, false otherwise
|
||||
bool isResolved() const
|
||||
{
|
||||
return storage != nullptr && table_function != nullptr;
|
||||
return storage != nullptr;
|
||||
}
|
||||
|
||||
/// Get table function, returns nullptr if table function node is not resolved
|
||||
|
@ -845,22 +845,18 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
|
||||
}
|
||||
else if (create.select)
|
||||
{
|
||||
ASTPtr query = create.select->clone();
|
||||
|
||||
if (create.isParameterizedView())
|
||||
{
|
||||
replaceQueryParametersWithDefaults(query);
|
||||
}
|
||||
return properties;
|
||||
|
||||
Block as_select_sample;
|
||||
|
||||
if (getContext()->getSettingsRef().allow_experimental_analyzer)
|
||||
{
|
||||
as_select_sample = InterpreterSelectQueryAnalyzer::getSampleBlock(query, getContext());
|
||||
as_select_sample = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext());
|
||||
}
|
||||
else
|
||||
{
|
||||
as_select_sample = InterpreterSelectWithUnionQuery::getSampleBlock(query, getContext());
|
||||
as_select_sample = InterpreterSelectWithUnionQuery::getSampleBlock(create.select->clone(), getContext());
|
||||
}
|
||||
|
||||
properties.columns = ColumnsDescription(as_select_sample.getNamesAndTypesList());
|
||||
|
@ -4,6 +4,10 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Analyzer/Passes/QueryAnalysisPass.h>
|
||||
#include <Analyzer/QueryTreeBuilder.h>
|
||||
#include <Analyzer/TableFunctionNode.h>
|
||||
#include <Analyzer/TableNode.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
@ -146,6 +150,35 @@ void InterpreterDescribeQuery::fillColumnsFromSubquery(const ASTTableExpression
|
||||
void InterpreterDescribeQuery::fillColumnsFromTableFunction(const ASTTableExpression & table_expression)
|
||||
{
|
||||
auto current_context = getContext();
|
||||
if (current_context->getSettingsRef().allow_experimental_analyzer)
|
||||
{
|
||||
auto query_tree = buildQueryTreeForTableFunction(table_expression, current_context);
|
||||
|
||||
QueryAnalysisPass query_analysis_pass(true);
|
||||
query_analysis_pass.run(query_tree, current_context);
|
||||
|
||||
StoragePtr storage;
|
||||
if (auto * table_function_node = query_tree->as<TableFunctionNode>())
|
||||
storage = table_function_node->getStorage();
|
||||
else
|
||||
storage = query_tree->as<TableNode &>().getStorage();
|
||||
|
||||
auto column_descriptions = storage->getInMemoryMetadata().getColumns();
|
||||
for (const auto & column : column_descriptions)
|
||||
columns.emplace_back(column);
|
||||
|
||||
if (settings.describe_include_virtual_columns)
|
||||
{
|
||||
auto virtuals = storage->getVirtualsPtr();
|
||||
for (const auto & column : *virtuals)
|
||||
{
|
||||
if (!column_descriptions.has(column.name))
|
||||
virtual_columns.push_back(column);
|
||||
}
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression.table_function, current_context);
|
||||
|
||||
auto column_descriptions = table_function_ptr->getActualTableStructure(getContext(), /*is_insert_query*/ true);
|
||||
|
@ -156,33 +156,4 @@ void ReplaceQueryParameterVisitor::visitIdentifier(ASTPtr & ast)
|
||||
ast_identifier->children.clear();
|
||||
}
|
||||
|
||||
void replaceQueryParametersWithDefaults(ASTPtr & ast)
|
||||
{
|
||||
std::vector<ASTPtr> nodes_to_process{ ast };
|
||||
|
||||
while (!nodes_to_process.empty())
|
||||
{
|
||||
auto node = nodes_to_process.back();
|
||||
nodes_to_process.pop_back();
|
||||
for (auto & child : node->children)
|
||||
{
|
||||
if (auto * query_param = child->as<ASTQueryParameter>())
|
||||
{
|
||||
const auto data_type = DataTypeFactory::instance().get(query_param->type);
|
||||
auto * old_ptr = child.get();
|
||||
|
||||
Field literal = data_type->getDefault();
|
||||
if (typeid_cast<const DataTypeString *>(data_type.get()))
|
||||
child = std::make_shared<ASTLiteral>(literal);
|
||||
else
|
||||
child = addTypeConversionToAST(std::make_shared<ASTLiteral>(literal), query_param->type);
|
||||
|
||||
node->updatePointerToChild(old_ptr, child.get());
|
||||
}
|
||||
else
|
||||
nodes_to_process.push_back(child);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -32,6 +32,4 @@ private:
|
||||
void visitChildren(ASTPtr & ast);
|
||||
};
|
||||
|
||||
void replaceQueryParametersWithDefaults(ASTPtr & ast);
|
||||
|
||||
}
|
||||
|
@ -116,7 +116,14 @@ StorageView::StorageView(
|
||||
: IStorage(table_id_)
|
||||
{
|
||||
StorageInMemoryMetadata storage_metadata;
|
||||
storage_metadata.setColumns(columns_);
|
||||
if (!is_parameterized_view_)
|
||||
{
|
||||
/// If CREATE query is to create parameterized view, then we dont want to set columns
|
||||
if (!query.isParameterizedView())
|
||||
storage_metadata.setColumns(columns_);
|
||||
}
|
||||
else
|
||||
storage_metadata.setColumns(columns_);
|
||||
|
||||
storage_metadata.setComment(comment);
|
||||
if (query.sql_security)
|
||||
|
@ -0,0 +1,2 @@
|
||||
number UInt64
|
||||
55
|
@ -0,0 +1,5 @@
|
||||
create view paramview as select * from system.numbers where number <= {top:UInt64};
|
||||
|
||||
describe paramview(top = 10);
|
||||
|
||||
select arrayReduce('sum', (select groupArray(number) from paramview(top=10)));
|
Loading…
Reference in New Issue
Block a user