mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
commit
8e2fba5be1
@ -20,6 +20,7 @@ namespace ErrorCodes
|
||||
extern const int TOO_DEEP_AST;
|
||||
extern const int CYCLIC_ALIASES;
|
||||
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 (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.
|
||||
size_t first_pos = 0;
|
||||
if (func_node->name == "lambda")
|
||||
|
@ -48,6 +48,7 @@ ASTPtr ASTFunction::clone() const
|
||||
auto res = std::make_shared<ASTFunction>(*this);
|
||||
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 (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_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?
|
||||
bool written = false;
|
||||
if (arguments && !parameters)
|
||||
|
@ -13,6 +13,7 @@ class ASTFunction : public ASTWithAlias
|
||||
{
|
||||
public:
|
||||
String name;
|
||||
ASTPtr query; // It's possible for a function to accept a query as its only argument.
|
||||
ASTPtr arguments;
|
||||
/// parameters - for parametric aggregate function. Example: quantile(0.9)(x) - what in first parens are 'parameters'.
|
||||
ASTPtr parameters;
|
||||
|
@ -18,6 +18,8 @@
|
||||
#include <Parsers/ASTQueryParameter.h>
|
||||
#include <Parsers/ASTTTLElement.h>
|
||||
#include <Parsers/ASTOrderByElement.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/ASTFunctionWithKeyValueArguments.h>
|
||||
|
||||
@ -217,10 +219,12 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
ParserIdentifier id_parser;
|
||||
ParserKeyword distinct("DISTINCT");
|
||||
ParserExpressionList contents(false);
|
||||
ParserSelectWithUnionQuery select;
|
||||
|
||||
bool has_distinct_modifier = false;
|
||||
|
||||
ASTPtr identifier;
|
||||
ASTPtr query;
|
||||
ASTPtr expr_list_args;
|
||||
ASTPtr expr_list_params;
|
||||
|
||||
@ -231,8 +235,36 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
return false;
|
||||
++pos;
|
||||
|
||||
|
||||
if (distinct.ignore(pos, expected))
|
||||
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;
|
||||
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);
|
||||
|
||||
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);
|
||||
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);
|
||||
registerTableFunctionJDBC(factory);
|
||||
|
||||
registerTableFunctionView(factory);
|
||||
|
||||
#if USE_MYSQL
|
||||
registerTableFunctionMySQL(factory);
|
||||
#endif
|
||||
|
@ -30,6 +30,8 @@ void registerTableFunctionHDFS(TableFunctionFactory & factory);
|
||||
void registerTableFunctionODBC(TableFunctionFactory & factory);
|
||||
void registerTableFunctionJDBC(TableFunctionFactory & factory);
|
||||
|
||||
void registerTableFunctionView(TableFunctionFactory & factory);
|
||||
|
||||
#if USE_MYSQL
|
||||
void registerTableFunctionMySQL(TableFunctionFactory & factory);
|
||||
#endif
|
||||
|
@ -21,6 +21,7 @@ SRCS(
|
||||
TableFunctionRemote.cpp
|
||||
TableFunctionURL.cpp
|
||||
TableFunctionValues.cpp
|
||||
TableFunctionView.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