Compare commits

...

11 Commits

Author SHA1 Message Date
Dmitry Novik
0ca6ce7daa
Merge 586182a045 into 7fd2207626 2024-09-18 16:37:15 +02:00
Dmitry Novik
586182a045
Fix clang-tidy build 2024-09-05 16:06:24 +02:00
Dmitry Novik
31026aa74a Add a test 2024-09-03 15:03:57 +02:00
Dmitry Novik
5c39591ad1 Add tests 2024-09-03 15:03:57 +02:00
Dmitry Novik
769589dbc9 Remove redundant code 2024-09-03 15:03:57 +02:00
Dmitry Novik
355d7bce05 Improve param view docs 2024-09-03 15:03:57 +02:00
Dmitry Novik
34ceebe992 Rollback changes in QueryTreeBuilder 2024-09-03 15:03:57 +02:00
Dmitry Novik
887b49649c Small refactoring 2024-09-03 15:03:57 +02:00
Dmitry Novik
f4c4b58e64 Rewrite DESCRIBE for parameterized view implementation 2024-09-03 15:03:57 +02:00
Dmitry Novik
79140a3589 Support DESCRIBE for parameterized view; Fix usage in scalars 2024-09-03 15:03:57 +02:00
Dmitry Novik
967ba9d3d4 Parameterized views: Analyze SELECT query 2024-09-03 15:03:57 +02:00
9 changed files with 127 additions and 23 deletions

View File

@ -41,15 +41,24 @@ SELECT a, b, c FROM (SELECT ...)
## Parameterized View
Parametrized views are similar to normal views, but can be created with parameters which are not resolved immediately. These views can be used with table functions, which specify the name of the view as function name and the parameter values as its arguments.
Parametrized views are similar to normal views, but can be created with parameters which are not resolved immediately.
These views can be used with table functions, which specify the name of the view as function name and the parameter values as its arguments.
``` sql
CREATE VIEW view AS SELECT * FROM TABLE WHERE Column1={column1:datatype1} and Column2={column2:datatype2} ...
CREATE VIEW param_view AS SELECT * FROM TABLE WHERE Column1={column1:datatype1} and Column2={column2:datatype2} ...
```
The above creates a view for table which can be used as table function by substituting parameters as shown below.
``` sql
SELECT * FROM view(column1=value1, column2=value2 ...)
SELECT * FROM param_view(column1=value1, column2=value2 ...)
```
Since the parameterized view depends on the parameter values, it doesn't have a schema when parameters are not provided.
That means there's no information about parameterized views in the `system.columns` table.
Also, `DESCRIBE` queries would work only if parameters are provided.
```sql
DESCRIBE param_view(column1=value1, column2=value2 ...)
```
## Materialized View

View File

@ -4511,20 +4511,7 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node,
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().tryGet(table_function_name, scope_context);
if (!table_function_ptr)
{
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)
{
table_name = table_identifier[0];
}
else if (table_identifier.getPartsSize() == 2)
{
database_name = table_identifier[0];
table_name = table_identifier[1];
}
auto [database_name, table_name] = extractDatabaseAndTableNameForParametrizedView(table_function_name, scope_context);
/// Collect parametrized view arguments
NameToNameMap view_params;
@ -4566,9 +4553,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;
}

View File

@ -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

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,8 +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>
@ -24,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_)
@ -125,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;
@ -146,7 +165,39 @@ void InterpreterDescribeQuery::fillColumnsFromSubquery(const ASTTableExpression
void InterpreterDescribeQuery::fillColumnsFromTableFunction(const ASTTableExpression & table_expression)
{
auto current_context = getContext();
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression.table_function, current_context);
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 [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());
if (auto * storage_view = table->as<StorageView>())
{
if (storage_view->isParameterizedView())
{
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);
fillColumnsFromSubqueryImpl(query, current_context);
return;
}
}
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)
@ -172,6 +223,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

@ -0,0 +1,8 @@
number UInt64
number UInt64
number UInt64
\'Biba\' String
CAST(dummy, \'Int\') Int32
CAST(dummy, \'String\') String
0
55

View File

@ -0,0 +1,19 @@
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);
create view p2 as select number, {name:String} from system.numbers where number <= {top:UInt64};
describe p2(top = 10); -- { serverError UNKNOWN_QUERY_PARAMETER }
describe p2(name = 'Biba', top = 2);
create view p3 as select CAST(dummy, {t:String});
describe p3(t = 'Int');
describe p3(t = 'String');
describe (SELECT * FROM p3(t = 'Int64') union all SELECT * FROM p3(t = 'UInt64')); -- { serverError NO_COMMON_TYPE }
SELECT * FROM p3(t = 'String');
select arrayReduce('sum', (select groupArray(number) from paramview(top=10)));