mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-12 01:12:12 +00:00
40907 Parameterized views as table functions
Implementation: * Updated parsers by adding a bool allow_query_parameters while creating ordinary view, which is used in interpreters to allow query parameters in SELECT. * Added a check in ActionsVisitor if multiple parameters have same names while creating parameterised view. * Added bool in StorageView to represent parameterized view. * Updated processing of SELECT with parameter values to check for views and added substitution of values in the query parameters. Testing: * Added a test tests/queries/0_stateless/02428_parameterized_view.sql Documentation: * Updated the english documentation for VIEW.
This commit is contained in:
parent
a0b064fc99
commit
bb451b2b27
@ -36,6 +36,18 @@ This query is fully equivalent to using the subquery:
|
||||
SELECT a, b, c FROM (SELECT ...)
|
||||
```
|
||||
|
||||
## Parameterized View
|
||||
This is similar to normal view but can be created with parameter instead of literals and can be used as table functions by substituting the values of the parametes.
|
||||
|
||||
``` sql
|
||||
CREATE VIEW 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 value1 & value2 as show below.
|
||||
|
||||
``` sql
|
||||
SELECT * FROM view(column1=value1, column2=value2 ...)
|
||||
```
|
||||
|
||||
## Materialized View
|
||||
|
||||
``` sql
|
||||
@ -74,9 +86,7 @@ To delete a view, use [DROP VIEW](../../../sql-reference/statements/drop.md#drop
|
||||
This is an experimental feature that may change in backwards-incompatible ways in the future releases. Enable usage of live views and `WATCH` query using [allow_experimental_live_view](../../../operations/settings/settings.md#allow-experimental-live-view) setting. Input the command `set allow_experimental_live_view = 1`.
|
||||
:::
|
||||
|
||||
```sql
|
||||
CREATE LIVE VIEW [IF NOT EXISTS] [db.]table_name [WITH [TIMEOUT [value_in_sec] [AND]] [REFRESH [value_in_sec]]] AS SELECT ...
|
||||
```
|
||||
|
||||
|
||||
Live views store result of the corresponding [SELECT](../../../sql-reference/statements/select/index.md) query and are updated any time the result of the query changes. Query result as well as partial result needed to combine with new data are stored in memory providing increased performance for repeated queries. Live views can provide push notifications when query result changes using the [WATCH](../../../sql-reference/statements/watch.md) query.
|
||||
|
||||
|
@ -38,6 +38,7 @@
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTQueryParameter.h>
|
||||
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
|
||||
@ -742,9 +743,29 @@ std::optional<NameAndTypePair> ActionsMatcher::getNameAndTypeFromAST(const ASTPt
|
||||
return NameAndTypePair(child_column_name, node->result_type);
|
||||
|
||||
if (!data.only_consts)
|
||||
throw Exception("Unknown identifier: " + child_column_name + "; there are columns: " + data.actions_stack.dumpNames(),
|
||||
ErrorCodes::UNKNOWN_IDENTIFIER);
|
||||
{
|
||||
bool has_query_parameter = false;
|
||||
|
||||
std::queue<ASTPtr> astQueue;
|
||||
astQueue.push(ast);
|
||||
|
||||
while (!astQueue.empty())
|
||||
{
|
||||
auto current = astQueue.front();
|
||||
astQueue.pop();
|
||||
|
||||
if (auto * ast_query_parameter = current->as<ASTQueryParameter>())
|
||||
has_query_parameter = true;
|
||||
|
||||
for (auto astChild : current->children)
|
||||
astQueue.push(astChild);
|
||||
}
|
||||
|
||||
if (!has_query_parameter)
|
||||
throw Exception(
|
||||
"Unknown identifier: " + child_column_name + "; there are columns: " + data.actions_stack.dumpNames(),
|
||||
ErrorCodes::UNKNOWN_IDENTIFIER);
|
||||
}
|
||||
return {};
|
||||
}
|
||||
|
||||
|
@ -94,6 +94,8 @@
|
||||
#include <Interpreters/TransactionLog.h>
|
||||
#include <filesystem>
|
||||
#include <re2/re2.h>
|
||||
#include <Storages/StorageView.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
||||
#if USE_ROCKSDB
|
||||
#include <rocksdb/table.h>
|
||||
@ -137,6 +139,7 @@ namespace ErrorCodes
|
||||
extern const int INVALID_SETTING_VALUE;
|
||||
extern const int UNKNOWN_READ_METHOD;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int UNKNOWN_FUNCTION;
|
||||
}
|
||||
|
||||
|
||||
@ -1130,6 +1133,8 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression)
|
||||
StoragePtr & res = table_function_results[key];
|
||||
|
||||
if (!res)
|
||||
{
|
||||
try
|
||||
{
|
||||
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression, shared_from_this());
|
||||
if (getSettingsRef().use_structure_from_insertion_table_in_table_functions && table_function_ptr->needStructureHint())
|
||||
@ -1157,6 +1162,44 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression)
|
||||
}
|
||||
|
||||
return res;
|
||||
}catch (DB::Exception &table_function_exception)
|
||||
{
|
||||
if (table_function_exception.code() == ErrorCodes::UNKNOWN_FUNCTION)
|
||||
{
|
||||
if (auto ast_function = table_expression->as<ASTFunction>())
|
||||
{
|
||||
try
|
||||
{
|
||||
res = DatabaseCatalog::instance().getTable({getCurrentDatabase(), ast_function->name}, getQueryContext());
|
||||
if (res.get()->isView() && res->as<StorageView>()->isParameterizedView())
|
||||
return res;
|
||||
else
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"Not a parameterized view {}",
|
||||
ast_function->name);
|
||||
}
|
||||
}
|
||||
catch (DB::Exception &view_exception)
|
||||
{
|
||||
if (view_exception.code() == ErrorCodes::UNKNOWN_TABLE)
|
||||
throw Exception(
|
||||
ErrorCodes::UNKNOWN_FUNCTION,
|
||||
"Unknown table function {} OR Unknown parameterized view {}",
|
||||
table_function_exception.message(),
|
||||
view_exception.message());
|
||||
else
|
||||
throw;
|
||||
}
|
||||
}
|
||||
else
|
||||
throw;
|
||||
}
|
||||
else
|
||||
throw;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
return res;
|
||||
|
@ -1286,6 +1286,9 @@ bool SelectQueryExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain,
|
||||
|
||||
getRootActions(select_query->where(), only_types, step.actions());
|
||||
|
||||
if (select_query->allow_query_parameters && select_query->hasQueryParameters())
|
||||
return true;
|
||||
|
||||
auto where_column_name = select_query->where()->getColumnName();
|
||||
step.addRequiredOutput(where_column_name);
|
||||
|
||||
@ -1902,12 +1905,17 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
|
||||
ExpressionActions(
|
||||
before_where,
|
||||
ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_where_sample);
|
||||
auto & column_elem = before_where_sample.getByName(query.where()->getColumnName());
|
||||
|
||||
if (!(query.allow_query_parameters && query.hasQueryParameters()))
|
||||
{
|
||||
auto & column_elem
|
||||
= before_where_sample.getByName(query.where()->getColumnName());
|
||||
/// If the filter column is a constant, record it.
|
||||
if (column_elem.column)
|
||||
where_constant_filter_description = ConstantFilterDescription(*column_elem.column);
|
||||
}
|
||||
}
|
||||
}
|
||||
chain.addStep();
|
||||
}
|
||||
|
||||
@ -2066,6 +2074,9 @@ void ExpressionAnalysisResult::finalize(
|
||||
ssize_t & having_step_num,
|
||||
const ASTSelectQuery & query)
|
||||
{
|
||||
if (query.allow_query_parameters && query.hasQueryParameters())
|
||||
return;
|
||||
|
||||
if (prewhere_step_num >= 0)
|
||||
{
|
||||
const ExpressionActionsChain::Step & step = *chain.steps.at(prewhere_step_num);
|
||||
|
@ -500,7 +500,10 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
/// Allow push down and other optimizations for VIEW: replace with subquery and rewrite it.
|
||||
ASTPtr view_table;
|
||||
if (view)
|
||||
{
|
||||
view->replaceWithSubquery(getSelectQuery(), view_table, metadata_snapshot);
|
||||
view->replaceQueryParameters(query_ptr, getSelectQuery().getQueryParameterValues());
|
||||
}
|
||||
|
||||
syntax_analyzer_result = TreeRewriter(context).analyzeSelect(
|
||||
query_ptr,
|
||||
|
@ -122,6 +122,16 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data)
|
||||
}
|
||||
}
|
||||
|
||||
void QueryNormalizer::visit(ASTQueryParameter & node, const ASTPtr & ast, Data & data)
|
||||
{
|
||||
auto it_alias = data.aliases.find(node.name);
|
||||
if (it_alias != data.aliases.end())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Same alias used multiple times {} ", backQuote(node.name));
|
||||
|
||||
data.aliases[node.name] =ast;
|
||||
}
|
||||
|
||||
|
||||
void QueryNormalizer::visit(ASTTablesInSelectQueryElement & node, const ASTPtr &, Data & data)
|
||||
{
|
||||
/// normalize JOIN ON section
|
||||
@ -142,6 +152,8 @@ static bool needVisitChild(const ASTPtr & child)
|
||||
/// special visitChildren() for ASTSelectQuery
|
||||
void QueryNormalizer::visit(ASTSelectQuery & select, const ASTPtr &, Data & data)
|
||||
{
|
||||
data.allow_query_parameters = select.allow_query_parameters;
|
||||
|
||||
for (auto & child : select.children)
|
||||
{
|
||||
if (needVisitChild(child))
|
||||
@ -257,7 +269,12 @@ void QueryNormalizer::visit(ASTPtr & ast, Data & data)
|
||||
else if (auto * node_select = ast->as<ASTSelectQuery>())
|
||||
visit(*node_select, ast, data);
|
||||
else if (auto * node_param = ast->as<ASTQueryParameter>())
|
||||
{
|
||||
if (data.allow_query_parameters)
|
||||
visit(*node_param, ast, data);
|
||||
else
|
||||
throw Exception("Query parameter " + backQuote(node_param->name) + " was not set", ErrorCodes::UNKNOWN_QUERY_PARAMETER);
|
||||
}
|
||||
else if (auto * node_function = ast->as<ASTFunction>())
|
||||
if (node_function->parameters)
|
||||
visit(node_function->parameters, data);
|
||||
|
@ -13,6 +13,7 @@ class ASTSelectQuery;
|
||||
class ASTIdentifier;
|
||||
struct ASTTablesInSelectQueryElement;
|
||||
class Context;
|
||||
class ASTQueryParameter;
|
||||
|
||||
|
||||
class QueryNormalizer
|
||||
@ -52,6 +53,7 @@ public:
|
||||
|
||||
/// It's Ok to have "c + 1 AS c" in queries, but not in table definition
|
||||
const bool allow_self_aliases; /// for constructs like "SELECT column + 1 AS column"
|
||||
bool allow_query_parameters;
|
||||
|
||||
Data(Aliases & aliases_, const NameSet & source_columns_set_, bool ignore_alias_, ExtractedSettings && settings_, bool allow_self_aliases_)
|
||||
: aliases(aliases_)
|
||||
@ -80,6 +82,7 @@ private:
|
||||
static void visit(ASTIdentifier &, ASTPtr &, Data &);
|
||||
static void visit(ASTTablesInSelectQueryElement &, const ASTPtr &, Data &);
|
||||
static void visit(ASTSelectQuery &, const ASTPtr &, Data &);
|
||||
static void visit(ASTQueryParameter &, const ASTPtr &, Data &);
|
||||
|
||||
static void visitChildren(IAST * node, Data & data);
|
||||
};
|
||||
|
@ -449,4 +449,11 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
|
||||
}
|
||||
}
|
||||
|
||||
bool ASTCreateQuery::isParameterizedView() const
|
||||
{
|
||||
if (is_ordinary_view && select && select->hasQueryParameters())
|
||||
return true;
|
||||
return false;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -24,7 +24,7 @@ public:
|
||||
IAST * sample_by = nullptr;
|
||||
IAST * ttl_table = nullptr;
|
||||
ASTSetQuery * settings = nullptr;
|
||||
|
||||
bool allow_query_parameters = false;
|
||||
|
||||
String getID(char) const override { return "Storage definition"; }
|
||||
|
||||
@ -120,6 +120,8 @@ public:
|
||||
|
||||
bool isView() const { return is_ordinary_view || is_materialized_view || is_live_view || is_window_view; }
|
||||
|
||||
bool isParameterizedView() const;
|
||||
|
||||
QueryKind getQueryKind() const override { return QueryKind::Create; }
|
||||
|
||||
protected:
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Interpreters/StorageID.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -474,4 +475,54 @@ void ASTSelectQuery::setFinal() // NOLINT method can be made const
|
||||
tables_element.table_expression->as<ASTTableExpression &>().final = true;
|
||||
}
|
||||
|
||||
bool ASTSelectQuery::hasQueryParameters() const
|
||||
{
|
||||
std::queue<ASTPtr> queue;
|
||||
queue.push(this->clone());
|
||||
|
||||
while (!queue.empty())
|
||||
{
|
||||
auto ast = queue.front();
|
||||
queue.pop();
|
||||
|
||||
if (ast->as<ASTQueryParameter>())
|
||||
return true;
|
||||
|
||||
for (auto child : ast->children)
|
||||
queue.push(child);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
NameToNameMap ASTSelectQuery::getQueryParameterValues() const
|
||||
{
|
||||
NameToNameMap parameter_values;
|
||||
std::queue<ASTPtr> queue;
|
||||
queue.push(this->clone());
|
||||
|
||||
while (!queue.empty())
|
||||
{
|
||||
auto ast = queue.front();
|
||||
queue.pop();
|
||||
if (auto expression_list = ast->as<ASTExpressionList>())
|
||||
{
|
||||
if (expression_list->children.size() == 2)
|
||||
{
|
||||
if (auto identifier = expression_list->children[0]->as<ASTIdentifier>())
|
||||
{
|
||||
if (auto literal = expression_list->children[1]->as<ASTLiteral>())
|
||||
{
|
||||
|
||||
parameter_values[identifier->name()] = toString(literal->value);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
for (auto child : ast->children)
|
||||
queue.push(child);
|
||||
}
|
||||
|
||||
return parameter_values;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,7 +2,9 @@
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Core/Names.h>
|
||||
|
||||
#include <queue>
|
||||
#include <Parsers/ASTQueryParameter.h>
|
||||
#include <Core/Field.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -88,6 +90,7 @@ public:
|
||||
bool group_by_with_constant_keys = false;
|
||||
bool group_by_with_grouping_sets = false;
|
||||
bool limit_with_ties = false;
|
||||
bool allow_query_parameters = false;
|
||||
|
||||
ASTPtr & refSelect() { return getExpression(Expression::SELECT); }
|
||||
ASTPtr & refTables() { return getExpression(Expression::TABLES); }
|
||||
@ -142,6 +145,8 @@ public:
|
||||
void setFinal();
|
||||
|
||||
QueryKind getQueryKind() const override { return QueryKind::Select; }
|
||||
bool hasQueryParameters() const;
|
||||
NameToNameMap getQueryParameterValues() const;
|
||||
|
||||
protected:
|
||||
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||
|
@ -86,4 +86,26 @@ bool ASTSelectWithUnionQuery::hasNonDefaultUnionMode() const
|
||||
|| set_of_modes.contains(SelectUnionMode::EXCEPT_DISTINCT);
|
||||
}
|
||||
|
||||
bool ASTSelectWithUnionQuery::hasQueryParameters() const
|
||||
{
|
||||
std::queue<ASTPtr> queue;
|
||||
queue.push(this->clone());
|
||||
|
||||
while (!queue.empty())
|
||||
{
|
||||
auto current = queue.front();
|
||||
queue.pop();
|
||||
|
||||
if (auto * select = current->as<ASTSelectQuery>())
|
||||
{
|
||||
if (select->hasQueryParameters())
|
||||
return true;
|
||||
}
|
||||
|
||||
for (auto child : current->children)
|
||||
queue.push(child);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,6 +2,8 @@
|
||||
|
||||
#include <Parsers/ASTQueryWithOutput.h>
|
||||
#include <Parsers/SelectUnionMode.h>
|
||||
#include <queue>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -31,6 +33,10 @@ public:
|
||||
|
||||
/// Consider any mode other than ALL as non-default.
|
||||
bool hasNonDefaultUnionMode() const;
|
||||
|
||||
bool hasQueryParameters() const;
|
||||
|
||||
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -121,7 +121,7 @@ bool ParserList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
|
||||
bool ParserUnionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ParserUnionQueryElement elem_parser;
|
||||
ParserUnionQueryElement elem_parser(allow_query_parameters);
|
||||
ParserKeyword s_union_parser("UNION");
|
||||
ParserKeyword s_all_parser("ALL");
|
||||
ParserKeyword s_distinct_parser("DISTINCT");
|
||||
|
@ -84,6 +84,11 @@ private:
|
||||
class ParserUnionList : public IParserBase
|
||||
{
|
||||
public:
|
||||
ParserUnionList(bool allow_query_parameters_=false)
|
||||
: allow_query_parameters(allow_query_parameters_)
|
||||
{
|
||||
}
|
||||
|
||||
template <typename ElemFunc, typename SepFunc>
|
||||
static bool parseUtil(Pos & pos, const ElemFunc & parse_element, const SepFunc & parse_separator)
|
||||
{
|
||||
@ -108,6 +113,7 @@ public:
|
||||
}
|
||||
|
||||
auto getUnionModes() const { return union_modes; }
|
||||
bool allow_query_parameters;
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "list of union elements"; }
|
||||
|
@ -1285,7 +1285,10 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
is_materialized_view = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
is_ordinary_view = true;
|
||||
select_p.allow_query_parameters = true;
|
||||
}
|
||||
|
||||
if (!s_view.ignore(pos, expected))
|
||||
return false;
|
||||
|
@ -34,6 +34,7 @@ namespace ErrorCodes
|
||||
bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
auto select_query = std::make_shared<ASTSelectQuery>();
|
||||
select_query->allow_query_parameters = allow_query_parameters;
|
||||
node = select_query;
|
||||
|
||||
ParserKeyword s_select("SELECT");
|
||||
|
@ -9,6 +9,13 @@ namespace DB
|
||||
|
||||
class ParserSelectQuery : public IParserBase
|
||||
{
|
||||
public:
|
||||
ParserSelectQuery(bool allow_query_parameters_=false)
|
||||
: allow_query_parameters(allow_query_parameters_)
|
||||
{
|
||||
}
|
||||
bool allow_query_parameters;
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "SELECT query"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
|
@ -10,7 +10,7 @@ namespace DB
|
||||
bool ParserSelectWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ASTPtr list_node;
|
||||
ParserUnionList parser;
|
||||
ParserUnionList parser(allow_query_parameters);
|
||||
|
||||
if (!parser.parse(pos, list_node, expected))
|
||||
return false;
|
||||
|
@ -8,6 +8,9 @@ namespace DB
|
||||
|
||||
class ParserSelectWithUnionQuery : public IParserBase
|
||||
{
|
||||
public:
|
||||
bool allow_query_parameters = false;
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "SELECT query, possibly with UNION"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
|
@ -10,7 +10,7 @@ namespace DB
|
||||
|
||||
bool ParserUnionQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
if (!ParserSubquery().parse(pos, node, expected) && !ParserSelectQuery().parse(pos, node, expected))
|
||||
if (!ParserSubquery().parse(pos, node, expected) && !ParserSelectQuery(allow_query_parameters).parse(pos, node, expected))
|
||||
return false;
|
||||
|
||||
if (const auto * ast_subquery = node->as<ASTSubquery>())
|
||||
|
@ -9,6 +9,13 @@ namespace DB
|
||||
|
||||
class ParserUnionQueryElement : public IParserBase
|
||||
{
|
||||
public:
|
||||
ParserUnionQueryElement(bool allow_query_parameters_=false)
|
||||
: allow_query_parameters(allow_query_parameters_)
|
||||
{
|
||||
}
|
||||
bool allow_query_parameters;
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "SELECT query, subquery, possibly with UNION"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
|
@ -23,6 +23,8 @@
|
||||
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
|
||||
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
|
||||
|
||||
#include <Interpreters/ReplaceQueryParameterVisitor.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -99,6 +101,7 @@ StorageView::StorageView(
|
||||
SelectQueryDescription description;
|
||||
|
||||
description.inner_query = query.select->ptr();
|
||||
is_parameterized_view = query.isParameterizedView();
|
||||
storage_metadata.setSelectQuery(description);
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
}
|
||||
@ -173,6 +176,15 @@ static ASTTableExpression * getFirstTableExpression(ASTSelectQuery & select_quer
|
||||
return select_element->table_expression->as<ASTTableExpression>();
|
||||
}
|
||||
|
||||
void StorageView::replaceQueryParameters(ASTPtr & outer_query, const NameToNameMap & parameter_values)
|
||||
{
|
||||
if (is_parameterized_view)
|
||||
{
|
||||
ReplaceQueryParameterVisitor visitor(parameter_values);
|
||||
visitor.visit(outer_query);
|
||||
}
|
||||
}
|
||||
|
||||
void StorageView::replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_query, ASTPtr & view_name)
|
||||
{
|
||||
ASTTableExpression * table_expression = getFirstTableExpression(outer_query);
|
||||
@ -185,8 +197,11 @@ void StorageView::replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_
|
||||
auto table_function_name = table_expression->table_function->as<ASTFunction>()->name;
|
||||
if (table_function_name == "view" || table_function_name == "viewIfPermitted")
|
||||
table_expression->database_and_table_name = std::make_shared<ASTTableIdentifier>("__view");
|
||||
if (table_function_name == "merge")
|
||||
else if (table_function_name == "merge")
|
||||
table_expression->database_and_table_name = std::make_shared<ASTTableIdentifier>("__merge");
|
||||
else
|
||||
table_expression->database_and_table_name = std::make_shared<ASTTableIdentifier>(table_function_name);
|
||||
|
||||
}
|
||||
if (!table_expression->database_and_table_name)
|
||||
throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -204,6 +219,8 @@ void StorageView::replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_
|
||||
for (auto & child : table_expression->children)
|
||||
if (child.get() == view_name.get())
|
||||
child = view_query;
|
||||
else if (child.get() && child->as<ASTFunction>() && child->as<ASTFunction>()->name == table_expression->table_function->as<ASTFunction>()->name)
|
||||
child = view_query;
|
||||
}
|
||||
|
||||
ASTPtr StorageView::restoreViewName(ASTSelectQuery & select_query, const ASTPtr & view_name)
|
||||
|
@ -19,6 +19,7 @@ public:
|
||||
|
||||
std::string getName() const override { return "View"; }
|
||||
bool isView() const override { return true; }
|
||||
bool isParameterizedView() const { return is_parameterized_view; }
|
||||
|
||||
/// It is passed inside the query and solved at its level.
|
||||
bool supportsSampling() const override { return true; }
|
||||
@ -34,6 +35,8 @@ public:
|
||||
size_t max_block_size,
|
||||
unsigned num_streams) override;
|
||||
|
||||
void replaceQueryParameters(ASTPtr & outer_query, const NameToNameMap & parameter_values);
|
||||
|
||||
static void replaceWithSubquery(ASTSelectQuery & select_query, ASTPtr & view_name, const StorageMetadataPtr & metadata_snapshot)
|
||||
{
|
||||
replaceWithSubquery(select_query, metadata_snapshot->getSelectQuery().inner_query->clone(), view_name);
|
||||
@ -41,6 +44,9 @@ public:
|
||||
|
||||
static void replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_query, ASTPtr & view_name);
|
||||
static ASTPtr restoreViewName(ASTSelectQuery & select_query, const ASTPtr & view_name);
|
||||
|
||||
protected:
|
||||
bool is_parameterized_view;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -0,0 +1,3 @@
|
||||
20
|
||||
50
|
||||
10
|
32
tests/queries/0_stateless/02428_parameterized_view.sql
Normal file
32
tests/queries/0_stateless/02428_parameterized_view.sql
Normal file
@ -0,0 +1,32 @@
|
||||
DROP TABLE IF EXISTS v1;
|
||||
DROP TABLE IF EXISTS Catalog;
|
||||
|
||||
CREATE TABLE Catalog (Name String, Price UInt64, Quantity UInt64) ENGINE = Memory;
|
||||
|
||||
INSERT INTO Catalog VALUES ('Pen', 10, 3);
|
||||
INSERT INTO Catalog VALUES ('Book', 50, 2);
|
||||
INSERT INTO Catalog VALUES ('Paper', 20, 1);
|
||||
|
||||
CREATE VIEW v1 AS SELECT * FROM Catalog WHERE Price={price:UInt64};
|
||||
SELECT Price FROM v1(price=20);
|
||||
|
||||
SELECT Price FROM v123(price=20); -- { serverError UNKNOWN_FUNCTION }
|
||||
|
||||
CREATE VIEW v10 AS SELECT * FROM Catalog WHERE Price=10;
|
||||
SELECT Price FROM v10(price=10); -- { serverError BAD_ARGUMENTS }
|
||||
|
||||
|
||||
CREATE VIEW v2 AS SELECT * FROM Catalog WHERE Price={price:UInt64} AND Quantity={quantity:UInt64};
|
||||
SELECT Price FROM v2(price=50,quantity=2);
|
||||
|
||||
SELECT Price FROM v2(price=50); -- { serverError UNKNOWN_QUERY_PARAMETER}
|
||||
|
||||
CREATE VIEW v3 AS SELECT * FROM Catalog WHERE Price={price:UInt64} AND Quantity=3;
|
||||
SELECT Price FROM v3(price=10);
|
||||
|
||||
CREATE VIEW v4 AS SELECT * FROM Catalog WHERE Price={price:UInt64} AND Quantity={price:UInt64}; -- {serverError BAD_ARGUMENTS}
|
||||
|
||||
DROP TABLE v1;
|
||||
DROP TABLE v2;
|
||||
DROP TABLE v3;
|
||||
DROP TABLE Catalog;
|
Loading…
Reference in New Issue
Block a user