mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
TableFunction view.
This commit is contained in:
parent
73d433a7dd
commit
6bd753d85d
@ -20,6 +20,7 @@ namespace ErrorCodes
|
|||||||
extern const int TOO_DEEP_AST;
|
extern const int TOO_DEEP_AST;
|
||||||
extern const int CYCLIC_ALIASES;
|
extern const int CYCLIC_ALIASES;
|
||||||
extern const int UNKNOWN_QUERY_PARAMETER;
|
extern const int UNKNOWN_QUERY_PARAMETER;
|
||||||
|
extern const int BAD_ARGUMENTS;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -151,6 +152,13 @@ void QueryNormalizer::visitChildren(const ASTPtr & node, Data & data)
|
|||||||
{
|
{
|
||||||
if (const auto * func_node = node->as<ASTFunction>())
|
if (const auto * func_node = node->as<ASTFunction>())
|
||||||
{
|
{
|
||||||
|
if (func_node->query)
|
||||||
|
{
|
||||||
|
if (func_node->name != "view")
|
||||||
|
throw Exception("Query argument can only be used in the `view` TableFunction", ErrorCodes::BAD_ARGUMENTS);
|
||||||
|
/// Don't go into query argument.
|
||||||
|
return;
|
||||||
|
}
|
||||||
/// We skip the first argument. We also assume that the lambda function can not have parameters.
|
/// We skip the first argument. We also assume that the lambda function can not have parameters.
|
||||||
size_t first_pos = 0;
|
size_t first_pos = 0;
|
||||||
if (func_node->name == "lambda")
|
if (func_node->name == "lambda")
|
||||||
|
@ -48,6 +48,7 @@ ASTPtr ASTFunction::clone() const
|
|||||||
auto res = std::make_shared<ASTFunction>(*this);
|
auto res = std::make_shared<ASTFunction>(*this);
|
||||||
res->children.clear();
|
res->children.clear();
|
||||||
|
|
||||||
|
if (query) { res->query = query->clone(); res->children.push_back(res->query); }
|
||||||
if (arguments) { res->arguments = arguments->clone(); res->children.push_back(res->arguments); }
|
if (arguments) { res->arguments = arguments->clone(); res->children.push_back(res->arguments); }
|
||||||
if (parameters) { res->parameters = parameters->clone(); res->children.push_back(res->parameters); }
|
if (parameters) { res->parameters = parameters->clone(); res->children.push_back(res->parameters); }
|
||||||
|
|
||||||
@ -118,6 +119,18 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
|
|||||||
nested_need_parens.need_parens = true;
|
nested_need_parens.need_parens = true;
|
||||||
nested_dont_need_parens.need_parens = false;
|
nested_dont_need_parens.need_parens = false;
|
||||||
|
|
||||||
|
if (query)
|
||||||
|
{
|
||||||
|
std::string nl_or_nothing = settings.one_line ? "" : "\n";
|
||||||
|
std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' ');
|
||||||
|
settings.ostr << (settings.hilite ? hilite_function : "") << name << "(" << nl_or_nothing;
|
||||||
|
FormatStateStacked frame_nested = frame;
|
||||||
|
frame_nested.need_parens = false;
|
||||||
|
++frame_nested.indent;
|
||||||
|
query->formatImpl(settings, state, frame_nested);
|
||||||
|
settings.ostr << nl_or_nothing << indent_str << ")";
|
||||||
|
return;
|
||||||
|
}
|
||||||
/// Should this function to be written as operator?
|
/// Should this function to be written as operator?
|
||||||
bool written = false;
|
bool written = false;
|
||||||
if (arguments && !parameters)
|
if (arguments && !parameters)
|
||||||
|
@ -13,6 +13,7 @@ class ASTFunction : public ASTWithAlias
|
|||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
String name;
|
String name;
|
||||||
|
ASTPtr query; // It's possible for a function to accept a query as its only argument.
|
||||||
ASTPtr arguments;
|
ASTPtr arguments;
|
||||||
/// parameters - for parametric aggregate function. Example: quantile(0.9)(x) - what in first parens are 'parameters'.
|
/// parameters - for parametric aggregate function. Example: quantile(0.9)(x) - what in first parens are 'parameters'.
|
||||||
ASTPtr parameters;
|
ASTPtr parameters;
|
||||||
|
@ -18,6 +18,8 @@
|
|||||||
#include <Parsers/ASTQueryParameter.h>
|
#include <Parsers/ASTQueryParameter.h>
|
||||||
#include <Parsers/ASTTTLElement.h>
|
#include <Parsers/ASTTTLElement.h>
|
||||||
#include <Parsers/ASTOrderByElement.h>
|
#include <Parsers/ASTOrderByElement.h>
|
||||||
|
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||||
|
#include <Parsers/ASTSelectQuery.h>
|
||||||
#include <Parsers/ASTSubquery.h>
|
#include <Parsers/ASTSubquery.h>
|
||||||
#include <Parsers/ASTFunctionWithKeyValueArguments.h>
|
#include <Parsers/ASTFunctionWithKeyValueArguments.h>
|
||||||
|
|
||||||
@ -217,10 +219,12 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
|||||||
ParserIdentifier id_parser;
|
ParserIdentifier id_parser;
|
||||||
ParserKeyword distinct("DISTINCT");
|
ParserKeyword distinct("DISTINCT");
|
||||||
ParserExpressionList contents(false);
|
ParserExpressionList contents(false);
|
||||||
|
ParserSelectWithUnionQuery select;
|
||||||
|
|
||||||
bool has_distinct_modifier = false;
|
bool has_distinct_modifier = false;
|
||||||
|
|
||||||
ASTPtr identifier;
|
ASTPtr identifier;
|
||||||
|
ASTPtr query;
|
||||||
ASTPtr expr_list_args;
|
ASTPtr expr_list_args;
|
||||||
ASTPtr expr_list_params;
|
ASTPtr expr_list_params;
|
||||||
|
|
||||||
@ -231,8 +235,36 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
|||||||
return false;
|
return false;
|
||||||
++pos;
|
++pos;
|
||||||
|
|
||||||
|
|
||||||
if (distinct.ignore(pos, expected))
|
if (distinct.ignore(pos, expected))
|
||||||
has_distinct_modifier = true;
|
has_distinct_modifier = true;
|
||||||
|
else
|
||||||
|
{
|
||||||
|
auto old_pos = pos;
|
||||||
|
auto maybe_an_subquery = pos->type == TokenType::OpeningRoundBracket;
|
||||||
|
|
||||||
|
if (select.parse(pos, query, expected))
|
||||||
|
{
|
||||||
|
auto & select_ast = query->as<ASTSelectWithUnionQuery &>();
|
||||||
|
if (select_ast.list_of_selects->children.size() == 1 && maybe_an_subquery)
|
||||||
|
{
|
||||||
|
// It's an subquery. Bail out.
|
||||||
|
pos = old_pos;
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
if (pos->type != TokenType::ClosingRoundBracket)
|
||||||
|
return false;
|
||||||
|
++pos;
|
||||||
|
auto function_node = std::make_shared<ASTFunction>();
|
||||||
|
tryGetIdentifierNameInto(identifier, function_node->name);
|
||||||
|
function_node->query = query;
|
||||||
|
function_node->children.push_back(function_node->query);
|
||||||
|
node = function_node;
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
const char * contents_begin = pos->begin;
|
const char * contents_begin = pos->begin;
|
||||||
if (!contents.parse(pos, expr_list_args, expected))
|
if (!contents.parse(pos, expr_list_args, expected))
|
||||||
|
@ -104,7 +104,13 @@ void StorageView::replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_
|
|||||||
ASTTableExpression * table_expression = getFirstTableExpression(outer_query);
|
ASTTableExpression * table_expression = getFirstTableExpression(outer_query);
|
||||||
|
|
||||||
if (!table_expression->database_and_table_name)
|
if (!table_expression->database_and_table_name)
|
||||||
throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR);
|
{
|
||||||
|
// If it's a view table function, add a fake db.table name.
|
||||||
|
if (table_expression->table_function && table_expression->table_function->as<ASTFunction>()->name == "view")
|
||||||
|
table_expression->database_and_table_name = std::make_shared<ASTIdentifier>("__view");
|
||||||
|
else
|
||||||
|
throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
|
||||||
DatabaseAndTableWithAlias db_table(table_expression->database_and_table_name);
|
DatabaseAndTableWithAlias db_table(table_expression->database_and_table_name);
|
||||||
String alias = db_table.alias.empty() ? db_table.table : db_table.alias;
|
String alias = db_table.alias.empty() ? db_table.table : db_table.alias;
|
||||||
|
45
src/TableFunctions/TableFunctionView.cpp
Normal file
45
src/TableFunctions/TableFunctionView.cpp
Normal file
@ -0,0 +1,45 @@
|
|||||||
|
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||||
|
#include <Parsers/ASTCreateQuery.h>
|
||||||
|
#include <Parsers/ASTFunction.h>
|
||||||
|
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||||
|
#include <Storages/StorageView.h>
|
||||||
|
#include <TableFunctions/ITableFunction.h>
|
||||||
|
#include <TableFunctions/TableFunctionFactory.h>
|
||||||
|
#include <TableFunctions/TableFunctionView.h>
|
||||||
|
#include "registerTableFunctions.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int BAD_ARGUMENTS;
|
||||||
|
}
|
||||||
|
|
||||||
|
StoragePtr TableFunctionView::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const
|
||||||
|
{
|
||||||
|
if (const auto * function = ast_function->as<ASTFunction>())
|
||||||
|
{
|
||||||
|
if (function->query)
|
||||||
|
{
|
||||||
|
if (auto * select = function->query->as<ASTSelectWithUnionQuery>())
|
||||||
|
{
|
||||||
|
auto sample = InterpreterSelectWithUnionQuery::getSampleBlock(function->query, context);
|
||||||
|
auto columns = ColumnsDescription(sample.getNamesAndTypesList());
|
||||||
|
ASTCreateQuery create;
|
||||||
|
create.select = select;
|
||||||
|
auto res = StorageView::create(StorageID(getDatabaseName(), table_name), create, columns);
|
||||||
|
res->startup();
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
throw Exception("Table function '" + getName() + "' requires a query argument.", ErrorCodes::BAD_ARGUMENTS);
|
||||||
|
}
|
||||||
|
|
||||||
|
void registerTableFunctionView(TableFunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<TableFunctionView>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
27
src/TableFunctions/TableFunctionView.h
Normal file
27
src/TableFunctions/TableFunctionView.h
Normal file
@ -0,0 +1,27 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <TableFunctions/ITableFunction.h>
|
||||||
|
#include <Core/Types.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
/* view(query)
|
||||||
|
* Turning subquery into a table.
|
||||||
|
* Useful for passing subquery around.
|
||||||
|
*/
|
||||||
|
class TableFunctionView : public ITableFunction
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = "view";
|
||||||
|
std::string getName() const override { return name; }
|
||||||
|
private:
|
||||||
|
StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override;
|
||||||
|
const char * getStorageTypeName() const override { return "View"; }
|
||||||
|
|
||||||
|
UInt64 evaluateArgument(const Context & context, ASTPtr & argument) const;
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
}
|
@ -30,6 +30,8 @@ void registerTableFunctions()
|
|||||||
registerTableFunctionODBC(factory);
|
registerTableFunctionODBC(factory);
|
||||||
registerTableFunctionJDBC(factory);
|
registerTableFunctionJDBC(factory);
|
||||||
|
|
||||||
|
registerTableFunctionView(factory);
|
||||||
|
|
||||||
#if USE_MYSQL
|
#if USE_MYSQL
|
||||||
registerTableFunctionMySQL(factory);
|
registerTableFunctionMySQL(factory);
|
||||||
#endif
|
#endif
|
||||||
|
@ -30,6 +30,8 @@ void registerTableFunctionHDFS(TableFunctionFactory & factory);
|
|||||||
void registerTableFunctionODBC(TableFunctionFactory & factory);
|
void registerTableFunctionODBC(TableFunctionFactory & factory);
|
||||||
void registerTableFunctionJDBC(TableFunctionFactory & factory);
|
void registerTableFunctionJDBC(TableFunctionFactory & factory);
|
||||||
|
|
||||||
|
void registerTableFunctionView(TableFunctionFactory & factory);
|
||||||
|
|
||||||
#if USE_MYSQL
|
#if USE_MYSQL
|
||||||
void registerTableFunctionMySQL(TableFunctionFactory & factory);
|
void registerTableFunctionMySQL(TableFunctionFactory & factory);
|
||||||
#endif
|
#endif
|
||||||
|
@ -21,6 +21,7 @@ SRCS(
|
|||||||
TableFunctionRemote.cpp
|
TableFunctionRemote.cpp
|
||||||
TableFunctionURL.cpp
|
TableFunctionURL.cpp
|
||||||
TableFunctionValues.cpp
|
TableFunctionValues.cpp
|
||||||
|
TableFunctionView.cpp
|
||||||
TableFunctionZeros.cpp
|
TableFunctionZeros.cpp
|
||||||
|
|
||||||
)
|
)
|
||||||
|
@ -0,0 +1,10 @@
|
|||||||
|
1
|
||||||
|
1
|
||||||
|
SELECT `1`
|
||||||
|
FROM view(
|
||||||
|
SELECT 1
|
||||||
|
)
|
||||||
|
SELECT `1`
|
||||||
|
FROM remote(\'127.0.0.1\', view(
|
||||||
|
SELECT 1
|
||||||
|
))
|
5
tests/queries/0_stateless/01415_table_function_view.sql
Normal file
5
tests/queries/0_stateless/01415_table_function_view.sql
Normal file
@ -0,0 +1,5 @@
|
|||||||
|
SELECT * FROM view(SELECT 1);
|
||||||
|
SELECT * FROM remote('127.0.0.1', view(SELECT 1));
|
||||||
|
|
||||||
|
EXPLAIN SYNTAX SELECT * FROM view(SELECT 1);
|
||||||
|
EXPLAIN SYNTAX SELECT * FROM remote('127.0.0.1', view(SELECT 1));
|
Loading…
Reference in New Issue
Block a user