Rewrite DESCRIBE for parameterized view implementation

This commit is contained in:
Dmitry Novik 2024-09-02 14:55:23 +02:00
parent 79140a3589
commit f4c4b58e64
7 changed files with 76 additions and 25 deletions

View File

@ -4518,7 +4518,6 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node,
String database_name = scope_context->getCurrentDatabase();
String table_name;
auto function_ast = table_function_node->toAST();
Identifier table_identifier{table_function_name};
if (table_identifier.getPartsSize() == 1)
{

View File

@ -938,4 +938,23 @@ QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(const QueryTreeNo
return buildSubqueryToReadColumnsFromTableExpression(columns_to_select, table_node, context);
}
std::pair<String, String> extractDatabaseAndTableNameForParametrizedView(const String & table_function_name, const ContextPtr & context)
{
String database_name = context->getCurrentDatabase();
String table_name;
Identifier table_identifier{table_function_name};
if (table_identifier.getPartsSize() == 1)
{
table_name = table_identifier[0];
}
else if (table_identifier.getPartsSize() == 2)
{
database_name = table_identifier[0];
table_name = table_identifier[1];
}
return { database_name, table_name };
}
}

View File

@ -159,5 +159,6 @@ QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(const NamesAndTyp
*/
QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(const QueryTreeNodePtr & table_node, const ContextPtr & context);
std::pair<String, String> extractDatabaseAndTableNameForParametrizedView(const String & table_function_name, const ContextPtr & context);
}

View File

@ -3,12 +3,15 @@
#include <QueryPipeline/BlockIO.h>
#include <DataTypes/DataTypeString.h>
#include <Parsers/queryToString.h>
#include <Parsers/FunctionParameterValuesVisitor.h>
#include <Common/typeid_cast.h>
#include <Analyzer/Utils.h>
#include <Analyzer/Passes/QueryAnalysisPass.h>
#include <Analyzer/QueryTreeBuilder.h>
#include <Analyzer/TableFunctionNode.h>
#include <Analyzer/TableNode.h>
#include <Core/Settings.h>
#include <Storages/StorageView.h>
#include <TableFunctions/ITableFunction.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
@ -28,6 +31,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNSUPPORTED_METHOD;
extern const int UNKNOWN_FUNCTION;
}
InterpreterDescribeQuery::InterpreterDescribeQuery(const ASTPtr & query_ptr_, ContextPtr context_)
: WithContext(context_)
, query_ptr(query_ptr_)
@ -129,10 +140,14 @@ BlockIO InterpreterDescribeQuery::execute()
void InterpreterDescribeQuery::fillColumnsFromSubquery(const ASTTableExpression & table_expression)
{
Block sample_block;
auto select_query = table_expression.subquery->children.at(0);
auto current_context = getContext();
fillColumnsFromSubqueryImpl(select_query, current_context);
}
void InterpreterDescribeQuery::fillColumnsFromSubqueryImpl(const ASTPtr & select_query, const ContextPtr & current_context)
{
Block sample_block;
if (settings.allow_experimental_analyzer)
{
SelectQueryOptions select_query_options;
@ -150,36 +165,38 @@ void InterpreterDescribeQuery::fillColumnsFromSubquery(const ASTTableExpression
void InterpreterDescribeQuery::fillColumnsFromTableFunction(const ASTTableExpression & table_expression)
{
auto current_context = getContext();
if (current_context->getSettingsRef().allow_experimental_analyzer)
auto table_function_name = table_expression.table_function->as<ASTFunction>()->name;
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().tryGet(table_function_name, current_context);
if (!table_function_ptr)
{
auto query_tree = buildQueryTreeForTableFunction(table_expression, current_context);
auto [database_name, table_name] = extractDatabaseAndTableNameForParametrizedView(table_function_name, current_context);
auto table_id = getContext()->resolveStorageID({database_name, table_name});
getContext()->checkAccess(AccessType::SHOW_COLUMNS, table_id);
auto table = DatabaseCatalog::instance().getTable(table_id, getContext());
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)
if (auto * storage_view = table->as<StorageView>())
{
auto virtuals = storage->getVirtualsPtr();
for (const auto & column : *virtuals)
if (storage_view->isParameterizedView())
{
if (!column_descriptions.has(column.name))
virtual_columns.push_back(column);
auto query = storage_view->getInMemoryMetadataPtr()->getSelectQuery().inner_query->clone();
NameToNameMap parameterized_view_values = analyzeFunctionParamValues(table_expression.table_function, current_context);
StorageView::replaceQueryParametersIfParametrizedView(query, parameterized_view_values);
return fillColumnsFromSubqueryImpl(query, current_context);
}
}
return;
}
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression.table_function, current_context);
auto hints = TableFunctionFactory::instance().getHints(table_function_name);
if (!hints.empty())
throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}. Maybe you meant: {}", table_function_name, toString(hints));
else
throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}", table_function_name);
}
else
{
table_function_ptr->parseArguments(table_expression.table_function, current_context);
}
auto column_descriptions = table_function_ptr->getActualTableStructure(getContext(), /*is_insert_query*/ true);
for (const auto & column : column_descriptions)
@ -205,6 +222,14 @@ void InterpreterDescribeQuery::fillColumnsFromTable(const ASTTableExpression & t
auto table_id = getContext()->resolveStorageID(table_expression.database_and_table_name);
getContext()->checkAccess(AccessType::SHOW_COLUMNS, table_id);
auto table = DatabaseCatalog::instance().getTable(table_id, getContext());
if (auto * storage_view = table->as<StorageView>())
{
if (storage_view->isParameterizedView())
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"Cannot infer table schema for the parametrized view when no query parameters are provided");
}
auto table_lock = table->lockForShare(getContext()->getInitialQueryId(), settings.lock_acquire_timeout);
auto metadata_snapshot = table->getInMemoryMetadataPtr();

View File

@ -1,5 +1,6 @@
#pragma once
#include <Interpreters/Context_fwd.h>
#include <Interpreters/IInterpreter.h>
#include <Storages/ColumnsDescription.h>
#include <Storages/StorageSnapshot.h>
@ -24,6 +25,7 @@ public:
private:
void fillColumnsFromSubquery(const ASTTableExpression & table_expression);
void fillColumnsFromSubqueryImpl(const ASTPtr & select_query, const ContextPtr & current_context);
void fillColumnsFromTableFunction(const ASTTableExpression & table_expression);
void fillColumnsFromTable(const ASTTableExpression & table_expression);

View File

@ -1,2 +1,3 @@
number UInt64
number UInt64
55

View File

@ -1,5 +1,9 @@
create view paramview as select * from system.numbers where number <= {top:UInt64};
describe paramview; -- { serverError UNSUPPORTED_METHOD }
describe paramview(top = 10);
describe paramview(top = 2 + 2);
select arrayReduce('sum', (select groupArray(number) from paramview(top=10)));