Merge pull request #12567 from amosbird/fmi

table function view
This commit is contained in:
alexey-milovidov 2020-09-02 17:07:57 +03:00 committed by GitHub
commit 8e2fba5be1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 153 additions and 1 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View 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>();
}
}

View 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;
};
}

View File

@ -30,6 +30,8 @@ void registerTableFunctions()
registerTableFunctionODBC(factory);
registerTableFunctionJDBC(factory);
registerTableFunctionView(factory);
#if USE_MYSQL
registerTableFunctionMySQL(factory);
#endif

View File

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

View File

@ -21,6 +21,7 @@ SRCS(
TableFunctionRemote.cpp
TableFunctionURL.cpp
TableFunctionValues.cpp
TableFunctionView.cpp
TableFunctionZeros.cpp
)

View File

@ -0,0 +1,10 @@
1
1
SELECT `1`
FROM view(
SELECT 1
)
SELECT `1`
FROM remote(\'127.0.0.1\', view(
SELECT 1
))

View 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));