mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-02 20:42:04 +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;
|
return query_tree_node;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
static QueryTreeNodePtr buildForTableFunction(
|
||||||
|
const ASTTableExpression & table_expression,
|
||||||
|
const ContextPtr & context)
|
||||||
|
{
|
||||||
|
QueryTreeBuilder builder;
|
||||||
|
return builder.buildTableFunction(table_expression, context);
|
||||||
|
}
|
||||||
|
|
||||||
private:
|
private:
|
||||||
|
QueryTreeBuilder() = default;
|
||||||
|
|
||||||
QueryTreeNodePtr buildSelectOrUnionExpression(const ASTPtr & select_or_union_query,
|
QueryTreeNodePtr buildSelectOrUnionExpression(const ASTPtr & select_or_union_query,
|
||||||
bool is_subquery,
|
bool is_subquery,
|
||||||
const std::string & cte_name,
|
const std::string & cte_name,
|
||||||
@ -109,6 +119,11 @@ private:
|
|||||||
|
|
||||||
QueryTreeNodePtr buildJoinTree(const ASTPtr & tables_in_select_query, const ContextPtr & context) const;
|
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;
|
ColumnTransformersNodes buildColumnTransformers(const ASTPtr & matcher_expression, const ContextPtr & context) const;
|
||||||
|
|
||||||
ASTPtr query;
|
ASTPtr query;
|
||||||
@ -854,34 +869,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select
|
|||||||
}
|
}
|
||||||
else if (table_expression.table_function)
|
else if (table_expression.table_function)
|
||||||
{
|
{
|
||||||
auto & table_function_expression = table_expression.table_function->as<ASTFunction &>();
|
table_expressions.push_back(buildTableFunction(table_expression, context, table_expression_modifiers));
|
||||||
|
|
||||||
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));
|
|
||||||
}
|
}
|
||||||
else
|
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 QueryTreeBuilder::buildColumnTransformers(const ASTPtr & matcher_expression, const ContextPtr & context) const
|
||||||
{
|
{
|
||||||
ColumnTransformersNodes column_transformers;
|
ColumnTransformersNodes column_transformers;
|
||||||
@ -1056,4 +1080,9 @@ QueryTreeNodePtr buildQueryTree(ASTPtr query, ContextPtr context)
|
|||||||
return builder.getQueryTreeNode();
|
return builder.getQueryTreeNode();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
QueryTreeNodePtr buildQueryTreeForTableFunction(const ASTTableExpression & table_expression, ContextPtr context)
|
||||||
|
{
|
||||||
|
return QueryTreeBuilder::buildForTableFunction(table_expression, context);
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -9,6 +9,8 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
struct ASTTableExpression;
|
||||||
|
|
||||||
/** Build query tree from AST.
|
/** Build query tree from AST.
|
||||||
* AST that represent query ASTSelectWithUnionQuery, ASTSelectIntersectExceptQuery, ASTSelectQuery.
|
* AST that represent query ASTSelectWithUnionQuery, ASTSelectIntersectExceptQuery, ASTSelectQuery.
|
||||||
* AST that represent a list of expressions ASTExpressionList.
|
* AST that represent a list of expressions ASTExpressionList.
|
||||||
@ -18,4 +20,7 @@ namespace DB
|
|||||||
*/
|
*/
|
||||||
QueryTreeNodePtr buildQueryTree(ASTPtr query, ContextPtr context);
|
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/TableExpressionsAliasVisitor.h>
|
||||||
#include <Analyzer/Resolve/ReplaceColumnsVisitor.h>
|
#include <Analyzer/Resolve/ReplaceColumnsVisitor.h>
|
||||||
|
|
||||||
|
#include <Analyzer/createUniqueTableAliases.h>
|
||||||
|
|
||||||
#include <Planner/PlannerActionsVisitor.h>
|
#include <Planner/PlannerActionsVisitor.h>
|
||||||
|
|
||||||
#include <Core/Settings.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*/);
|
auto options = SelectQueryOptions(QueryProcessingStage::Complete, scope.subquery_depth, true /*is_subquery*/);
|
||||||
options.only_analyze = only_analyze;
|
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)
|
if (only_analyze)
|
||||||
{
|
{
|
||||||
@ -4566,9 +4570,9 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node,
|
|||||||
|
|
||||||
if (parametrized_view_storage)
|
if (parametrized_view_storage)
|
||||||
{
|
{
|
||||||
auto fake_table_node = std::make_shared<TableNode>(parametrized_view_storage, scope_context);
|
std::vector<size_t> skip_analysis_arguments_indexes(table_function_node_typed.getArguments().getNodes().size());
|
||||||
fake_table_node->setAlias(table_function_node->getAlias());
|
std::iota(skip_analysis_arguments_indexes.begin(), skip_analysis_arguments_indexes.end(), 0);
|
||||||
table_function_node = fake_table_node;
|
table_function_node_typed.resolve({}, parametrized_view_storage, scope_context, std::move(skip_analysis_arguments_indexes));
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -73,7 +73,7 @@ public:
|
|||||||
/// Returns true, if table function is resolved, false otherwise
|
/// Returns true, if table function is resolved, false otherwise
|
||||||
bool isResolved() const
|
bool isResolved() const
|
||||||
{
|
{
|
||||||
return storage != nullptr && table_function != nullptr;
|
return storage != nullptr;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Get table function, returns nullptr if table function node is not resolved
|
/// Get table function, returns nullptr if table function node is not resolved
|
||||||
|
@ -845,22 +845,18 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
|
|||||||
}
|
}
|
||||||
else if (create.select)
|
else if (create.select)
|
||||||
{
|
{
|
||||||
ASTPtr query = create.select->clone();
|
|
||||||
|
|
||||||
if (create.isParameterizedView())
|
if (create.isParameterizedView())
|
||||||
{
|
return properties;
|
||||||
replaceQueryParametersWithDefaults(query);
|
|
||||||
}
|
|
||||||
|
|
||||||
Block as_select_sample;
|
Block as_select_sample;
|
||||||
|
|
||||||
if (getContext()->getSettingsRef().allow_experimental_analyzer)
|
if (getContext()->getSettingsRef().allow_experimental_analyzer)
|
||||||
{
|
{
|
||||||
as_select_sample = InterpreterSelectQueryAnalyzer::getSampleBlock(query, getContext());
|
as_select_sample = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext());
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
as_select_sample = InterpreterSelectWithUnionQuery::getSampleBlock(query, getContext());
|
as_select_sample = InterpreterSelectWithUnionQuery::getSampleBlock(create.select->clone(), getContext());
|
||||||
}
|
}
|
||||||
|
|
||||||
properties.columns = ColumnsDescription(as_select_sample.getNamesAndTypesList());
|
properties.columns = ColumnsDescription(as_select_sample.getNamesAndTypesList());
|
||||||
|
@ -4,6 +4,10 @@
|
|||||||
#include <DataTypes/DataTypeString.h>
|
#include <DataTypes/DataTypeString.h>
|
||||||
#include <Parsers/queryToString.h>
|
#include <Parsers/queryToString.h>
|
||||||
#include <Common/typeid_cast.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 <Core/Settings.h>
|
||||||
#include <TableFunctions/ITableFunction.h>
|
#include <TableFunctions/ITableFunction.h>
|
||||||
#include <TableFunctions/TableFunctionFactory.h>
|
#include <TableFunctions/TableFunctionFactory.h>
|
||||||
@ -146,6 +150,35 @@ void InterpreterDescribeQuery::fillColumnsFromSubquery(const ASTTableExpression
|
|||||||
void InterpreterDescribeQuery::fillColumnsFromTableFunction(const ASTTableExpression & table_expression)
|
void InterpreterDescribeQuery::fillColumnsFromTableFunction(const ASTTableExpression & table_expression)
|
||||||
{
|
{
|
||||||
auto current_context = getContext();
|
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);
|
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);
|
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();
|
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 visitChildren(ASTPtr & ast);
|
||||||
};
|
};
|
||||||
|
|
||||||
void replaceQueryParametersWithDefaults(ASTPtr & ast);
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -116,7 +116,14 @@ StorageView::StorageView(
|
|||||||
: IStorage(table_id_)
|
: IStorage(table_id_)
|
||||||
{
|
{
|
||||||
StorageInMemoryMetadata storage_metadata;
|
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);
|
storage_metadata.setComment(comment);
|
||||||
if (query.sql_security)
|
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